From fce1ca255df049ecde15e2de57010327798e53d9 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Wed, 20 Jan 2021 19:17:34 +0900 Subject: [PATCH 001/652] Implement sequenceNextNode --- .../AggregateFunctionNull.h | 11 +- .../AggregateFunctionSequenceNextNode.cpp | 102 ++++++ .../AggregateFunctionSequenceNextNode.h | 298 ++++++++++++++++++ src/AggregateFunctions/IAggregateFunction.h | 1 + .../registerAggregateFunctions.cpp | 2 + src/Columns/ColumnNullable.cpp | 1 - .../01656_sequence_next_node.reference | 120 +++++++ .../0_stateless/01656_sequence_next_node.sql | 51 +++ 8 files changed, 583 insertions(+), 3 deletions(-) create mode 100644 src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionSequenceNextNode.h create mode 100644 tests/queries/0_stateless/01656_sequence_next_node.reference create mode 100644 tests/queries/0_stateless/01656_sequence_next_node.sql diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 5c94e68cb26..3bfcacf7d7b 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -157,8 +157,15 @@ public: ColumnNullable & to_concrete = assert_cast(to); if (getFlag(place)) { - nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); + if (unlikely(nested_function->doesInsertResultNeedNullableColumn())) + { + nested_function->insertResultInto(nestedPlace(place), to_concrete, arena); + } + else + { + nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); + } } else { diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp new file mode 100644 index 00000000000..66f24ec8cbf --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerAggregateFunctions.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +template +inline AggregateFunctionPtr createAggregateFunctionSequenceNextNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) +{ + if (descending_order) + return std::make_shared>(data_type, argument_types); + else + return std::make_shared>(data_type, argument_types); +} + +AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) +{ + bool descending_order = false; + + if (parameters.size() == 1) + { + auto type = parameters[0].getType(); + if (type != Field::Types::Int64 && type != Field::Types::UInt64) + throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); + + descending_order = parameters[0].get(); + } + else + throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (argument_types.size() < 3) + throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > 2 + 64) + throw Exception("Aggregate function " + name + " requires at most 66(timestamp, value_column, 64 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (const auto i : ext::range(2, argument_types.size())) + { + const auto * cond_arg = argument_types[i].get(); + if (!isUInt8(cond_arg)) + throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + " of aggregate function " + + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) + throw Exception{"Illegal type " + argument_types.front().get()->getName() + + " of second argument of aggregate function " + name + ", must be String", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + DataTypePtr data_type; + if (typeid_cast(argument_types[1].get())) + data_type = argument_types[1]; + else + data_type = std::make_shared(argument_types[1]); + + WhichDataType timestamp_type(argument_types[0].get()); + if (timestamp_type.idx == TypeIndex::UInt8) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.idx == TypeIndex::UInt16) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.idx == TypeIndex::UInt32) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.idx == TypeIndex::UInt64) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.isDate()) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.isDateTime()) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + + throw Exception{"Illegal type " + argument_types.front().get()->getName() + + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; +} + +} + +void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) +{ + AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; + + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h new file mode 100644 index 00000000000..a455e16e267 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -0,0 +1,298 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include + +#include + + +namespace DB +{ + +template +struct NodeBase +{ + UInt64 size; // size of payload + + DataTypeDateTime::FieldType event_time; + UInt64 events_bitmap; + + /// Returns pointer to actual payload + char * data() { return reinterpret_cast(this) + sizeof(Node); } + + const char * data() const { return reinterpret_cast(this) + sizeof(Node); } + + /// Clones existing node (does not modify next field) + Node * clone(Arena * arena) const + { + return reinterpret_cast( + const_cast(arena->alignedInsert(reinterpret_cast(this), sizeof(Node) + size, alignof(Node)))); + } + + /// Write node to buffer + void write(WriteBuffer & buf) const + { + writeVarUInt(size, buf); + buf.write(data(), size); + } + + /// Reads and allocates node from ReadBuffer's data (doesn't set next) + static Node * read(ReadBuffer & buf, Arena * arena) + { + UInt64 size; + readVarUInt(size, buf); + + Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); + node->size = size; + buf.read(node->data(), size); + return node; + } +}; + +struct NodeString : public NodeBase +{ + using Node = NodeString; + + /// Create node from string + static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) + { + StringRef string = assert_cast(column).getDataAt(row_num); + + Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node))); + node->size = string.size; + memcpy(node->data(), string.data, string.size); + + return node; + } + + void insertInto(IColumn & column) + { + assert_cast(column).insertData(data(), size); + } +}; + +template +struct SequenceNextNodeGeneralData +{ + // Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena + using Allocator = MixedAlignedArenaAllocator; + using Array = PODArray; + + Array value; + bool sorted = false; + + struct Comparator final + { + bool operator()(const Node * lhs, const Node * rhs) const + { + if (Descending) + return lhs->event_time == rhs->event_time ? + lhs->events_bitmap < rhs->events_bitmap: lhs->event_time > rhs->event_time; + else + return lhs->event_time == rhs->event_time ? + lhs->events_bitmap < rhs->events_bitmap : lhs->event_time < rhs->event_time; + } + }; + + void sort() + { + if (!sorted) + { + std::stable_sort(std::begin(value), std::end(value), Comparator{}); + sorted = true; + } + } +}; + +/// Implementation of groupArray for String or any ComplexObject via Array +template +class SequenceNextNodeImpl final + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> +{ + using Data = SequenceNextNodeGeneralData; + static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } + static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } + + DataTypePtr & data_type; + UInt8 events_size; + UInt64 max_elems; + +public: + SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + {data_type_}, {}) + , data_type(this->argument_types[0]) + , events_size(arguments.size() - 2) + , max_elems(max_elems_) + { + } + + String getName() const override { return "sequenceNextNode"; } + + DataTypePtr getReturnType() const override { return data_type; } + + void insert(Data & a, const Node * v, Arena * arena) const + { + ++a.total_values; + a.value.push_back(v->clone(arena), arena); + } + + void create(AggregateDataPtr place) const override + { + [[maybe_unused]] auto a = new (place) Data; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + Node * node = Node::allocate(*columns[1], row_num, arena); + + const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; + + UInt64 events_bitmap = 0; + for (UInt8 i = 0; i < events_size; ++i) + if (assert_cast *>(columns[2 + i])->getData()[row_num]) + events_bitmap += (1 << i); + + node->event_time = timestamp; + node->events_bitmap = events_bitmap; + + data(place).value.push_back(node, arena); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + if (data(rhs).value.empty()) /// rhs state is empty + return; + + UInt64 new_elems; + if (data(place).value.size() >= max_elems) + return; + + new_elems = std::min(data(rhs).value.size(), static_cast(max_elems) - data(place).value.size()); + + auto & a = data(place).value; + const auto size = a.size(); + + auto & b = data(rhs).value; + for (UInt64 i = 0; i < new_elems; ++i) + a.push_back(b[i]->clone(arena), arena); + + using Comparator = typename SequenceNextNodeGeneralData::Comparator; + + /// either sort whole container or do so partially merging ranges afterwards + if (!data(place).sorted && !data(rhs).sorted) + std::stable_sort(std::begin(a), std::end(a), Comparator{}); + else + { + const auto begin = std::begin(a); + const auto middle = std::next(begin, size); + const auto end = std::end(a); + + if (!data(place).sorted) + std::stable_sort(begin, middle, Comparator{}); + + if (!data(rhs).sorted) + std::stable_sort(middle, end, Comparator{}); + + std::inplace_merge(begin, middle, end, Comparator{}); + } + + data(place).sorted = true; + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + writeVarUInt(data(place).value.size(), buf); + + auto & value = data(place).value; + for (auto & node : value) + node->write(buf); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + { + UInt64 elems; + readVarUInt(elems, buf); + + if (unlikely(elems == 0)) + return; + + auto & value = data(place).value; + + value.resize(elems, arena); + for (UInt64 i = 0; i < elems; ++i) + value[i] = Node::read(buf, arena); + } + + inline UInt64 getSkipCount(const Data & data, const UInt64 i, const UInt64 j) const + { + UInt64 k = 0; + for (; k < events_size - j; ++k) + if (data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j - k))) + return k; + return k; + } + + UInt64 getNextNodeIndex(Data & data) const + { + if (data.value.size() <= events_size) + return 0; + + data.sort(); + + UInt64 i = events_size - 1; + while (i < data.value.size()) + { + UInt64 j = 0; + for (; j < events_size; ++j) + if (!(data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j)))) + break; + + if (j == events_size) + return i + 1; + + i += getSkipCount(data, i, j); + } + + return 0; + } + + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + { + auto & value = data(place).value; + + UInt64 event_idx = getNextNodeIndex(this->data(place)); + if (event_idx != 0 && event_idx < value.size()) + { + ColumnNullable & to_concrete = assert_cast(to); + value[event_idx]->insertInto(to_concrete.getNestedColumn()); + to_concrete.getNullMapData().push_back(0); + } + else + to.insertDefault(); + } + + bool doesInsertResultNeedNullableColumn() const override { return true; } + + bool allocatesMemoryInArena() const override { return true; } +}; + +} diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index a9fe26688d7..d9570fa5f8b 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -112,6 +112,7 @@ public: /// in `runningAccumulate`, or when calculating an aggregate function as a /// window function. virtual void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const = 0; + virtual bool doesInsertResultNeedNullableColumn() const { return false; } /// Used for machine learning methods. Predict result from trained model. /// Will insert result into `to` column for rows in range [offset, offset + limit). diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index d8e4eb7ba98..28b758aee2c 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -44,6 +44,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &); void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); +void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -101,6 +102,7 @@ void registerAggregateFunctions() registerAggregateFunctionMannWhitney(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); + registerAggregateFunctionSequenceNextNode(factory); } { diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 35ce005073a..a9be62e515f 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -34,7 +34,6 @@ ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumn throw Exception{"ColumnNullable cannot have constant null map", ErrorCodes::ILLEGAL_COLUMN}; } - void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const { const auto & arr = getNullMapData(); diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference new file mode 100644 index 00000000000..540e5bdfb5a --- /dev/null +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -0,0 +1,120 @@ +(0, A) 1 B +(0, A) 2 B +(0, A) 3 B +(0, A) 4 A +(0, A) 5 B +(0, A) 6 B +(0, B) 1 C +(0, B) 2 D +(0, B) 3 \N +(0, B) 4 C +(0, B) 5 A +(0, B) 6 A +(0, C) 1 D +(0, C) 2 \N +(0, C) 3 \N +(0, C) 4 \N +(0, C) 5 \N +(0, C) 6 \N +(0, D) 1 \N +(0, D) 2 C +(0, D) 3 \N +(0, D) 4 \N +(0, D) 5 \N +(0, D) 6 \N +(0, E) 1 \N +(0, E) 2 \N +(0, E) 3 \N +(0, E) 4 \N +(0, E) 5 \N +(0, E) 6 \N +(1, A) 1 \N +(1, A) 2 \N +(1, A) 3 \N +(1, A) 4 A +(1, A) 5 B +(1, A) 6 B +(1, B) 1 A +(1, B) 2 A +(1, B) 3 A +(1, B) 4 A +(1, B) 5 A +(1, B) 6 A +(1, C) 1 B +(1, C) 2 D +(1, C) 3 \N +(1, C) 4 B +(1, C) 5 A +(1, C) 6 B +(1, D) 1 C +(1, D) 2 B +(1, D) 3 \N +(1, D) 4 \N +(1, D) 5 \N +(1, D) 6 \N +(1, E) 1 \N +(1, E) 2 \N +(1, E) 3 \N +(1, E) 4 \N +(1, E) 5 \N +(1, E) 6 \N +(0, A->B) 1 C +(0, A->B) 2 D +(0, A->B) 3 \N +(0, A->B) 4 C +(0, A->B) 5 A +(0, A->B) 6 A +(0, A->C) 1 \N +(0, A->C) 2 \N +(0, A->C) 3 \N +(0, A->C) 4 \N +(0, A->C) 5 \N +(0, A->C) 6 \N +(0, B->A) 1 \N +(0, B->A) 2 \N +(0, B->A) 3 \N +(0, B->A) 4 \N +(0, B->A) 5 C +(0, B->A) 6 B +(1, A->B) 1 \N +(1, A->B) 2 \N +(1, A->B) 3 \N +(1, A->B) 4 \N +(1, A->B) 5 A +(1, A->B) 6 A +(1, A->C) 1 \N +(1, A->C) 2 \N +(1, A->C) 3 \N +(1, A->C) 4 \N +(1, A->C) 5 \N +(1, A->C) 6 \N +(1, B->A) 1 \N +(1, B->A) 2 \N +(1, B->A) 3 \N +(1, B->A) 4 A +(1, B->A) 5 \N +(1, B->A) 6 B +(0, A->A->B) 1 \N +(0, A->A->B) 2 \N +(0, A->A->B) 3 \N +(0, A->A->B) 4 C +(0, A->A->B) 5 \N +(0, A->A->B) 6 \N +(0, B->A->A) 1 \N +(0, B->A->A) 2 \N +(0, B->A->A) 3 \N +(0, B->A->A) 4 \N +(0, B->A->A) 5 \N +(0, B->A->A) 6 \N +(1, A->A->B) 1 \N +(1, A->A->B) 2 \N +(1, A->A->B) 3 \N +(1, A->A->B) 4 \N +(1, A->A->B) 5 \N +(1, A->A->B) 6 \N +(1, B->A->A) 1 \N +(1, B->A->A) 2 \N +(1, B->A->A) 3 \N +(1, B->A->A) 4 A +(1, B->A->A) 5 \N +(1, B->A->A) 6 \N diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql new file mode 100644 index 00000000000..b11a5c7bc0e --- /dev/null +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -0,0 +1,51 @@ +DROP TABLE IF EXISTS test_sequenceNextNode; + +CREATE TABLE iF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',1,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',1,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',1,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',1,'D'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',2,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',2,'D'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',2,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',3,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',3,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',4,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',4,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',5,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',5,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',5,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',5,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',6,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',6,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); + +SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_sequenceNextNode; From e93caefd62c68dc34dce623b192f26483b642a05 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 12:00:37 +0900 Subject: [PATCH 002/652] Refactoring --- .../AggregateFunctionNull.h | 42 ++++++++--- .../AggregateFunctionSequenceNextNode.cpp | 4 +- .../AggregateFunctionSequenceNextNode.h | 71 ++++++++++--------- src/AggregateFunctions/IAggregateFunction.h | 1 - 4 files changed, 71 insertions(+), 47 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 3bfcacf7d7b..4dc3c580fd7 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -157,15 +157,8 @@ public: ColumnNullable & to_concrete = assert_cast(to); if (getFlag(place)) { - if (unlikely(nested_function->doesInsertResultNeedNullableColumn())) - { - nested_function->insertResultInto(nestedPlace(place), to_concrete, arena); - } - else - { - nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); - } + nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); } else { @@ -235,7 +228,7 @@ public: }; -template +template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> @@ -283,6 +276,35 @@ public: this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); } + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override + { + if constexpr (result_is_nullable) + { + ColumnNullable & to_concrete = assert_cast(to); + if (this->getFlag(place)) + { + if constexpr (insertion_requires_nullable_column) + { + this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete, arena); + } + else + { + this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); + } + } + else + { + to_concrete.insertDefault(); + } + } + else + { + this->nested_function->insertResultInto(this->nestedPlace(place), to, arena); + } + } + + private: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 66f24ec8cbf..b185859e00e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -49,8 +49,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & if (argument_types.size() < 3) throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > 2 + 64) - throw Exception("Aggregate function " + name + " requires at most 66(timestamp, value_column, 64 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > 2 + 32) + throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 32 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) { diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index a455e16e267..9bdd54e8b4b 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -11,7 +11,6 @@ #include #include -#include #include #include #include @@ -20,6 +19,7 @@ #include #include +#include #include @@ -33,28 +33,24 @@ struct NodeBase UInt64 size; // size of payload DataTypeDateTime::FieldType event_time; - UInt64 events_bitmap; + UInt32 events_bitset; // UInt32 for combiniant comparesons between bitsets (< operator on bitsets). - /// Returns pointer to actual payload char * data() { return reinterpret_cast(this) + sizeof(Node); } const char * data() const { return reinterpret_cast(this) + sizeof(Node); } - /// Clones existing node (does not modify next field) Node * clone(Arena * arena) const { return reinterpret_cast( const_cast(arena->alignedInsert(reinterpret_cast(this), sizeof(Node) + size, alignof(Node)))); } - /// Write node to buffer void write(WriteBuffer & buf) const { writeVarUInt(size, buf); buf.write(data(), size); } - /// Reads and allocates node from ReadBuffer's data (doesn't set next) static Node * read(ReadBuffer & buf, Arena * arena) { UInt64 size; @@ -71,7 +67,6 @@ struct NodeString : public NodeBase { using Node = NodeString; - /// Create node from string static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) { StringRef string = assert_cast(column).getDataAt(row_num); @@ -92,7 +87,6 @@ struct NodeString : public NodeBase template struct SequenceNextNodeGeneralData { - // Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena using Allocator = MixedAlignedArenaAllocator; using Array = PODArray; @@ -103,12 +97,12 @@ struct SequenceNextNodeGeneralData { bool operator()(const Node * lhs, const Node * rhs) const { - if (Descending) + if constexpr (Descending) return lhs->event_time == rhs->event_time ? - lhs->events_bitmap < rhs->events_bitmap: lhs->event_time > rhs->event_time; + lhs->events_bitset < rhs->events_bitset: lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? - lhs->events_bitmap < rhs->events_bitmap : lhs->event_time < rhs->event_time; + lhs->events_bitset < rhs->events_bitset : lhs->event_time < rhs->event_time; } }; @@ -122,7 +116,6 @@ struct SequenceNextNodeGeneralData } }; -/// Implementation of groupArray for String or any ComplexObject via Array template class SequenceNextNodeImpl final : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> @@ -149,6 +142,18 @@ public: DataTypePtr getReturnType() const override { return data_type; } + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties & /*properties*/) const override + { + // This aggregate function sets insertion_requires_nullable_column on. + // Even though some values are mapped to aggregating key, it could return nulls for the below case. + // aggregated events: [A -> B -> C] + // events to find: [C -> D] + // [C -> D] is not matched to 'A -> B -> C' so that it returns null. + return std::make_shared>(nested_function, arguments, params); + } + void insert(Data & a, const Node * v, Arena * arena) const { ++a.total_values; @@ -166,44 +171,42 @@ public: const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; - UInt64 events_bitmap = 0; + UInt32 events_bitset = 0; for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) - events_bitmap += (1 << i); + events_bitset += (1 << i); node->event_time = timestamp; - node->events_bitmap = events_bitmap; + node->events_bitset = events_bitset; data(place).value.push_back(node, arena); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override { - if (data(rhs).value.empty()) /// rhs state is empty + if (data(rhs).value.empty()) return; - UInt64 new_elems; if (data(place).value.size() >= max_elems) return; - new_elems = std::min(data(rhs).value.size(), static_cast(max_elems) - data(place).value.size()); - auto & a = data(place).value; - const auto size = a.size(); - auto & b = data(rhs).value; + const auto a_size = a.size(); + + const UInt64 new_elems = std::min(data(rhs).value.size(), static_cast(max_elems) - data(place).value.size()); for (UInt64 i = 0; i < new_elems; ++i) a.push_back(b[i]->clone(arena), arena); + /// either sort whole container or do so partially merging ranges afterwards using Comparator = typename SequenceNextNodeGeneralData::Comparator; - /// either sort whole container or do so partially merging ranges afterwards if (!data(place).sorted && !data(rhs).sorted) std::stable_sort(std::begin(a), std::end(a), Comparator{}); else { const auto begin = std::begin(a); - const auto middle = std::next(begin, size); + const auto middle = std::next(begin, a_size); const auto end = std::end(a); if (!data(place).sorted) @@ -242,34 +245,36 @@ public: value[i] = Node::read(buf, arena); } - inline UInt64 getSkipCount(const Data & data, const UInt64 i, const UInt64 j) const + inline UInt32 calculateJump(const Data & data, const UInt32 i, const UInt32 j) const { - UInt64 k = 0; + UInt32 k = 0; for (; k < events_size - j; ++k) - if (data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j - k))) + if (data.value[i - j]->events_bitset & (1 << (events_size - 1 - j - k))) return k; return k; } - UInt64 getNextNodeIndex(Data & data) const + // This method returns an index of next node that matched the events. + // It is one as referring Boyer-Moore-Algorithm. + UInt32 getNextNodeIndex(Data & data) const { if (data.value.size() <= events_size) return 0; data.sort(); - UInt64 i = events_size - 1; + UInt32 i = events_size - 1; while (i < data.value.size()) { - UInt64 j = 0; + UInt32 j = 0; for (; j < events_size; ++j) - if (!(data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j)))) + if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) break; if (j == events_size) return i + 1; - i += getSkipCount(data, i, j); + i += calculateJump(data, i, j); } return 0; @@ -279,7 +284,7 @@ public: { auto & value = data(place).value; - UInt64 event_idx = getNextNodeIndex(this->data(place)); + UInt32 event_idx = getNextNodeIndex(this->data(place)); if (event_idx != 0 && event_idx < value.size()) { ColumnNullable & to_concrete = assert_cast(to); @@ -290,8 +295,6 @@ public: to.insertDefault(); } - bool doesInsertResultNeedNullableColumn() const override { return true; } - bool allocatesMemoryInArena() const override { return true; } }; diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index d9570fa5f8b..a9fe26688d7 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -112,7 +112,6 @@ public: /// in `runningAccumulate`, or when calculating an aggregate function as a /// window function. virtual void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const = 0; - virtual bool doesInsertResultNeedNullableColumn() const { return false; } /// Used for machine learning methods. Predict result from trained model. /// Will insert result into `to` column for rows in range [offset, offset + limit). From 994e01e0abae2556cb76615c529e2450ac89abb4 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 16:11:32 +0900 Subject: [PATCH 003/652] Serialization --- .../AggregateFunctionSequenceNextNode.h | 15 +++++++++------ src/AggregateFunctions/ya.make | 1 + 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 9bdd54e8b4b..0a6f686e9bc 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -223,25 +223,28 @@ public: void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { - writeVarUInt(data(place).value.size(), buf); + writeBinary(data(place).sorted, buf); auto & value = data(place).value; + writeVarUInt(value.size(), buf); for (auto & node : value) node->write(buf); } void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override { - UInt64 elems; - readVarUInt(elems, buf); + readBinary(data(place).sorted, buf); - if (unlikely(elems == 0)) + UInt64 size; + readVarUInt(size, buf); + + if (unlikely(size == 0)) return; auto & value = data(place).value; - value.resize(elems, arena); - for (UInt64 i = 0; i < elems; ++i) + value.resize(size, arena); + for (UInt64 i = 0; i < size; ++i) value[i] = Node::read(buf, arena); } diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make index f2105688feb..5af62a68396 100644 --- a/src/AggregateFunctions/ya.make +++ b/src/AggregateFunctions/ya.make @@ -42,6 +42,7 @@ SRCS( AggregateFunctionResample.cpp AggregateFunctionRetention.cpp AggregateFunctionSequenceMatch.cpp + AggregateFunctionSequenceNextNode.cpp AggregateFunctionSimpleLinearRegression.cpp AggregateFunctionSimpleState.cpp AggregateFunctionState.cpp From 7ca8811216bae8ed3b5ccff11d84d3fc7f2bbb33 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 17:14:56 +0900 Subject: [PATCH 004/652] Fix --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 0a6f686e9bc..44a2018980a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -49,6 +49,9 @@ struct NodeBase { writeVarUInt(size, buf); buf.write(data(), size); + + writeBinary(event_time, buf); + writeBinary(events_bitset, buf); } static Node * read(ReadBuffer & buf, Arena * arena) @@ -59,6 +62,10 @@ struct NodeBase Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; buf.read(node->data(), size); + + readBinary(node->event_time, buf); + readBinary(node->events_bitset, buf); + return node; } }; From 05c3767d05c42440610e9a387b95b3be1eae5d81 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 18:55:33 +0900 Subject: [PATCH 005/652] Fix --- .../AggregateFunctionSequenceNextNode.cpp | 4 +- .../AggregateFunctionSequenceNextNode.h | 2 +- .../01656_sequence_next_node.reference | 132 ++++++++++++++++++ .../0_stateless/01656_sequence_next_node.sql | 119 +++++++++++++--- 4 files changed, 233 insertions(+), 24 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index b185859e00e..47279aa882d 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -94,9 +94,7 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { - AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 44a2018980a..3fbb481c19a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -158,7 +158,7 @@ public: // aggregated events: [A -> B -> C] // events to find: [C -> D] // [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 540e5bdfb5a..ecb39686bfa 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -118,3 +118,135 @@ (1, B->A->A) 4 A (1, B->A->A) 5 \N (1, B->A->A) 6 \N +(0, A) id >= 10 10 B +(0, A) id >= 10 10 C +(0, A) id >= 10 10 D +(0, A) id >= 10 10 B +(0, A) id >= 10 10 B +(0, A) id >= 10 10 A +(0, A) 1 B +(0, A) 2 B +(0, A) 3 B +(0, A) 4 A +(0, A) 5 B +(0, A) 6 B +(0, B) 1 C +(0, B) 2 D +(0, B) 3 \N +(0, B) 4 C +(0, B) 5 A +(0, B) 6 A +(0, C) 1 D +(0, C) 2 \N +(0, C) 3 \N +(0, C) 4 \N +(0, C) 5 \N +(0, C) 6 \N +(0, D) 1 \N +(0, D) 2 C +(0, D) 3 \N +(0, D) 4 \N +(0, D) 5 \N +(0, D) 6 \N +(0, E) 1 \N +(0, E) 2 \N +(0, E) 3 \N +(0, E) 4 \N +(0, E) 5 \N +(0, E) 6 \N +(1, A) 1 \N +(1, A) 2 \N +(1, A) 3 \N +(1, A) 4 A +(1, A) 5 B +(1, A) 6 B +(1, B) 1 A +(1, B) 2 A +(1, B) 3 A +(1, B) 4 A +(1, B) 5 A +(1, B) 6 A +(1, C) 1 B +(1, C) 2 D +(1, C) 3 \N +(1, C) 4 B +(1, C) 5 A +(1, C) 6 B +(1, D) 1 C +(1, D) 2 B +(1, D) 3 \N +(1, D) 4 \N +(1, D) 5 \N +(1, D) 6 \N +(1, E) 1 \N +(1, E) 2 \N +(1, E) 3 \N +(1, E) 4 \N +(1, E) 5 \N +(1, E) 6 \N +(0, A->B) 1 C +(0, A->B) 2 D +(0, A->B) 3 \N +(0, A->B) 4 C +(0, A->B) 5 A +(0, A->B) 6 A +(0, A->C) 1 \N +(0, A->C) 2 \N +(0, A->C) 3 \N +(0, A->C) 4 \N +(0, A->C) 5 \N +(0, A->C) 6 \N +(0, B->A) 1 \N +(0, B->A) 2 \N +(0, B->A) 3 \N +(0, B->A) 4 \N +(0, B->A) 5 C +(0, B->A) 6 B +(1, A->B) 1 \N +(1, A->B) 2 \N +(1, A->B) 3 \N +(1, A->B) 4 \N +(1, A->B) 5 A +(1, A->B) 6 A +(1, A->C) 1 \N +(1, A->C) 2 \N +(1, A->C) 3 \N +(1, A->C) 4 \N +(1, A->C) 5 \N +(1, A->C) 6 \N +(1, B->A) 1 \N +(1, B->A) 2 \N +(1, B->A) 3 \N +(1, B->A) 4 A +(1, B->A) 5 \N +(1, B->A) 6 B +(0, A->A->B) 1 \N +(0, A->A->B) 2 \N +(0, A->A->B) 3 \N +(0, A->A->B) 4 C +(0, A->A->B) 5 \N +(0, A->A->B) 6 \N +(0, B->A->A) 1 \N +(0, B->A->A) 2 \N +(0, B->A->A) 3 \N +(0, B->A->A) 4 \N +(0, B->A->A) 5 \N +(0, B->A->A) 6 \N +(1, A->A->B) 1 \N +(1, A->A->B) 2 \N +(1, A->A->B) 3 \N +(1, A->A->B) 4 \N +(1, A->A->B) 5 \N +(1, A->A->B) 6 \N +(1, B->A->A) 1 \N +(1, B->A->A) 2 \N +(1, B->A->A) 3 \N +(1, B->A->A) 4 A +(1, B->A->A) 5 \N +(1, B->A->A) 6 \N +(0, A) id >= 10 10 B +(0, A) id >= 10 10 C +(0, A) id >= 10 10 D +(0, A) id >= 10 10 B +(0, A) id >= 10 10 B +(0, A) id >= 10 10 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index b11a5c7bc0e..e5a89da7341 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -1,3 +1,70 @@ +DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; + +CREATE TABLE iF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',1,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',1,'D'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',2,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',2,'D'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',2,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',3,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',3,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',4,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',4,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',5,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',5,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',5,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',5,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',6,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',6,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); + +SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); + +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; + +-- The same testcases for a non-null type. + DROP TABLE IF EXISTS test_sequenceNextNode; CREATE TABLE iF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; @@ -27,25 +94,37 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); + +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode; From e092eeee7a3085026e699e7d27b6779b71a8fd0b Mon Sep 17 00:00:00 2001 From: achimbab Date: Fri, 29 Jan 2021 08:51:57 +0900 Subject: [PATCH 006/652] Documentation for sequenceNextNode --- .../parametric-functions.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 3b02e145ff4..b371cbcb9fa 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -495,3 +495,56 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= ## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys. + +## sequenceNextNode {#sequenceNextNode} + +Returns a value of next event that matched an event chain. + +**Syntax** + +``` sql +sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, ... event32) +``` + +**Parameters** +- `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. +- `timestamp` — Name of the column containing the timestamp. Data types supported: Date, DateTime and other unsigned integer types. +- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: String and Nullable(String) +- `cond` — Conditions or data describing the chain of events. UInt8. + +**Returned value** + +Nullable(String). + +- event_column[next_index] - if the pattern is matched and next value exists. +- null - if the pattern isn’t matched or next value doesn't exist. + +Type: `Nullable(String)`. + +**Example** + +It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E. + +The query statement searching the event following B->C : + +``` sql +CREATE TABLE test_flow ( + dt DateTime, + id int, + action String) +ENGINE = MergeTree() +PARTITION BY toYYYYMMDD(dt) +ORDER BY id; + +INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); + +SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') as next_flow FROM test_flow GROUP BY id; +``` + +Result: + +``` text +┌─id─┬─next_flow─┐ +│ 1 │ E │ +└────┴───────────┘ +``` From 138c5ab64476fc7c071214515a7973bf61f2add8 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 29 Jan 2021 10:33:42 +0900 Subject: [PATCH 007/652] For some functional stateless tests (ubsan) --- src/AggregateFunctions/AggregateFunctionNull.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 4dc3c580fd7..adf2aed1e82 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -231,12 +231,12 @@ public: template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> + AggregateFunctionNullVariadic> { public: AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), + AggregateFunctionNullVariadic>(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) From 753c32ee13706d34c10cb11e455d0d864dd1ef96 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 1 Feb 2021 14:01:18 +0900 Subject: [PATCH 008/652] Use 0x80000000 as unmatched event --- .../AggregateFunctionSequenceNextNode.cpp | 4 +-- .../AggregateFunctionSequenceNextNode.h | 3 +- .../01656_sequence_next_node.reference | 10 +++--- .../0_stateless/01656_sequence_next_node.sql | 35 +++++++++++++++---- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 47279aa882d..adfa1592c86 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -49,8 +49,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & if (argument_types.size() < 3) throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > 2 + 32) - throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 32 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > 2 + 31) + throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 31 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) { diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 3fbb481c19a..85413513d50 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -106,7 +106,7 @@ struct SequenceNextNodeGeneralData { if constexpr (Descending) return lhs->event_time == rhs->event_time ? - lhs->events_bitset < rhs->events_bitset: lhs->event_time > rhs->event_time; + lhs->events_bitset < rhs->events_bitset : lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? lhs->events_bitset < rhs->events_bitset : lhs->event_time < rhs->event_time; @@ -182,6 +182,7 @@ public: for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) events_bitset += (1 << i); + if (events_bitset == 0) events_bitset = 0x80000000; // Any events are not matched. node->event_time = timestamp; node->events_bitset = events_bitset; diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index ecb39686bfa..b0689ff136c 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -245,8 +245,8 @@ (1, B->A->A) 5 \N (1, B->A->A) 6 \N (0, A) id >= 10 10 B -(0, A) id >= 10 10 C -(0, A) id >= 10 10 D -(0, A) id >= 10 10 B -(0, A) id >= 10 10 B -(0, A) id >= 10 10 A +(0, A->B) id >= 10 10 C +(0, B->C) id >= 10 10 D +(0, C->) id >= 10 10 B +(0, D->C) id >= 10 10 B +(0, C->B) id >= 10 10 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index e5a89da7341..55d98e93342 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -61,6 +61,15 @@ SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS n SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); + DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; -- The same testcases for a non-null type. @@ -120,11 +129,25 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, A->B) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, B->C) id = 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, D->C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, C->B) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); + +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); DROP TABLE IF EXISTS test_sequenceNextNode; From 03ad81ee726cb1bb2bc90c93499c1e78812b0127 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 1 Feb 2021 15:26:16 +0900 Subject: [PATCH 009/652] Correct testcases --- .../01656_sequence_next_node.reference | 28 +++++++++++++++---- .../0_stateless/01656_sequence_next_node.sql | 16 +++++------ 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index b0689ff136c..3e8da4bbd48 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -124,6 +124,14 @@ (0, A) id >= 10 10 B (0, A) id >= 10 10 B (0, A) id >= 10 10 A +(0, A) id = 11 0 +(0, C) id = 11 0 +(0, B->C) id = 11 0 +(0, A->B->C) id = 11 0 +(0, A) id = 11 0 +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 0 (0, A) 1 B (0, A) 2 B (0, A) 3 B @@ -244,9 +252,17 @@ (1, B->A->A) 4 A (1, B->A->A) 5 \N (1, B->A->A) 6 \N -(0, A) id >= 10 10 B -(0, A->B) id >= 10 10 C -(0, B->C) id >= 10 10 D -(0, C->) id >= 10 10 B -(0, D->C) id >= 10 10 B -(0, C->B) id >= 10 10 A +(0, A) id = 10 10 B +(0, A->B) id = 10 10 C +(0, B->C) id = 10 10 D +(0, C) id = 10 10 B +(0, D->C) id = 10 10 B +(0, C->B) id = 10 10 A +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, B->C) id = 11 1 +(0, A->B->C) id = 11 1 +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, C->B) id = 11 1 +(0, C->B->A) id = 11 1 diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 55d98e93342..31c224fd2a4 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -61,14 +61,14 @@ SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS n SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; From 19dd09ea8ea0471a25de5b86003b70dc9c2c84bf Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Tue, 2 Feb 2021 17:50:08 +0900 Subject: [PATCH 010/652] Fix paramters for nulls --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp | 3 ++- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index adfa1592c86..57ba87c922f 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -94,7 +94,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode }); + AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 85413513d50..888149c77da 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -158,7 +158,7 @@ public: // aggregated events: [A -> B -> C] // events to find: [C -> D] // [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const From 2cc69893f26920aa41c5da497035b46e6ce67360 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 4 Feb 2021 16:15:04 +0900 Subject: [PATCH 011/652] Add sequenceFirstNode --- .../AggregateFunctionSequenceNextNode.cpp | 43 ++++-- .../AggregateFunctionSequenceNextNode.h | 135 ++++++++++++++++++ .../01656_sequence_next_node.reference | 48 +++++++ .../0_stateless/01656_sequence_next_node.sql | 10 ++ 4 files changed, 221 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 57ba87c922f..af90c80de61 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -23,15 +23,27 @@ namespace { template -inline AggregateFunctionPtr createAggregateFunctionSequenceNextNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) { - if (descending_order) - return std::make_shared>(data_type, argument_types); + if (argument_types.size() == 2) + { + // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. + if (descending_order) + return std::make_shared>(data_type); + else + return std::make_shared>(data_type); + } else - return std::make_shared>(data_type, argument_types); + { + if (descending_order) + return std::make_shared>(data_type, argument_types); + else + return std::make_shared>(data_type, argument_types); + } } -AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) +template +AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) { bool descending_order = false; @@ -47,9 +59,9 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (argument_types.size() < 3) - throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > 2 + 31) + if (argument_types.size() < 2) + throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > MaxArgs) throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 31 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) @@ -73,17 +85,17 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.idx == TypeIndex::UInt16) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.idx == TypeIndex::UInt32) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.idx == TypeIndex::UInt64) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.isDate()) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.isDateTime()) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); throw Exception{"Illegal type " + argument_types.front().get()->getName() + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", @@ -95,7 +107,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode<2 + 31>, properties }); + factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNode<2>, properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 888149c77da..ffcc02b805a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -309,4 +309,139 @@ public: bool allocatesMemoryInArena() const override { return true; } }; +template +class SequenceFirstNodeImpl final + : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> +{ + using Data = SequenceNextNodeGeneralData; + static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } + static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } + + DataTypePtr & data_type; + +public: + SequenceFirstNodeImpl(const DataTypePtr & data_type_) + : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( + {data_type_}, {}) + , data_type(this->argument_types[0]) + { + } + + String getName() const override { return "sequenceFirstNode"; } + + DataTypePtr getReturnType() const override { return data_type; } + + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties &) const override + { + return std::make_shared>(nested_function, arguments, params); + } + + void insert(Data & a, const Node * v, Arena * arena) const + { + ++a.total_values; + a.value.push_back(v->clone(arena), arena); + } + + void create(AggregateDataPtr place) const override + { + [[maybe_unused]] auto a = new (place) Data; + } + + bool compare(const T lhs_timestamp, const T rhs_timestamp) const + { + return Descending ? lhs_timestamp < rhs_timestamp : lhs_timestamp > rhs_timestamp; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + bool is_first = true; + auto & value = data(place).value; + const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; + + if (value.size() != 0) + { + if (compare(value[0]->event_time, timestamp)) + value.pop_back(); + else + is_first = false; + } + + + if (is_first) + { + Node * node = Node::allocate(*columns[1], row_num, arena); + node->event_time = timestamp; + node->events_bitset = 0x80000000; + + data(place).value.push_back(node, arena); + } + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & a = data(place).value; + auto & b = data(rhs).value; + + if (b.empty()) + return; + + if (a.empty()) + { + a.push_back(b[0]->clone(arena), arena); + return; + } + + if (compare(a[0]->event_time, b[0]->event_time)) + { + data(place).value.pop_back(); + a.push_back(b[0]->clone(arena), arena); + } + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + writeBinary(data(place).sorted, buf); + + auto & value = data(place).value; + writeVarUInt(value.size(), buf); + for (auto & node : value) + node->write(buf); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + { + readBinary(data(place).sorted, buf); + + UInt64 size; + readVarUInt(size, buf); + + if (unlikely(size == 0)) + return; + + auto & value = data(place).value; + + value.resize(size, arena); + for (UInt64 i = 0; i < size; ++i) + value[i] = Node::read(buf, arena); + } + + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + { + auto & value = data(place).value; + + if (value.size() > 0) + { + ColumnNullable & to_concrete = assert_cast(to); + value[0]->insertInto(to_concrete.getNestedColumn()); + to_concrete.getNullMapData().push_back(0); + } + else + to.insertDefault(); + } + + bool allocatesMemoryInArena() const override { return true; } +}; + } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 3e8da4bbd48..50755232cb9 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -132,6 +132,30 @@ (0, C) id = 11 0 (0, C->B) id = 11 0 (0, C->B->A) id = 11 0 +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C (0, A) 1 B (0, A) 2 B (0, A) 3 B @@ -266,3 +290,27 @@ (0, C) id = 11 1 (0, C->B) id = 11 1 (0, C->B->A) id = 11 1 +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 31c224fd2a4..9af59d5c8e2 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -70,6 +70,11 @@ SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; + DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; -- The same testcases for a non-null type. @@ -150,4 +155,9 @@ SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; + DROP TABLE IF EXISTS test_sequenceNextNode; From ac149592c687646992df1cb0ddc19f0ab169e55a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 11 Feb 2021 19:05:04 +0300 Subject: [PATCH 012/652] wip --- .../common.py | 87 ++++++++++--------- .../test_cassandra.py | 16 ++-- .../test_clickhouse_remote.py | 28 +++--- 3 files changed, 62 insertions(+), 69 deletions(-) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index ef6d133893a..a3d0e8a019b 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -1,4 +1,5 @@ import os +import shutil from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout @@ -89,34 +90,54 @@ VALUES = { SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/dictionaries') +DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs', 'dictionaries') -def get_dict(source, layout, fields, suffix_name=''): - global DICT_CONFIG_PATH - structure = DictionaryStructure(layout, fields) - dict_name = source.name + "_" + layout.name + '_' + suffix_name - dict_path = os.path.join(DICT_CONFIG_PATH, dict_name + '.xml') - dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) - dictionary.generate_config() - return dictionary +class BaseLayoutTester: + def __init__(self, test_name): + self.test_name = test_name + self.layouts = [] -class SimpleLayoutTester: - def __init__(self): - self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] - self.values = VALUES["simple"] - self.data = [Row(self.fields, vals) for vals in self.values] - self.layout_to_dictionary = dict() + def get_dict_directory(self): + return os.path.join(DICT_CONFIG_PATH, self.test_name) + + def cleanup(self): + shutil.rmtree(self.get_dict_directory(), ignore_errors=True) + os.makedirs(self.get_dict_directory()) + + def list_dictionaries(self): + dictionaries = [] + directory = self.get_dict_directory() + for fname in os.listdir(directory): + dictionaries.append(os.path.join(directory, fname)) + return dictionaries def create_dictionaries(self, source_): - for layout in LAYOUTS_SIMPLE: + for layout in self.layouts: if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) + self.layout_to_dictionary[layout] = self.get_dict(source_, Layout(layout), self.fields) def prepare(self, cluster_): for _, dictionary in list(self.layout_to_dictionary.items()): dictionary.prepare_source(cluster_) dictionary.load_data(self.data) + def get_dict(self, source, layout, fields, suffix_name=''): + structure = DictionaryStructure(layout, fields) + dict_name = source.name + "_" + layout.name + '_' + suffix_name + dict_path = os.path.join(self.get_dict_directory(), dict_name + '.xml') + dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) + dictionary.generate_config() + return dictionary + +class SimpleLayoutTester(BaseLayoutTester): + def __init__(self, test_name): + self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] + self.values = VALUES["simple"] + self.data = [Row(self.fields, vals) for vals in self.values] + self.layout_to_dictionary = dict() + self.test_name = test_name + self.layouts = LAYOUTS_SIMPLE + def execute(self, layout_name, node): if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) @@ -157,22 +178,14 @@ class SimpleLayoutTester: assert node.query(query) == str(answer) + '\n' -class ComplexLayoutTester: - def __init__(self): +class ComplexLayoutTester(BaseLayoutTester): + def __init__(self, test_name): self.fields = KEY_FIELDS["complex"] + START_FIELDS["complex"] + MIDDLE_FIELDS + END_FIELDS["complex"] self.values = VALUES["complex"] self.data = [Row(self.fields, vals) for vals in self.values] self.layout_to_dictionary = dict() - - def create_dictionaries(self, source_): - for layout in LAYOUTS_COMPLEX: - if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) - - def prepare(self, cluster_): - for _, dictionary in list(self.layout_to_dictionary.items()): - dictionary.prepare_source(cluster_) - dictionary.load_data(self.data) + self.test_name = test_name + self.layouts = LAYOUTS_COMPLEX def execute(self, layout_name, node): if layout_name not in self.layout_to_dictionary: @@ -200,22 +213,14 @@ class ComplexLayoutTester: assert node.query(query) == str(answer) + '\n' -class RangedLayoutTester: - def __init__(self): +class RangedLayoutTester(BaseLayoutTester): + def __init__(self, test_name): self.fields = KEY_FIELDS["ranged"] + START_FIELDS["ranged"] + MIDDLE_FIELDS + END_FIELDS["ranged"] self.values = VALUES["ranged"] self.data = [Row(self.fields, vals) for vals in self.values] self.layout_to_dictionary = dict() - - def create_dictionaries(self, source_): - for layout in LAYOUTS_RANGED: - if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) - - def prepare(self, cluster_): - for _, dictionary in list(self.layout_to_dictionary.items()): - dictionary.prepare_source(cluster_) - dictionary.load_data(self.data) + self.test_name = test_name + self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 81f9db1964b..8154cc55453 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -15,7 +15,7 @@ node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "cassandra" def setup_module(module): global cluster @@ -24,10 +24,9 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) simple_tester = SimpleLayoutTester() + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) complex_tester = ComplexLayoutTester() @@ -37,24 +36,19 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py index 374e620e1c3..3ed335a1987 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceClickHouse -SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse1", "9000", "default", "") +SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse_remote", "9000", "default", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "remote" def setup_module(module): global cluster @@ -24,37 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_remote', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('remote_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + simple_tester.cleanup() @pytest.fixture(scope="module") From 581e080b3532807dfd74186feaa5d462eebfec7f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 12 Feb 2021 18:37:29 +0300 Subject: [PATCH 013/652] wip --- .../test_cassandra.py | 8 ++--- .../test_clickhouse_local.py | 30 ++++++++----------- .../test_executable_cache.py | 29 +++++++----------- .../test_executable_hashed.py | 28 +++++++---------- .../test_file.py | 27 +++++++---------- .../test_http.py | 29 +++++++----------- .../test_https.py | 29 +++++++----------- .../test_mongo.py | 27 +++++++---------- .../test_mongo_uri.py | 29 ++++++++---------- .../test_mysql.py | 26 +++++++--------- 10 files changed, 102 insertions(+), 160 deletions(-) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 8154cc55453..8be25207d51 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -25,14 +25,14 @@ def setup_module(module): global ranged_tester - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created @@ -44,7 +44,7 @@ def setup_module(module): dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('cass_node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py index 3d56746be6e..051b4ff3086 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceClickHouse -SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "node", "9000", "default", "") +SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "local_node", "9000", "default", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "local" def setup_module(module): global cluster @@ -24,35 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('local_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + simple_tester.cleanup() @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py index 1d741d5271c..5d694bc09a2 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceExecutableCache -SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "cache_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "cache" def setup_module(module): global cluster @@ -24,36 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('cache_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py index dfcc35c54f8..8c0e6f8b878 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py @@ -8,13 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceExecutableHashed -SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "hashed_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "hashed" def setup_module(module): @@ -24,36 +25,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('hashed_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py index aa81cca466b..97a06fadc5e 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceFile -SOURCE = SourceFile("File", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceFile("File", "localhost", "9000", "file_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "file" def setup_module(module): global cluster @@ -24,36 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('file_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py index 7c8b5a41b01..c8c73011f61 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceHTTP -SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse1", "9000", "", "") +SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse_h", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "http" def setup_module(module): global cluster @@ -24,38 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_h', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('http_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py index 44950f013b3..42f33e3da3c 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceHTTPS -SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse1", "9000", "", "") +SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse_hs", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "https" def setup_module(module): global cluster @@ -24,38 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_hs', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('https_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py index 7d808845854..deaaf044bce 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -8,14 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMongo -SOURCE = SourceMongo("MongoDB", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "mongo" def setup_module(module): global cluster @@ -24,36 +23,30 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) + SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 5c09627d0b9..9561ee5acc0 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -8,13 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMongoURI -SOURCE = SourceMongoURI("MongoDB_URI", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "mongo_uri" def setup_module(module): @@ -24,36 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) - simple_tester = SimpleLayoutTester() + SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") + + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + node = cluster.add_instance('uri_node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 7a6b0b7ce8d..9bf3e888f31 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -15,6 +15,7 @@ node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "mysql" def setup_module(module): @@ -24,37 +25,30 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('mysql_node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): From 8aca6e9cd00111b0dc74a282e5c374c5e3dfa5f2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 12 Feb 2021 18:50:52 +0300 Subject: [PATCH 014/652] update runner --- docker/test/integration/runner/Dockerfile | 8 ++++---- .../integration/runner/compose/docker_compose_mongo.yml | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 502dc3736b2..9e850fe5281 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-integration-tests-runner . -FROM ubuntu:18.04 +FROM ubuntu:20.04 RUN apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ @@ -14,7 +14,6 @@ RUN apt-get update \ wget \ git \ iproute2 \ - module-init-tools \ cgroupfs-mount \ python3-pip \ tzdata \ @@ -65,17 +64,18 @@ RUN python3 -m pip install \ dict2xml \ dicttoxml \ docker \ - docker-compose==1.22.0 \ + docker-compose==1.28.2 \ grpcio \ grpcio-tools \ kafka-python \ kazoo \ minio \ protobuf \ - psycopg2-binary==2.7.5 \ + psycopg2-binary==2.8.6 \ pymongo \ pytest \ pytest-timeout \ + pytest-xdist \ redis \ tzlocal \ urllib3 \ diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 8c54544ed88..e131fad21d6 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -7,5 +7,5 @@ services: MONGO_INITDB_ROOT_USERNAME: root MONGO_INITDB_ROOT_PASSWORD: clickhouse ports: - - 27018:27017 + - ${MONGO_EXTERNAL_PORT}:${MONGO_INTERNAL_PORT} command: --profile=2 --verbose From 20864076edd6619dbe6b2d526c9353557e53771e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 12 Feb 2021 18:51:21 +0300 Subject: [PATCH 015/652] wip --- tests/integration/helpers/cluster.py | 127 ++++++++++-------- .../test_mongo_uri.py | 2 +- 2 files changed, 72 insertions(+), 57 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 14aa2f252c5..43a1a0bd673 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -32,17 +32,23 @@ from .hdfs_api import HDFSApi HELPERS_DIR = p.dirname(__file__) CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") LOCAL_DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "docker/test/integration/runner/compose/") -DEFAULT_ENV_NAME = 'env_file' +DEFAULT_ENV_NAME = '.env' SANITIZER_SIGN = "==================" -def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): - full_path = os.path.join(path, fname) - with open(full_path, 'w') as f: +def _create_env_file(path, variables): + with open(path, 'w') as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") - return full_path + return path + +def env_to_compose_args(env): + args = [] + for key, value in env.items(): + args += ["-e", "{}={}".format(key, value)] + return args + def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) @@ -50,8 +56,18 @@ def run_and_check(args, env=None, shell=False): # check_call(...) from subprocess does not print stderr, so we do it manually print('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) print('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) + print('Env:\n{}\n'.format(env)) raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) +# Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 +def get_open_port(): + import socket + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + s.bind(("",0)) + s.listen(1) + port = s.getsockname()[1] + s.close() + return port def subprocess_check_call(args): # Uncomment for debugging @@ -165,6 +181,10 @@ class ClickHouseCluster: self.schema_registry_host = "schema-registry" self.schema_registry_port = 8081 + # available when with_mongo == True + self.mongo_host = "mongo1" + self.mongo_port = get_open_port() + self.zookeeper_use_tmpfs = True self.docker_client = None @@ -246,6 +266,8 @@ class ClickHouseCluster: docker_compose_yml_dir = get_docker_compose_path() + assert instance.env_file is not None + self.instances[name] = instance if ipv4_address is not None or ipv6_address is not None: self.with_net_trics = True @@ -261,14 +283,14 @@ class ClickHouseCluster: self.with_zookeeper = True self.zookeeper_use_tmpfs = zookeeper_use_tmpfs self.base_cmd.extend(['--file', zookeeper_docker_compose_path]) - self.base_zookeeper_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_zookeeper_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', zookeeper_docker_compose_path] cmds.append(self.base_zookeeper_cmd) if with_mysql and not self.with_mysql: self.with_mysql = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) @@ -276,7 +298,7 @@ class ClickHouseCluster: if with_postgres and not self.with_postgres: self.with_postgres = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) @@ -285,56 +307,61 @@ class ClickHouseCluster: if not self.with_mysql: self.with_mysql = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) if not self.with_postgres: self.with_postgres = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_kafka and not self.with_kafka: self.with_kafka = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) - self.base_kafka_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) if with_kerberized_kafka and not self.with_kerberized_kafka: + env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' self.with_kerberized_kafka = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_kerberized_kafka_cmd = ['docker-compose','--project-name', self.project_name, + self.base_kerberized_kafka_cmd = ['docker-compose', '--env-file', instance.env_file,'--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] cmds.append(self.base_kerberized_kafka_cmd) if with_rabbitmq and not self.with_rabbitmq: self.with_rabbitmq = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) - self.base_rabbitmq_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] cmds.append(self.base_rabbitmq_cmd) if with_hdfs and not self.with_hdfs: self.with_hdfs = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) - self.base_hdfs_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] cmds.append(self.base_hdfs_cmd) if with_kerberized_hdfs and not self.with_kerberized_hdfs: self.with_kerberized_hdfs = True + env_variables['KERBERIZED_HDFS_DIR'] = instance.path + '/' self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')]) - self.base_kerberized_hdfs_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_kerberized_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')] cmds.append(self.base_kerberized_hdfs_cmd) if with_mongo and not self.with_mongo: self.with_mongo = True + env_variables['MONGO_HOST'] = self.mongo_host + env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) + env_variables['MONGO_INTERNAL_PORT'] = "27017" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) - self.base_mongo_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] cmds.append(self.base_mongo_cmd) @@ -345,21 +372,30 @@ class ClickHouseCluster: if with_redis and not self.with_redis: self.with_redis = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) - self.base_redis_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] if with_minio and not self.with_minio: self.with_minio = True self.minio_certs_dir = minio_certs_dir + if self.minio_certs_dir: + env_variables['MINIO_CERTS_DIR'] = p.join(self.base_dir, self.minio_certs_dir) + # Minio client (urllib3) uses SSL_CERT_FILE for certificate validation. + env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') + else: + # Attach empty certificates directory to ensure non-secure mode. + minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') + os.mkdir(minio_certs_dir) + env_variables['MINIO_CERTS_DIR'] = minio_certs_dir self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) - self.base_minio_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] cmds.append(self.base_minio_cmd) if with_cassandra and not self.with_cassandra: self.with_cassandra = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) - self.base_cassandra_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] print("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( @@ -532,7 +568,7 @@ class ClickHouseCluster: def wait_mongo_to_start(self, timeout=30): connection_str = 'mongodb://{user}:{password}@{host}:{port}'.format( - host='localhost', port='27018', user='root', password='clickhouse') + host='localhost', port=self.mongo_port, user='root', password='clickhouse') connection = pymongo.MongoClient(connection_str) start = time.time() while time.time() - start < timeout: @@ -662,9 +698,7 @@ class ClickHouseCluster: if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: print('Setup kerberized kafka') - env = os.environ.copy() - env['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) + run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) @@ -678,9 +712,7 @@ class ClickHouseCluster: if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: print('Setup kerberized HDFS') - env = os.environ.copy() - env['KERBERIZED_HDFS_DIR'] = instance.path + '/' - run_and_check(self.base_kerberized_hdfs_cmd + common_opts, env=env) + run_and_check(self.base_kerberized_hdfs_cmd + common_opts) self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(timeout=300) @@ -691,38 +723,16 @@ class ClickHouseCluster: if self.with_redis and self.base_redis_cmd: print('Setup Redis') - subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) + subprocess_check_call(self.base_redis_cmd + common_opts) time.sleep(10) if self.with_minio and self.base_minio_cmd: - env = os.environ.copy() - prev_ca_certs = os.environ.get('SSL_CERT_FILE') - if self.minio_certs_dir: - minio_certs_dir = p.join(self.base_dir, self.minio_certs_dir) - env['MINIO_CERTS_DIR'] = minio_certs_dir - # Minio client (urllib3) uses SSL_CERT_FILE for certificate validation. - os.environ['SSL_CERT_FILE'] = p.join(minio_certs_dir, 'public.crt') - else: - # Attach empty certificates directory to ensure non-secure mode. - minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') - os.mkdir(minio_certs_dir) - env['MINIO_CERTS_DIR'] = minio_certs_dir - minio_start_cmd = self.base_minio_cmd + common_opts logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd))) - run_and_check(minio_start_cmd, env=env) - - try: - logging.info("Trying to connect to Minio...") - self.wait_minio_to_start(secure=self.minio_certs_dir is not None) - finally: - # Safely return previous value of SSL_CERT_FILE environment variable. - if self.minio_certs_dir: - if prev_ca_certs: - os.environ['SSL_CERT_FILE'] = prev_ca_certs - else: - os.environ.pop('SSL_CERT_FILE') + run_and_check(minio_start_cmd) + logging.info("Trying to connect to Minio...") + self.wait_minio_to_start(secure=self.minio_certs_dir is not None) if self.with_cassandra and self.base_cassandra_cmd: subprocess_check_call(self.base_cassandra_cmd + ['up', '-d', '--force-recreate']) @@ -843,7 +853,9 @@ class ClickHouseCluster: subprocess_check_call(self.base_zookeeper_cmd + ["start", n]) -CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" +CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/config.xml" \ + " --log-file=/var/log/clickhouse-server/clickhouse-server.log " \ + " --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND) @@ -933,6 +945,7 @@ class ClickHouseInstance: self.path = p.join(self.cluster.instances_dir, name) self.docker_compose_path = p.join(self.path, 'docker-compose.yml') self.env_variables = env_variables or {} + self.env_file = None if with_odbc_drivers: self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini" self.with_mysql = True @@ -956,6 +969,8 @@ class ClickHouseInstance: self.ipv4_address = ipv4_address self.ipv6_address = ipv6_address self.with_installed_binary = with_installed_binary + self.env_file = os.path.join(os.path.dirname(self.docker_compose_path), DEFAULT_ENV_NAME) + def is_built_with_thread_sanitizer(self): build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") @@ -1333,9 +1348,9 @@ class ClickHouseInstance: if self.with_minio: depends_on.append("minio1") - env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) + _create_env_file(os.path.join(self.env_file), self.env_variables) - print("Env {} stored in {}".format(self.env_variables, env_file)) + print("Env {} stored in {}".format(self.env_variables, self.env_file)) odbc_ini_path = "" if self.odbc_ini_path: @@ -1383,7 +1398,7 @@ class ClickHouseInstance: logs_dir=logs_dir, depends_on=str(depends_on), user=os.getuid(), - env_file=env_file, + env_file=self.env_file, odbc_ini_path=odbc_ini_path, keytab_path=self.keytab_path, krb5_conf=self.krb5_conf, diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 9561ee5acc0..fdf4826cb63 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -26,7 +26,7 @@ def setup_module(module): cluster = ClickHouseCluster(__file__, name=test_name) - SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") + SOURCE = SourceMongoURI("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() From 06e1188d9e68c2fd8d79f767a49a31e0a3763321 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Feb 2021 12:35:45 +0300 Subject: [PATCH 016/652] wip --- .../compose/docker_compose_cassandra.yml | 2 +- .../runner/compose/docker_compose_mysql.yml | 4 +- tests/integration/helpers/cluster.py | 44 ++++++++++++------- tests/integration/helpers/external_sources.py | 1 + .../test_cassandra.py | 8 ++-- .../test_mysql.py | 11 ++--- .../integration/test_dictionaries_ddl/test.py | 14 +++--- .../test_dictionaries_mysql/test.py | 7 +-- .../test_disabled_mysql_server/test.py | 11 ++--- .../materialize_with_ddl.py | 6 +-- .../test_materialize_mysql_database/test.py | 2 +- .../test_mysql_database_engine/test.py | 41 ++++++++--------- .../integration/test_odbc_interaction/test.py | 8 ++-- tests/integration/test_storage_mysql/test.py | 22 +++++----- 14 files changed, 101 insertions(+), 80 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_cassandra.yml b/docker/test/integration/runner/compose/docker_compose_cassandra.yml index 6567a352027..7b2d297d089 100644 --- a/docker/test/integration/runner/compose/docker_compose_cassandra.yml +++ b/docker/test/integration/runner/compose/docker_compose_cassandra.yml @@ -4,4 +4,4 @@ services: image: cassandra restart: always ports: - - 9043:9042 + - ${CASSANDRA_EXTERNAL_PORT}:${CASSANDRA_INTERNAL_PORT} diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 90daf8a4238..a9e540be99a 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -1,10 +1,10 @@ version: '2.3' services: - mysql1: + mysql57: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse ports: - - 3308:3306 + - ${MYSQL_EXTERNAL_PORT}:${MYSQL_INTERNAL_PORT} command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 43a1a0bd673..f6e277f3711 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -185,6 +185,14 @@ class ClickHouseCluster: self.mongo_host = "mongo1" self.mongo_port = get_open_port() + # available when with_cassandra == True + self.cassandra_host = "cassandra1" + self.cassandra_port = get_open_port() + + # available when with_mysql == True + self.mysql_host = "mysql57" + self.mysql_port = get_open_port() + self.zookeeper_use_tmpfs = True self.docker_client = None @@ -197,6 +205,17 @@ class ClickHouseCluster: cmd += " client" return cmd + def setup_mysql_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mysql = True + env_variables['MYSQL_HOST'] = self.mysql_host + env_variables['MYSQL_EXTERNAL_PORT'] = str(self.mysql_port) + env_variables['MYSQL_INTERNAL_PORT'] = "3306" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) + self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] + + return self.base_mysql_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -288,12 +307,7 @@ class ClickHouseCluster: cmds.append(self.base_zookeeper_cmd) if with_mysql and not self.with_mysql: - self.with_mysql = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] - - cmds.append(self.base_mysql_cmd) + cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) if with_postgres and not self.with_postgres: self.with_postgres = True @@ -305,11 +319,7 @@ class ClickHouseCluster: if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True if not self.with_mysql: - self.with_mysql = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] - cmds.append(self.base_mysql_cmd) + cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) if not self.with_postgres: self.with_postgres = True @@ -385,7 +395,7 @@ class ClickHouseCluster: else: # Attach empty certificates directory to ensure non-secure mode. minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') - os.mkdir(minio_certs_dir) + os.makedirs(minio_certs_dir, exist_ok=True) env_variables['MINIO_CERTS_DIR'] = minio_certs_dir self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, @@ -394,6 +404,8 @@ class ClickHouseCluster: if with_cassandra and not self.with_cassandra: self.with_cassandra = True + env_variables['CASSANDRA_EXTERNAL_PORT'] = str(self.cassandra_port) + env_variables['CASSANDRA_INTERNAL_PORT'] = "9042" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] @@ -488,7 +500,7 @@ class ClickHouseCluster: start = time.time() while time.time() - start < timeout: try: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) conn.close() print("Mysql Started") return @@ -622,7 +634,7 @@ class ClickHouseCluster: time.sleep(1) def wait_cassandra_to_start(self, timeout=30): - cass_client = cassandra.cluster.Cluster(["localhost"], port="9043") + cass_client = cassandra.cluster.Cluster(["localhost"], self.cassandra_port) start = time.time() while time.time() - start < timeout: try: @@ -1213,7 +1225,7 @@ class ClickHouseInstance: "Database": "clickhouse", "Uid": "root", "Pwd": "clickhouse", - "Server": "mysql1", + "Server": self.cluster.mysql_host, }, "PostgreSQL": { "DSN": "postgresql_odbc", @@ -1325,7 +1337,7 @@ class ClickHouseInstance: depends_on = [] if self.with_mysql: - depends_on.append("mysql1") + depends_on.append("mysql57") if self.with_kafka: depends_on.append("kafka1") diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index 47de9dd0caf..fe4c5269a56 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -100,6 +100,7 @@ class SourceMySQL(ExternalSource): def prepare(self, structure, table_name, cluster): self.create_mysql_conn() self.execute_mysql_query("create database if not exists test default character set 'utf8'") + self.execute_mysql_query("drop table if exists test.{}".format(table_name)) fields_strs = [] for field in structure.keys + structure.ordinary_fields + structure.range_fields: fields_strs.append(field.name + ' ' + self.TYPE_MAPPING[field.field_type]) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 8be25207d51..1271619f1f7 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -8,8 +8,7 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceCassandra -SOURCE = SourceCassandra("Cassandra", "localhost", "9043", "cassandra1", "9042", "", "") - +SOURCE = None cluster = None node = None simple_tester = None @@ -24,6 +23,9 @@ def setup_module(module): global complex_tester global ranged_tester + cluster = ClickHouseCluster(__file__, name=test_name) + + SOURCE = SourceCassandra("Cassandra", "localhost", cluster.cassandra_port, cluster.cassandra_host, "9042", "", "") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() @@ -36,8 +38,6 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) - main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 9bf3e888f31..119709bf229 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -8,8 +8,7 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMySQL -SOURCE = SourceMySQL("MySQL", "localhost", "3308", "mysql1", "3306", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None @@ -25,6 +24,10 @@ def setup_module(module): global complex_tester global ranged_tester + cluster = ClickHouseCluster(__file__, name=test_name) + + SOURCE = SourceMySQL("MySQL", "localhost", cluster.mysql_port, cluster.mysql_host, "3306", "root", "clickhouse") + simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) @@ -36,15 +39,13 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) - main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('mysql_node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 3ea64383fbf..59ccc59d0af 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -4,6 +4,7 @@ import time import pymysql import pytest +import logging from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -23,6 +24,7 @@ node4 = cluster.add_instance('node4', user_configs=['configs/user_admin.xml', 'c def create_mysql_conn(user, password, hostname, port): + logging.debug("Created MySQL connection user:{}, password:{}, host:{}, port{}".format(user, password, hostname, port)) return pymysql.connect( user=user, password=password, @@ -31,6 +33,7 @@ def create_mysql_conn(user, password, hostname, port): def execute_mysql_query(connection, query): + logging.debug("Execute MySQL query:{}".format(query)) with warnings.catch_warnings(): warnings.simplefilter("ignore") with connection.cursor() as cursor: @@ -63,8 +66,9 @@ def started_cluster(): (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) - execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") + mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) + execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS clickhouse") + execute_mysql_query(mysql_conn, "CREATE DATABASE clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format( name)) @@ -87,7 +91,7 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): DB 'clickhouse' TABLE '{}' REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333) - REPLICA(PRIORITY 2 HOST 'mysql1' PORT 3306) + REPLICA(PRIORITY 2 HOST 'mysql57' PORT 3306) )) {} LIFETIME(MIN 1 MAX 3) @@ -241,7 +245,7 @@ def test_file_dictionary_restrictions(started_cluster): def test_dictionary_with_where(started_cluster): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) + mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))") @@ -258,7 +262,7 @@ def test_dictionary_with_where(started_cluster): PASSWORD 'clickhouse' DB 'clickhouse' TABLE 'special_table' - REPLICA(PRIORITY 1 HOST 'mysql1' PORT 3306) + REPLICA(PRIORITY 1 HOST 'mysql57' PORT 3306) WHERE 'value1 = \\'qweqwe\\' OR value1 = \\'\\\\u3232\\'' )) LAYOUT(FLAT()) diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 16e432c6425..70af73a1592 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -37,7 +37,7 @@ def started_cluster(): instance.query("CREATE DATABASE IF NOT EXISTS test") # Create database in ClickChouse using MySQL protocol (will be used for data insertion) - instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql1:3306', 'test', 'root', 'clickhouse')") + instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql57:3306', 'test', 'root', 'clickhouse')") yield cluster @@ -66,7 +66,8 @@ def test_load_mysql_dictionaries(started_cluster): def create_mysql_db(mysql_connection, name): with mysql_connection.cursor() as cursor: - cursor.execute("CREATE DATABASE IF NOT EXISTS {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def prepare_mysql_table(table_name, index): @@ -88,7 +89,7 @@ def prepare_mysql_table(table_name, index): def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=cluster.mysql_port) return conn diff --git a/tests/integration/test_disabled_mysql_server/test.py b/tests/integration/test_disabled_mysql_server/test.py index a2cbcb17534..2fc84ee74a5 100644 --- a/tests/integration/test_disabled_mysql_server/test.py +++ b/tests/integration/test_disabled_mysql_server/test.py @@ -22,7 +22,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): self.user = user self.port = port self.hostname = hostname @@ -46,15 +46,16 @@ class MySQLNodeInstance: def test_disabled_mysql_server(started_cluster): with contextlib.closing(MySQLNodeInstance()) as mysql_node: - mysql_node.query("CREATE DATABASE test_db;") - mysql_node.query("CREATE TABLE test_db.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_db_disabled;") + mysql_node.query("CREATE DATABASE test_db_disabled;") + mysql_node.query("CREATE TABLE test_db_disabled.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") with PartitionManager() as pm: - clickhouse_node.query("CREATE DATABASE test_db ENGINE = MySQL('mysql1:3306', 'test_db', 'root', 'clickhouse')") + clickhouse_node.query("CREATE DATABASE test_db_disabled ENGINE = MySQL('mysql57:3306', 'test_db_disabled', 'root', 'clickhouse')") pm._add_rule({'source': clickhouse_node.ip_address, 'destination_port': 3306, 'action': 'DROP'}) clickhouse_node.query("SELECT * FROM system.parts") clickhouse_node.query("SELECT * FROM system.mutations") clickhouse_node.query("SELECT * FROM system.graphite_retentions") - clickhouse_node.query("DROP DATABASE test_db") + clickhouse_node.query("DROP DATABASE test_db_disabled") diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index c9be2387fc7..0dcb4c146c7 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -246,7 +246,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql1" else "(id)")) + "0" if service_name == "mysql57" else "(id)")) # create mapping clickhouse_node.query( @@ -264,9 +264,9 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql1" else "(id)")) + "0" if service_name == "mysql57" else "(id)")) - default_expression = "DEFAULT\t0" if service_name == "mysql1" else "DEFAULT\tid" + default_expression = "DEFAULT\t0" if service_name == "mysql57" else "DEFAULT\tid" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index e55772d9e1d..13c702f998b 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -28,7 +28,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=3308, docker_compose=None, project_name=cluster.project_name): + def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=cluster.mysql_port, docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 4d10e2ea6f5..f5fb9be337d 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -21,7 +21,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): self.user = user self.port = port self.hostname = hostname @@ -55,11 +55,12 @@ class MySQLNodeInstance: def test_mysql_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") assert 'test_database' in clickhouse_node.query('SHOW DATABASES') mysql_node.query( @@ -88,13 +89,13 @@ def test_mysql_ddl_for_mysql_database(started_cluster): def test_clickhouse_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;') clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") assert 'test_table' in clickhouse_node.query('SHOW TABLES FROM test_database') clickhouse_node.query("DROP TABLE test_database.test_table") @@ -113,12 +114,12 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster): def test_clickhouse_dml_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `i``d` int(11) NOT NULL, PRIMARY KEY (`i``d`)) ENGINE=InnoDB;') clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test_database, 'root', 'clickhouse')") assert clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`").rstrip() == '0' clickhouse_node.query("INSERT INTO `test_database`.`test_table`(`i``d`) select number from numbers(10000)") @@ -131,7 +132,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster): def test_clickhouse_join_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test.t1_mysql_local (" "pays VARCHAR(55) DEFAULT 'FRA' NOT NULL," @@ -143,9 +144,9 @@ def test_clickhouse_join_for_mysql_database(started_cluster): "opco VARCHAR(5) DEFAULT ''" ")") clickhouse_node.query( - "CREATE TABLE default.t1_remote_mysql AS mysql('mysql1:3306','test','t1_mysql_local','root','clickhouse')") + "CREATE TABLE default.t1_remote_mysql AS mysql('mysql57:3306','test','t1_mysql_local','root','clickhouse')") clickhouse_node.query( - "CREATE TABLE default.t2_remote_mysql AS mysql('mysql1:3306','test','t2_mysql_local','root','clickhouse')") + "CREATE TABLE default.t2_remote_mysql AS mysql('mysql57:3306','test','t2_mysql_local','root','clickhouse')") assert clickhouse_node.query("SELECT s.pays " "FROM default.t1_remote_mysql AS s " "LEFT JOIN default.t1_remote_mysql AS s_ref " @@ -154,19 +155,19 @@ def test_clickhouse_join_for_mysql_database(started_cluster): def test_bad_arguments_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( - "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") + "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql57:3306', test_bad_arguments, root, 'clickhouse')") assert 'Database engine MySQL requested literal argument.' in str(exception.value) mysql_node.query("DROP DATABASE test_bad_arguments") def test_data_types_support_level_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse')", + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") @@ -177,7 +178,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") clickhouse_node.query( - "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", + "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", settings={"mysql_datatypes_support_level": "decimal"}) assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") @@ -306,7 +307,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m else: return [do_execute(q) for q in query] - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: execute_query(mysql_node, [ "DROP DATABASE IF EXISTS ${mysql_db}", "CREATE DATABASE ${mysql_db} DEFAULT CHARACTER SET 'utf8'", @@ -323,7 +324,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # MySQL TABLE ENGINE execute_query(clickhouse_node, [ "DROP TABLE IF EXISTS ${ch_mysql_table};", - "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", + "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", ], settings=clickhouse_query_settings) # Validate type @@ -343,7 +344,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # MySQL DATABASE ENGINE execute_query(clickhouse_node, [ "DROP DATABASE IF EXISTS ${ch_mysql_db}", - "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql1:3306', '${mysql_db}', 'root', 'clickhouse')" + "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql57:3306', '${mysql_db}', 'root', 'clickhouse')" ], settings=clickhouse_query_settings) # Validate type @@ -364,7 +365,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # Validate type assert \ execute_query(clickhouse_node, - "SELECT toTypeName(value) FROM mysql('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", + "SELECT toTypeName(value) FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", settings=clickhouse_query_settings) \ == \ expected_ch_type @@ -374,5 +375,5 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m execute_query(mysql_node, "SELECT value FROM ${mysql_db}.${table_name}") \ == \ execute_query(clickhouse_node, - "SELECT value FROM mysql('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", + "SELECT value FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", settings=clickhouse_query_settings) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 084fc407f39..470d816d2d1 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -27,14 +27,14 @@ create_table_sql_template = """ def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) return conn def create_mysql_db(conn, name): with conn.cursor() as cursor: - cursor.execute( - "CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def create_mysql_table(conn, table_name): @@ -119,7 +119,7 @@ def test_mysql_simple_select_works(started_cluster): settings={"external_table_functions_use_nulls": "0"}) == '0\n511\n' node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 7b23e20e200..a285b263688 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -39,7 +39,7 @@ def test_many_connections(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query("INSERT INTO {} (id, name) SELECT number, concat('name_', toString(number)) from numbers(10) ".format(table_name)) @@ -58,7 +58,7 @@ def test_insert_select(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -74,7 +74,7 @@ def test_replace_select(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -93,7 +93,7 @@ def test_insert_on_duplicate_select(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -111,7 +111,7 @@ def test_where(started_cluster): conn = get_mysql_conn() create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -130,7 +130,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): conn = get_mysql_conn() create_mysql_table(conn, 'table_function') - table_function = "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') + table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' node1.query( "INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format( @@ -152,7 +152,7 @@ def test_binary_type(started_cluster): conn = get_mysql_conn() with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") - table_function = "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') + table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') node1.query("INSERT INTO {} VALUES (42, 'clickhouse')".format('TABLE FUNCTION ' + table_function)) assert node1.query("SELECT * FROM {}".format(table_function)) == '42\tclickhouse\\0\\0\\0\\0\\0\\0\n' @@ -161,7 +161,7 @@ def test_enum_type(started_cluster): conn = get_mysql_conn() create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); '''.format(table_name, table_name)) node1.query("INSERT INTO {} (id, name, age, money, source) VALUES (1, 'name', 0, 0, 'URL')".format(table_name)) assert node1.query("SELECT source FROM {} LIMIT 1".format(table_name)).rstrip() == 'URL' @@ -169,14 +169,14 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) return conn def create_mysql_db(conn, name): with conn.cursor() as cursor: - cursor.execute( - "CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def create_mysql_table(conn, tableName): From 0b62ef4aa9eb3d49ff7681c6946d8de7a7aa58c1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 10:10:01 +0300 Subject: [PATCH 017/652] more fixes --- tests/integration/helpers/cluster.py | 67 ++++++- .../materialize_with_ddl.py | 51 +++--- .../test_materialize_mysql_database/test.py | 169 +++++++----------- .../integration/test_odbc_interaction/test.py | 27 ++- 4 files changed, 176 insertions(+), 138 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f6e277f3711..1b07122469a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -134,6 +134,8 @@ class ClickHouseCluster: self.project_name = re.sub(r'[^a-z0-9]', '', project_name.lower()) self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name)) self.docker_logs_path = p.join(self.instances_dir, 'docker.log') + self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) + self.env_variables = {} custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') self.docker_api_version = os.environ.get("DOCKER_API_VERSION") @@ -142,6 +144,7 @@ class ClickHouseCluster: self.base_cmd = ['docker-compose'] if custom_dockerd_host: self.base_cmd += ['--host', custom_dockerd_host] + self.base_cmd += ['--env-file', self.env_file] self.base_cmd += ['--project-name', self.project_name] self.base_zookeeper_cmd = None @@ -154,6 +157,7 @@ class ClickHouseCluster: self.instances = {} self.with_zookeeper = False self.with_mysql = False + self.with_mysql8 = False self.with_postgres = False self.with_kafka = False self.with_kerberized_kafka = False @@ -193,6 +197,10 @@ class ClickHouseCluster: self.mysql_host = "mysql57" self.mysql_port = get_open_port() + # available when with_mysql8 == True + self.mysql8_host = "mysql80" + self.mysql8_port = get_open_port() + self.zookeeper_use_tmpfs = True self.docker_client = None @@ -216,9 +224,20 @@ class ClickHouseCluster: return self.base_mysql_cmd + def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mysql8 = True + env_variables['MYSQL8_HOST'] = self.mysql8_host + env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port) + env_variables['MYSQL8_INTERNAL_PORT'] = "3306" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')]) + self.base_mysql8_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')] + + return self.base_mysql8_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, - with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, + with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -261,6 +280,7 @@ class ClickHouseCluster: with_zookeeper=with_zookeeper, zookeeper_config_path=self.zookeeper_config_path, with_mysql=with_mysql, + with_mysql8=with_mysql8, with_kafka=with_kafka, with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, @@ -285,8 +305,6 @@ class ClickHouseCluster: docker_compose_yml_dir = get_docker_compose_path() - assert instance.env_file is not None - self.instances[name] = instance if ipv4_address is not None or ipv6_address is not None: self.with_net_trics = True @@ -309,6 +327,9 @@ class ClickHouseCluster: if with_mysql and not self.with_mysql: cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_mysql8 and not self.with_mysql8: + cmds.append(self.setup_mysql8_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_postgres and not self.with_postgres: self.with_postgres = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) @@ -445,6 +466,9 @@ class ClickHouseCluster: node.wait_for_start(start_deadline) return node + def restart_service(self, service_name): + run_and_check(self.base_cmd + ["restart", service_name]) + def get_instance_ip(self, instance_name): print("get_instance_ip instance_name={}".format(instance_name)) docker_id = self.get_instance_docker_id(instance_name) @@ -498,6 +522,7 @@ class ClickHouseCluster: def wait_mysql_to_start(self, timeout=60): start = time.time() + errors = [] while time.time() - start < timeout: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) @@ -505,12 +530,28 @@ class ClickHouseCluster: print("Mysql Started") return except Exception as ex: - print("Can't connect to MySQL " + str(ex)) + errors += [str(ex)] time.sleep(0.5) subprocess_call(['docker-compose', 'ps', '--services', '--all']) + logging.error("Can't connect to MySQL:{}".format(errors)) raise Exception("Cannot wait MySQL container") + def wait_mysql8_to_start(self, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql8_port) + conn.close() + print("Mysql 8 Started") + return + except Exception as ex: + print("Can't connect to MySQL 8 " + str(ex)) + time.sleep(0.5) + + subprocess_call(['docker-compose', 'ps', '--services', '--all']) + raise Exception("Cannot wait MySQL 8 container") + def wait_postgres_to_start(self, timeout=60): start = time.time() while time.time() - start < timeout: @@ -671,7 +712,7 @@ class ClickHouseCluster: self.docker_client = docker.from_env(version=self.docker_api_version) - common_opts = ['up', '-d', '--force-recreate'] + common_opts = ['up', '-d', '--force-recreate', '--remove-orphans'] if self.with_zookeeper and self.base_zookeeper_cmd: print('Setup ZooKeeper') @@ -697,6 +738,11 @@ class ClickHouseCluster: subprocess_check_call(self.base_mysql_cmd + common_opts) self.wait_mysql_to_start(120) + if self.with_mysql8 and self.base_mysql8_cmd: + print('Setup MySQL 8') + subprocess_check_call(self.base_mysql8_cmd + common_opts) + self.wait_mysql8_to_start(120) + if self.with_postgres and self.base_postgres_cmd: print('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) @@ -750,6 +796,8 @@ class ClickHouseCluster: subprocess_check_call(self.base_cassandra_cmd + ['up', '-d', '--force-recreate']) self.wait_cassandra_to_start() + _create_env_file(os.path.join(self.env_file), self.env_variables) + clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) subprocess.check_output(clickhouse_start_cmd) @@ -796,7 +844,7 @@ class ClickHouseCluster: subprocess_check_call(self.base_cmd + ['kill']) try: - subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + subprocess_check_call(self.base_cmd + ['down', '--volumes']) except Exception as e: print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) @@ -917,7 +965,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, - macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs, + macros, with_zookeeper, zookeeper_config_path, with_mysql, with_mysql8, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, @@ -945,6 +993,7 @@ class ClickHouseInstance: self.odbc_bridge_bin_path = odbc_bridge_bin_path self.with_mysql = with_mysql + self.with_mysql8 = with_mysql8 self.with_kafka = with_kafka self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq @@ -1339,6 +1388,9 @@ class ClickHouseInstance: if self.with_mysql: depends_on.append("mysql57") + if self.with_mysql8: + depends_on.append("mysql80") + if self.with_kafka: depends_on.append("kafka1") depends_on.append("schema-registry") @@ -1360,6 +1412,7 @@ class ClickHouseInstance: if self.with_minio: depends_on.append("minio1") + self.cluster.env_variables.update(self.env_variables) _create_env_file(os.path.join(self.env_file), self.env_variables) print("Env {} stored in {}".format(self.env_variables, self.env_file)) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 0dcb4c146c7..1c9655b457f 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -31,12 +31,12 @@ def check_query(clickhouse_node, query, result_set, retry_count=60, interval_sec def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_dml") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_dml") + mysql_node.query("CREATE DATABASE test_database_dml DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (" + mysql_node.query("CREATE TABLE test_database_dml.test_table_1 (" "`key` INT NOT NULL PRIMARY KEY, " "unsigned_tiny_int TINYINT UNSIGNED, tiny_int TINYINT, " "unsigned_small_int SMALLINT UNSIGNED, small_int SMALLINT, " @@ -53,68 +53,68 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam # it already has some data mysql_node.query(""" - INSERT INTO test_database.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', + INSERT INTO test_database_dml.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', '2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', true); """) clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_dml ENGINE = MaterializeMySQL('{}:3306', 'test_database_dml', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_database_dml" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query(""" - INSERT INTO test_database.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', + INSERT INTO test_database_dml.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', '2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', false); """) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") - mysql_node.query("UPDATE test_database.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") + mysql_node.query("UPDATE test_database_dml.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") check_query(clickhouse_node, """ SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int, small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ - _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV + _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV """, "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t0\n") # update primary key - mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") + mysql_node.query("UPDATE test_database_dml.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, " " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " - " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + " _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t0\n3\t2\t-1\t2\t-2\t3\t-3\t" "4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t1\n") - mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `key` = 2') + mysql_node.query('DELETE FROM test_database_dml.test_table_1 WHERE `key` = 2') check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, " " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " - " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + " _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t1\n") - mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `unsigned_tiny_int` = 2') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "") + mysql_node.query('DELETE FROM test_database_dml.test_table_1 WHERE `unsigned_tiny_int` = 2') + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_dml") + mysql_node.query("DROP DATABASE test_database_dml") def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): @@ -525,7 +525,7 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABАASE IF EXISTS priv_err_db") mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -670,6 +670,7 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") mysql_node.query("CREATE DATABASE kill_mysql_while_insert") mysql_node.query("CREATE TABLE kill_mysql_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse')".format(service_name)) @@ -684,17 +685,14 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): t = threading.Thread(target=insert, args=(10000,)) t.start() - run_and_check( - ['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'stop']) + clickhouse_node.cluster.restart_service(service_name) finally: with pytest.raises(QueryRuntimeException) as execption: time.sleep(5) clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test") assert "Master maybe lost." in str(execption.value) - run_and_check( - ['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'start']) - mysql_node.wait_mysql_to_start(120) + mysql_node.alloc_connection() clickhouse_node.query("DETACH DATABASE kill_mysql_while_insert") clickhouse_node.query("ATTACH DATABASE kill_mysql_while_insert") @@ -709,6 +707,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): def clickhouse_killed_while_insert(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS kill_clickhouse_while_insert") mysql_node.query("CREATE DATABASE kill_clickhouse_while_insert") mysql_node.query("CREATE TABLE kill_clickhouse_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") clickhouse_node.query("CREATE DATABASE kill_clickhouse_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_clickhouse_while_insert', 'root', 'clickhouse')".format(service_name)) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 13c702f998b..04c9868a282 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -7,15 +7,18 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check import docker +import logging from . import materialize_with_ddl DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) +mysql_node = None +mysql8_node = None -node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True) -node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql=False, stay_alive=True) +node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=True, stay_alive=True) +node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql8=True, stay_alive=True) @pytest.fixture(scope="module") @@ -27,26 +30,29 @@ def started_cluster(): cluster.shutdown() -class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=cluster.mysql_port, docker_compose=None, project_name=cluster.project_name): +class MySQLConnection: + def __init__(self, port, user='root', password='clickhouse', ip_address='127.0.0.1', docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address self.password = password self.mysql_connection = None # lazy init - self.docker_compose = docker_compose - self.project_name = project_name - def alloc_connection(self): - if self.mysql_connection is None: - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, - port=self.port, autocommit=True) - else: - if self.mysql_connection.ping(): - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, - port=self.port, autocommit=True) - return self.mysql_connection + errors = [] + for _ in range(5): + try: + if self.mysql_connection is None: + self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, + port=self.port, autocommit=True) + else: + self.mysql_connection.ping(reconnect=True) + logging.debug("MySQL Connection establised: {}:{}".format(self.ip_address, self.port)) + return self.mysql_connection + except Exception as e: + errors += [str(e)] + time.sleep(1) + raise Exception("Connection not establised, {}".format(errors)) def query(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -75,174 +81,133 @@ class MySQLNodeInstance: if self.mysql_connection is not None: self.mysql_connection.close() - def wait_mysql_to_start(self, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - self.alloc_connection() - print("Mysql Started") - return - except Exception as ex: - print("Can't connect to MySQL " + str(ex)) - time.sleep(0.5) - - run_and_check(['docker-compose', 'ps', '--services', 'all']) - raise Exception("Cannot wait MySQL container") - @pytest.fixture(scope="module") def started_mysql_5_7(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose) - - try: - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) - mysql_node.wait_mysql_to_start(120) - yield mysql_node - finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) - + mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', '127.0.0.1') + yield mysql_node @pytest.fixture(scope="module") def started_mysql_8_0(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose) - - try: - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) - mysql_node.wait_mysql_to_start(120) - yield mysql_node - finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) - + mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', '127.0.0.1') + yield mysql8_node @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") # mysql 5.7 cannot support alter rename column - # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_select_without_columns_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_select_without_columns_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_insert_with_modify_binlog_checksum_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_insert_with_modify_binlog_checksum_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_tables_table(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql80") diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 470d816d2d1..3ef59e7bd23 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -3,6 +3,8 @@ import time import psycopg2 import pymysql.cursors import pytest +import logging + from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT @@ -15,6 +17,11 @@ node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) + +drop_table_sql_template = """ + DROP TABLE IF EXISTS `clickhouse`.`{}` + """ + create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( `id` int(11) NOT NULL, @@ -27,9 +34,22 @@ create_table_sql_template = """ def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) - return conn - + errors = [] + conn = None + for _ in range(5): + try: + if conn is None: + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + else: + conn.ping(reconnect=True) + logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) + def create_mysql_db(conn, name): with conn.cursor() as cursor: @@ -39,6 +59,7 @@ def create_mysql_db(conn, name): def create_mysql_table(conn, table_name): with conn.cursor() as cursor: + cursor.execute(drop_table_sql_template.format(table_name)) cursor.execute(create_table_sql_template.format(table_name)) From 178d3fdba07e9c8dda213a503bb9787d10fff9a1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 10:36:15 +0300 Subject: [PATCH 018/652] typo --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 1c9655b457f..096ffc2a8f4 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -525,7 +525,7 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABАASE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABАSE IF EXISTS priv_err_db") mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") From e4b3511d28420875e289c1284606a56a8a738b8e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 10:37:35 +0300 Subject: [PATCH 019/652] no more separate compose files for materialize mysql --- ...ompose_mysql_5_7_for_materialize_mysql.yml | 10 --------- ...mysql.yml => docker_compose_mysql_8_0.yml} | 6 +++--- .../configs/dictionaries/mysql_dict1.xml | 4 ++-- .../configs/dictionaries/mysql_dict2.xml | 6 +++--- .../test_dictionaries_mysql/test.py | 21 +++++++++++++++---- 5 files changed, 25 insertions(+), 22 deletions(-) delete mode 100644 docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml rename docker/test/integration/runner/compose/{docker_compose_mysql_8_0_for_materialize_mysql.yml => docker_compose_mysql_8_0.yml} (76%) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml deleted file mode 100644 index e7d762203ee..00000000000 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ /dev/null @@ -1,10 +0,0 @@ -version: '2.3' -services: - mysql1: - image: mysql:5.7 - restart: 'no' - environment: - MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml similarity index 76% rename from docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml index 918a2b5f80f..7b551371ab2 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -1,10 +1,10 @@ version: '2.3' services: - mysql8_0: + mysql80: image: mysql:8.0 - restart: 'no' + restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse ports: - - 33308:3306 + - ${MYSQL8_EXTERNAL_PORT}:${MYSQL8_INTERNAL_PORT} command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml index 514c73f3be2..2242088ebc4 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml @@ -5,7 +5,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -41,7 +41,7 @@ test - mysql1 + mysql57 3306 root clickhouse diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml index 91506481cc9..278fad49d03 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml @@ -5,7 +5,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -42,7 +42,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -78,7 +78,7 @@ test - mysql1 + mysql57 3306 root clickhouse diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 70af73a1592..90cfe53dd68 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -2,6 +2,8 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster +import time +import logging CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml'] @@ -87,11 +89,22 @@ def prepare_mysql_table(table_name, index): # Create CH Dictionary tables based on MySQL tables query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index))) - def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=cluster.mysql_port) - return conn - + errors = [] + conn = None + for _ in range(5): + try: + if conn is None: + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + else: + conn.ping(reconnect=True) + logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) def create_mysql_table(conn, table_name): with conn.cursor() as cursor: From 3d1bcb1272c0279978b752af2bc827a3d67d03ac Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 11:04:52 +0300 Subject: [PATCH 020/652] fix --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 096ffc2a8f4..1e4d51e325c 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -525,7 +525,7 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABАSE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") From 8603855f5fd70f282566f9113fd27447e0a876a9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 17:16:15 +0300 Subject: [PATCH 021/652] fix --- .../compose/docker_compose_rabbitmq.yml | 4 +- .../runner/compose/docker_compose_redis.yml | 2 +- tests/integration/helpers/cluster.py | 44 ++++++++++--- .../test_dictionaries_redis/test.py | 7 +- .../integration/test_storage_rabbitmq/test.py | 64 +++++++++---------- 5 files changed, 74 insertions(+), 47 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index 1e9c3777505..d16a19f7ac2 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -5,8 +5,8 @@ services: image: rabbitmq:3-management hostname: rabbitmq1 ports: - - "5672:5672" - - "15672:15672" + - ${RABBITMQ_EXTERNAL_PORT}:${RABBITMQ_INTERNAL_PORT} + - ${RABBITMQ_EXTERNAL_HTTP_PORT}:${RABBITMQ_INTERNAL_HTTP_PORT} environment: RABBITMQ_DEFAULT_USER: "root" RABBITMQ_DEFAULT_PASS: "clickhouse" diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml index 72df99ec59b..b14d0dce98b 100644 --- a/docker/test/integration/runner/compose/docker_compose_redis.yml +++ b/docker/test/integration/runner/compose/docker_compose_redis.yml @@ -4,5 +4,5 @@ services: image: redis restart: always ports: - - 6380:6379 + - ${REDIS_EXTERNAL_PORT}:${REDIS_INTERNAL_PORT} command: redis-server --requirepass "clickhouse" --databases 32 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1b07122469a..560650a09d7 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -193,6 +193,15 @@ class ClickHouseCluster: self.cassandra_host = "cassandra1" self.cassandra_port = get_open_port() + # available when with_rabbitmq == True + self.rabbitmq_host = "rabbitmq1" + self.rabbitmq_port = get_open_port() + self.rabbitmq_http_port = get_open_port() + + # available when with_redis == True + self.redis_host = "redis1" + self.redis_port = get_open_port() + # available when with_mysql == True self.mysql_host = "mysql57" self.mysql_port = get_open_port() @@ -235,6 +244,30 @@ class ClickHouseCluster: return self.base_mysql8_cmd + def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_redis = True + env_variables['REDIS_HOST'] = self.redis_host + env_variables['REDIS_EXTERNAL_PORT'] = str(self.redis_port) + env_variables['REDIS_INTERNAL_PORT'] = "6379" + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] + return self.base_redis_cmd + + def setup_rabbitmq_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_rabbitmq = True + env_variables['RABBITMQ_HOST'] = self.rabbitmq_host + env_variables['RABBITMQ_EXTERNAL_PORT'] = str(self.rabbitmq_port) + env_variables['RABBITMQ_INTERNAL_PORT'] = "5672" + env_variables['RABBITMQ_EXTERNAL_HTTP_PORT'] = str(self.rabbitmq_http_port) + env_variables['RABBITMQ_INTERNAL_HTTP_PORT'] = "15672" + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) + self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] + return self.base_rabbitmq_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -365,11 +398,7 @@ class ClickHouseCluster: cmds.append(self.base_kerberized_kafka_cmd) if with_rabbitmq and not self.with_rabbitmq: - self.with_rabbitmq = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) - self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] - cmds.append(self.base_rabbitmq_cmd) + cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) if with_hdfs and not self.with_hdfs: self.with_hdfs = True @@ -401,10 +430,7 @@ class ClickHouseCluster: cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_net.yml')]) if with_redis and not self.with_redis: - self.with_redis = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) - self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] + cmds.append(self.setup_redis_cmd(instance, env_variables, docker_compose_yml_dir)) if with_minio and not self.with_minio: self.with_minio = True diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index d7d7e0ee3ad..7fcbd0d0909 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -93,15 +93,17 @@ def setup_module(module): global node global dict_configs_path + cluster = ClickHouseCluster(__file__) + for f in os.listdir(dict_configs_path): os.remove(os.path.join(dict_configs_path, f)) for i, field in enumerate(FIELDS): DICTIONARIES.append([]) sources = [] - sources.append(SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2, + sources.append(SourceRedis("RedisSimple", "localhost", cluster.redis_port, cluster.redis_host, "6379", "", "clickhouse", i * 2, storage_type="simple")) - sources.append(SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2 + 1, + sources.append(SourceRedis("RedisHash", "localhost", cluster.redis_port, cluster.redis_host, "6379", "", "clickhouse", i * 2 + 1, storage_type="hash_map")) for source in sources: for layout in LAYOUTS: @@ -117,7 +119,6 @@ def setup_module(module): for fname in os.listdir(dict_configs_path): dictionaries.append(os.path.join(dict_configs_path, fname)) - cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_redis=True) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 911f6d144f9..c78a5f8d600 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -128,14 +128,14 @@ def test_rabbitmq_select(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'select', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -164,11 +164,11 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'empty', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) assert int(instance.query('SELECT count() FROM test.rabbitmq')) == 0 @@ -178,13 +178,13 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -227,7 +227,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -262,7 +262,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -304,7 +304,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -347,7 +347,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -397,7 +397,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -445,7 +445,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -498,7 +498,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(rabbitmq_messages)] credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -562,7 +562,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -639,7 +639,7 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -697,7 +697,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -750,7 +750,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -979,7 +979,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1052,7 +1052,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1138,7 +1138,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1216,7 +1216,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1295,7 +1295,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1392,7 +1392,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1446,7 +1446,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1513,7 +1513,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1591,7 +1591,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -1671,7 +1671,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1731,7 +1731,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): messages_num = 150000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1803,7 +1803,7 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1889,7 +1889,7 @@ def test_rabbitmq_no_connection_at_startup(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() for i in range(messages_num): From 677528326fa7401122a8d47b837f1174e3aa7b89 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 17 Feb 2021 18:40:01 +0300 Subject: [PATCH 022/652] postgres --- .../compose/docker_compose_postgres.yml | 9 +- ...esql.yml => docker_compose_postgresql.yml} | 0 ...docker_compose_postgresql_java_client.yml} | 0 tests/integration/helpers/client.py | 12 +-- tests/integration/helpers/cluster.py | 85 ++++++++++--------- .../test_dictionaries_postgresql/test.py | 20 ++--- .../postgres_odbc_hashed_dictionary.xml | 2 +- .../integration/test_odbc_interaction/test.py | 31 ++++--- .../test_postgresql_database_engine/test.py | 4 +- .../test_postgresql_protocol/test.py | 4 +- .../test_storage_postgresql/test.py | 4 +- 11 files changed, 98 insertions(+), 73 deletions(-) rename docker/test/integration/runner/compose/{docker_compose_postgesql.yml => docker_compose_postgresql.yml} (100%) rename docker/test/integration/runner/compose/{docker_compose_postgesql_java_client.yml => docker_compose_postgresql_java_client.yml} (100%) diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index fff4fb1fa42..cb0017ccd33 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -6,8 +6,15 @@ services: environment: POSTGRES_PASSWORD: mysecretpassword ports: - - 5432:5432 + - ${POSTGRES_EXTERNAL_PORT}:${POSTGRES_INTERNAL_PORT} + healthcheck: + test: ["CMD-SHELL", "pg_isready -U postgres"] + interval: 10s + timeout: 5s + retries: 5 networks: default: aliases: - postgre-sql.local + environment: + POSTGRES_HOST_AUTH_METHOD: "trust" \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql.yml b/docker/test/integration/runner/compose/docker_compose_postgresql.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgesql.yml rename to docker/test/integration/runner/compose/docker_compose_postgresql.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml rename to docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index 04b6082e95f..95f8a58dbf1 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -101,8 +101,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read().decode() - stderr = self.stderr_file.read().decode() + stdout = self.stdout_file.read().decode('utf-8', errors='replace') + stderr = self.stderr_file.read().decode('utf-8', errors='replace') if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') @@ -118,8 +118,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read().decode() - stderr = self.stderr_file.read().decode() + stdout = self.stdout_file.read().decode('utf-8', errors='replace') + stderr = self.stderr_file.read().decode('utf-8', errors='replace') if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') @@ -134,8 +134,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read().decode() - stderr = self.stderr_file.read().decode() + stdout = self.stdout_file.read().decode('utf-8', errors='replace') + stderr = self.stderr_file.read().decode('utf-8', errors='replace') if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 560650a09d7..d582c0b7401 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -38,6 +38,7 @@ SANITIZER_SIGN = "==================" def _create_env_file(path, variables): + logging.debug("Env {} stored in {}".format(variables, path)) with open(path, 'w') as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") @@ -71,7 +72,7 @@ def get_open_port(): def subprocess_check_call(args): # Uncomment for debugging - # print('run:', ' ' . join(args)) + print('run:', ' '.join(args)) run_and_check(args) @@ -136,6 +137,7 @@ class ClickHouseCluster: self.docker_logs_path = p.join(self.instances_dir, 'docker.log') self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} + self.up_called = False custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') self.docker_api_version = os.environ.get("DOCKER_API_VERSION") @@ -202,6 +204,10 @@ class ClickHouseCluster: self.redis_host = "redis1" self.redis_port = get_open_port() + # available when with_postgres == True + self.postgres_host = "postgres1" + self.postgres_port = get_open_port() + # available when with_mysql == True self.mysql_host = "mysql57" self.mysql_port = get_open_port() @@ -268,6 +274,16 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] return self.base_rabbitmq_cmd + def setup_postgres_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) + env_variables['POSTGRES_HOST'] = self.postgres_host + env_variables['POSTGRES_EXTERNAL_PORT'] = str(self.postgres_port) + env_variables['POSTGRES_INTERNAL_PORT'] = "5432" + self.with_postgres = True + self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] + return self.base_postgres_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -364,11 +380,7 @@ class ClickHouseCluster: cmds.append(self.setup_mysql8_cmd(instance, env_variables, docker_compose_yml_dir)) if with_postgres and not self.with_postgres: - self.with_postgres = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] - cmds.append(self.base_postgres_cmd) + cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True @@ -376,11 +388,7 @@ class ClickHouseCluster: cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) if not self.with_postgres: - self.with_postgres = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] - cmds.append(self.base_postgres_cmd) + cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kafka and not self.with_kafka: self.with_kafka = True @@ -582,8 +590,7 @@ class ClickHouseCluster: start = time.time() while time.time() - start < timeout: try: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" - conn = psycopg2.connect(conn_string) + conn = psycopg2.connect(host='127.0.0.1', port=self.postgres_port, user='postgres', password='mysecretpassword') conn.close() print("Postgres Started") return @@ -738,7 +745,7 @@ class ClickHouseCluster: self.docker_client = docker.from_env(version=self.docker_api_version) - common_opts = ['up', '-d', '--force-recreate', '--remove-orphans'] + common_opts = ['up', '-d', '--force-recreate'] if self.with_zookeeper and self.base_zookeeper_cmd: print('Setup ZooKeeper') @@ -772,7 +779,7 @@ class ClickHouseCluster: if self.with_postgres and self.base_postgres_cmd: print('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) - self.wait_postgres_to_start(120) + self.wait_postgres_to_start(30) if self.with_kafka and self.base_kafka_cmd: print('Setup Kafka') @@ -826,6 +833,7 @@ class ClickHouseCluster: clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) + self.up_called = True subprocess.check_output(clickhouse_start_cmd) print("ClickHouse instance created") @@ -850,29 +858,30 @@ class ClickHouseCluster: def shutdown(self, kill=True): sanitizer_assert_instance = None - with open(self.docker_logs_path, "w+") as f: - try: - subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL - except Exception as e: - print("Unable to get logs from docker.") - f.seek(0) - for line in f: - if SANITIZER_SIGN in line: - sanitizer_assert_instance = line.split('|')[0].strip() - break + if self.up_called: + with open(self.docker_logs_path, "w+") as f: + try: + subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + except Exception as e: + print("Unable to get logs from docker.") + f.seek(0) + for line in f: + if SANITIZER_SIGN in line: + sanitizer_assert_instance = line.split('|')[0].strip() + break - if kill: - try: - subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) - except Exception as e: - print("Kill command failed during shutdown. {}".format(repr(e))) - print("Trying to kill forcefully") - subprocess_check_call(self.base_cmd + ['kill']) + if kill: + try: + subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) + except Exception as e: + print("Kill command failed during shutdown. {}".format(repr(e))) + print("Trying to kill forcefully") + subprocess_check_call(self.base_cmd + ['kill']) - try: - subprocess_check_call(self.base_cmd + ['down', '--volumes']) - except Exception as e: - print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) + try: + subprocess_check_call(self.base_cmd + ['down', '--volumes']) + except Exception as e: + print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) self.is_up = False @@ -1308,7 +1317,7 @@ class ClickHouseInstance: "UserName": "postgres", "Password": "mysecretpassword", "Port": "5432", - "Servername": "postgres1", + "Servername": self.cluster.postgres_host, "Protocol": "9.3", "ReadOnly": "No", "RowVersioning": "No", @@ -1441,8 +1450,6 @@ class ClickHouseInstance: self.cluster.env_variables.update(self.env_variables) _create_env_file(os.path.join(self.env_file), self.env_variables) - print("Env {} stored in {}".format(self.env_variables, self.env_file)) - odbc_ini_path = "" if self.odbc_ini_path: self._create_odbc_config_file() diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index b83c00409af..1be6ea6d295 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -18,11 +18,11 @@ click_dict_table_template = """ ) ENGINE = Dictionary({}) """ -def get_postgres_conn(database=False): +def get_postgres_conn(database, port): if database == True: - conn_string = "host='localhost' dbname='clickhouse' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' dbname='clickhouse' user='postgres' password='mysecretpassword'".format(port) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' user='postgres' password='mysecretpassword'".format(port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -36,8 +36,8 @@ def create_postgres_table(conn, table_name): cursor = conn.cursor() cursor.execute(postgres_dict_table_template.format(table_name)) -def create_and_fill_postgres_table(table_name): - conn = get_postgres_conn(True) +def create_and_fill_postgres_table(table_name, port): + conn = get_postgres_conn(True, port) create_postgres_table(conn, table_name) # Fill postgres table using clickhouse postgres table function and check table_func = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) @@ -54,7 +54,7 @@ def create_dict(table_name, index=0): def started_cluster(): try: cluster.start() - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(False, cluster.postgres_port) node1.query("CREATE DATABASE IF NOT EXISTS test") print("postgres connected") create_postgres_db(postgres_conn, 'clickhouse') @@ -65,10 +65,10 @@ def started_cluster(): def test_load_dictionaries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(True, started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(table_name) + create_and_fill_postgres_table(table_name, started_cluster.postgres_port) create_dict(table_name) dict_name = 'dict0' @@ -80,10 +80,10 @@ def test_load_dictionaries(started_cluster): def test_invalidate_query(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(True, started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(table_name) + create_and_fill_postgres_table(table_name, started_cluster.postgres_port) # invalidate query: SELECT value FROM test0 WHERE id = 0 dict_name = 'dict0' diff --git a/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml index 19eed6ebd6a..6aad3ad9917 100644 --- a/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml +++ b/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -4,7 +4,7 @@ clickhouse.test_table
- DSN=postgresql_odbc; + DSN=postgresql_odbc postgres
diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 3ef59e7bd23..7510ff02011 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -4,13 +4,14 @@ import psycopg2 import pymysql.cursors import pytest import logging +import os.path from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, with_postgres=True, main_configs=['configs/openssl.xml', 'configs/odbc_logging.xml', 'configs/enable_dictionaries.xml', 'configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', @@ -36,7 +37,7 @@ create_table_sql_template = """ def get_mysql_conn(): errors = [] conn = None - for _ in range(5): + for _ in range(15): try: if conn is None: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) @@ -64,11 +65,20 @@ def create_mysql_table(conn, table_name): def get_postgres_conn(): - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" - conn = psycopg2.connect(conn_string) - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True - return conn + conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + errors = [] + for _ in range(15): + try: + conn = psycopg2.connect(conn_string) + logging.debug("Postgre Connection establised: {}".format(conn_string)) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Postgre connection not establised DSN={}, {}".format(conn_string, errors)) def create_postgres_db(conn, name): @@ -319,12 +329,13 @@ def test_postgres_insert(started_cluster): "create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')") node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')") assert node1.query("select * from pg_insert") == '1\thello\n2\tworld\n' - node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') format CSV 3,test") + node1.query("insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,test") node1.query( - "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)") + "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" \ + " select number, 's' || toString(number) from numbers (4, 7)") assert node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n" assert node1.query( - "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table')))") == "55\t10\n" + "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))") == "55\t10\n" def test_bridge_dies_with_parent(started_cluster): diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index b3f7c0fa9af..2cb148db6ef 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -16,9 +16,9 @@ postgres_table_template = """ def get_postgres_conn(database=False): if database == True: - conn_string = "host='localhost' dbname='test_database' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port={} dbname='test_database' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index c63896eb196..633c02ef408 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -38,7 +38,7 @@ def server_address(): @pytest.fixture(scope='module') def psql_client(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_psql_1') @@ -62,7 +62,7 @@ def psql_server(psql_client): @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java_1') diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 4f567c19f2b..e46d6f98aae 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -11,9 +11,9 @@ node1 = cluster.add_instance('node1', main_configs=[], with_postgres=True) def get_postgres_conn(database=False): if database == True: - conn_string = "host='localhost' dbname='clickhouse' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' dbname='clickhouse' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True From 44fddb2c295f83093f429d91d95e6f9a2b5d786e Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 18 Feb 2021 20:36:29 +0900 Subject: [PATCH 023/652] Add some comments --- .../parametric-functions.md | 4 ++-- .../AggregateFunctionSequenceNextNode.h | 21 ++++++++++++++++--- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 1e84bf5e084..56fb5207648 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -503,14 +503,14 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, ... event32) +sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, event3, ...) ``` **Parameters** - `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. - `timestamp` — Name of the column containing the timestamp. Data types supported: Date, DateTime and other unsigned integer types. - `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: String and Nullable(String) -- `cond` — Conditions or data describing the chain of events. UInt8. +- `cond` — Conditions describing the chain of events. UInt8. **Returned value** diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index ffcc02b805a..be672974ac6 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,13 +27,15 @@ namespace DB { + +/// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template struct NodeBase { UInt64 size; // size of payload DataTypeDateTime::FieldType event_time; - UInt32 events_bitset; // UInt32 for combiniant comparesons between bitsets (< operator on bitsets). + UInt32 events_bitset; // Bitsets of UInt32 are easy to compare. (< operator on bitsets) char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -70,6 +72,7 @@ struct NodeBase } }; +/// It stores String, timestamp, bitset of matched events. struct NodeString : public NodeBase { using Node = NodeString; @@ -91,6 +94,7 @@ struct NodeString : public NodeBase } }; +/// TODO : Expends SequenceNextNodeGeneralData to support other types template struct SequenceNextNodeGeneralData { @@ -123,6 +127,7 @@ struct SequenceNextNodeGeneralData } }; +/// Implementation of sequenceNextNode template class SequenceNextNodeImpl final : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> @@ -178,6 +183,13 @@ public: const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; + /// The events_bitset variable stores matched events in the form of bitset. + /// It uses UInt32 instead of std::bitset because bitsets of UInt32 are easy to compare. (< operator on bitsets) + /// Each Nth-bit indicates that the Nth-event are matched. + /// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005. + /// 0x00000000 + /// + 1 (bit of event1) + /// + 4 (bit of event3) UInt32 events_bitset = 0; for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) @@ -265,8 +277,8 @@ public: return k; } - // This method returns an index of next node that matched the events. - // It is one as referring Boyer-Moore-Algorithm. + /// This method returns an index of next node that matched the events. + /// It is one as referring Boyer-Moore-Algorithm. UInt32 getNextNodeIndex(Data & data) const { if (data.value.size() <= events_size) @@ -278,10 +290,12 @@ public: while (i < data.value.size()) { UInt32 j = 0; + /// It checks whether the chain of events are matched or not. for (; j < events_size; ++j) if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) break; + /// If the chain of events are matched returns the index of result value. if (j == events_size) return i + 1; @@ -309,6 +323,7 @@ public: bool allocatesMemoryInArena() const override { return true; } }; +/// Implementation of sequenceFirstNode template class SequenceFirstNodeImpl final : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> From 3409aea6f6ba7cab2c203ce042d3ee9af50d7ae6 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 18 Feb 2021 21:18:44 +0900 Subject: [PATCH 024/652] Fix registerAggregateFunctions.cpp --- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 28b758aee2c..9bc951629c6 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -100,9 +100,9 @@ void registerAggregateFunctions() registerAggregateFunctionAggThrow(factory); registerAggregateFunctionRankCorrelation(factory); registerAggregateFunctionMannWhitney(factory); + registerAggregateFunctionSequenceNextNode(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); - registerAggregateFunctionSequenceNextNode(factory); } { From 872e36c207f5ed96f97883cb974038474160c84d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 18 Feb 2021 15:57:45 +0300 Subject: [PATCH 025/652] wip --- .../runner/compose/docker_compose_kafka.yml | 9 +- .../docker_compose_kerberized_kafka.yml | 6 +- tests/integration/helpers/cluster.py | 55 +++-- .../test_aggregate_function_state_avg.py | 2 +- .../test_short_strings_aggregation.py | 2 +- .../test_format_avro_confluent/test.py | 2 +- tests/integration/test_storage_kafka/test.py | 232 ++++++++++-------- .../test_storage_kerberized_kafka/test.py | 7 +- 8 files changed, 174 insertions(+), 141 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 219d977ffd9..46f3383e25a 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -15,10 +15,11 @@ services: image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - - "9092:9092" + - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kafka1:19092 - KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:19092 + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 + KAFKA_ADVERTISED_HOST_NAME: kafka1 + KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE KAFKA_BROKER_ID: 1 @@ -34,7 +35,7 @@ services: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry ports: - - "8081:8081" + - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_INTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 6e1e11344bb..1c1ab837592 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -23,13 +23,13 @@ services: # restart: always hostname: kerberized_kafka1 ports: - - "9092:9092" - - "9093:9093" + - ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT} environment: KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 - KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093 + KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT} # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 + KAFKA_ADVERTISED_HOST_NAME: kerberized_kafka1 KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d582c0b7401..0673deaa892 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -183,9 +183,19 @@ class ClickHouseCluster: self.minio_redirect_port = 8080 # available when with_kafka == True + self.kafka_host = "kafka1" + self.kafka_port = get_open_port() + self.kafka_docker_id = None self.schema_registry_client = None self.schema_registry_host = "schema-registry" - self.schema_registry_port = 8081 + self.schema_registry_port = get_open_port() + self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + + # available when with_kerberozed_kafka == True + self.kerberized_kafka_host = "kerberized_kafka1" + self.kerberized_kafka_port = get_open_port() + self.kerberized_kafka_docker_id = None + self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kafka_host) # available when with_mongo == True self.mongo_host = "mongo1" @@ -250,6 +260,27 @@ class ClickHouseCluster: return self.base_mysql8_cmd + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_redis = True + env_variables['KAFKA_HOST'] = self.kafka_host + env_variables['KAFKA_EXTERNAL_PORT'] = str(self.kafka_port) + env_variables['SCHEMA_REGISTRY_EXTERNAL_PORT'] = str(self.schema_registry_port) + env_variables['SCHEMA_REGISTRY_INTERNAL_PORT'] = "8081" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] + return self.base_redis_cmd + + def setup_kerberized_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_redis = True + env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' + env_variables['KERBERIZED_KAFKA_HOST'] = self.kerberized_kafka_host + env_variables['KERBERIZED_KAFKA_EXTERNAL_PORT'] = str(self.kerberized_kafka_port) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] + return self.base_redis_cmd + def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True env_variables['REDIS_HOST'] = self.redis_host @@ -391,19 +422,10 @@ class ClickHouseCluster: cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kafka and not self.with_kafka: - self.with_kafka = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) - self.base_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] - cmds.append(self.base_kafka_cmd) + cmds.append(self.setup_kafka_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kerberized_kafka and not self.with_kerberized_kafka: - env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - self.with_kerberized_kafka = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_kerberized_kafka_cmd = ['docker-compose', '--env-file', instance.env_file,'--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] - cmds.append(self.base_kerberized_kafka_cmd) + cmds.append(self.setup_kerberized_kafka_cmd(instance, env_variables, docker_compose_yml_dir)) if with_rabbitmq and not self.with_rabbitmq: cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -695,14 +717,13 @@ class ClickHouseCluster: raise Exception("Can't wait Minio to start") def wait_schema_registry_to_start(self, timeout=10): - sr_client = CachedSchemaRegistryClient('http://localhost:8081') + sr_client = CachedSchemaRegistryClient('http://localhost:{}'.format(cluster.schema_registry_port)) start = time.time() while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) - self.schema_registry_client = sr_client print("Connected to SchemaRegistry") - return + return sr_client except Exception as ex: print(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) @@ -784,13 +805,11 @@ class ClickHouseCluster: if self.with_kafka and self.base_kafka_cmd: print('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.kafka_docker_id = self.get_instance_docker_id('kafka1') - self.wait_schema_registry_to_start(120) + self.schema_registry_client = self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: print('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py index 3b35c112887..5ed97e7a9a5 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="aggregate_state") node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 91a0a87b6e2..463fadc36e8 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="short_strings") node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True) node2 = cluster.add_instance('node2', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index cc006801735..ec2ed875075 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -65,7 +65,7 @@ def test_select(cluster): instance = cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( cluster.schema_registry_host, - cluster.schema_registry_port + 8081 ) run_query(instance, "create table avro_data(value Int64) engine = Memory()") diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 5f2726832cc..e1b5d1068a0 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5,6 +5,7 @@ import socket import subprocess import threading import time +import logging import avro.schema from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient @@ -41,7 +42,7 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], with_kafka=True, - with_zookeeper=True, +# with_zookeeper=True, macros={"kafka_broker":"kafka1", "kafka_topic_old":"old", "kafka_group_name_old":"old", @@ -50,7 +51,7 @@ instance = cluster.add_instance('instance', "kafka_client_id":"instance", "kafka_format_json_each_row":"JSONEachRow"}, clickhouse_path_dir='clickhouse_path') -kafka_id = '' +kafka_id = cluster.kafka_docker_id # Helpers @@ -62,8 +63,8 @@ def check_kafka_is_available(): kafka_id, '/usr/bin/kafka-broker-api-versions', '--bootstrap-server', - 'INSIDE://localhost:9092'), - stdout=subprocess.PIPE) + 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), + stdout=subprocess.PIPE, stderr=subprocess.PIPE) p.communicate() return p.returncode == 0 @@ -76,21 +77,35 @@ def wait_kafka_is_available(max_retries=50): else: retries += 1 if retries > max_retries: - raise "Kafka is not available" - print("Waiting for Kafka to start up") + raise Exception("Kafka is not available") + logging.debug("Waiting for Kafka to start up") time.sleep(1) +def get_kafka_producer(port, serializer): + errors = [] + for _ in range(15): + try: + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) + logging.debug("Kafka Connection establised: localhost:{}".format(port)) + return producer + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) + def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) +def kafka_produce(kafka_cluster, topic, messages, timestamp=None): + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() -def kafka_consume(topic): - consumer = KafkaConsumer(bootstrap_servers="localhost:9092", auto_offset_reset="earliest") +def kafka_consume(kafka_cluster, topic): + consumer = KafkaConsumer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest") consumer.subscribe(topics=(topic)) for toppar, messages in list(consumer.poll(5000).items()): if toppar.topic == topic: @@ -100,7 +115,7 @@ def kafka_consume(topic): consumer.close() -def kafka_produce_protobuf_messages(topic, start_index, num_messages): +def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() @@ -108,14 +123,14 @@ def kafka_produce_protobuf_messages(topic, start_index, num_messages): msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print(("Produced {} messages for topic {}".format(num_messages, topic))) + logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) -def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messages): +def kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, topic, start_index, num_messages): data = '' - producer = KafkaProducer(bootstrap_servers="localhost:9092") + producer = KafkaProducer(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() msg.key = i @@ -123,9 +138,9 @@ def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messag serialized_msg = msg.SerializeToString() producer.send(topic=topic, value=serialized_msg) producer.flush() - print("Produced {} messages for topic {}".format(num_messages, topic)) + logging.debug("Produced {} messages for topic {}".format(num_messages, topic)) -def kafka_produce_protobuf_social(topic, start_index, num_messages): +def kafka_produce_protobuf_social(kafka_cluster,topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): msg = social_pb2.User() @@ -133,10 +148,10 @@ def kafka_produce_protobuf_social(topic, start_index, num_messages): msg.timestamp=1000000+i serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print(("Produced {} messages for topic {}".format(num_messages, topic))) + logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) def avro_confluent_message(schema_registry_client, value): @@ -159,7 +174,7 @@ def avro_confluent_message(schema_registry_client, value): @pytest.mark.timeout(180) def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', + kafka_produce(kafka_cluster, 'kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) instance.query(''' @@ -180,11 +195,13 @@ def test_kafka_json_as_string(kafka_cluster): ''' assert TSV(result) == TSV(expected) assert instance.contains_in_log( - "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") + "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: [0-9]*) return no rows") @pytest.mark.timeout(300) def test_kafka_formats(kafka_cluster): + schema_registry_client = CachedSchemaRegistryClient('http://localhost:{}'.format(kafka_cluster.schema_registry_port)) + # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -453,19 +470,19 @@ def test_kafka_formats(kafka_cluster): # }, 'AvroConfluent': { 'data_sample': [ - avro_confluent_message(cluster.schema_registry_client, + avro_confluent_message(schema_registry_client, {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), - b''.join([avro_confluent_message(cluster.schema_registry_client, + b''.join([avro_confluent_message(schema_registry_client, {'id': id, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}) for id in range(1, 16)]), - avro_confluent_message(cluster.schema_registry_client, + avro_confluent_message(schema_registry_client, {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), ], 'extra_settings': ", format_avro_schema_registry_url='http://{}:{}'".format( - cluster.schema_registry_host, - cluster.schema_registry_port + kafka_cluster.schema_registry_host, + 8081 ), 'supports_empty_value': True, } @@ -514,14 +531,14 @@ def test_kafka_formats(kafka_cluster): } for format_name, format_opts in list(all_formats.items()): - print(('Set up {}'.format(format_name))) + logging.debug(('Set up {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) data_sample = format_opts['data_sample'] data_prefix = [] # prepend empty value when supported if format_opts.get('supports_empty_value', False): data_prefix = data_prefix + [''] - kafka_produce(topic_name, data_prefix + data_sample) + kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; @@ -548,7 +565,7 @@ def test_kafka_formats(kafka_cluster): time.sleep(12) for format_name, format_opts in list(all_formats.items()): - print(('Checking {}'.format(format_name))) + logging.debug(('Checking {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) # shift offsets by 1 if format supports empty value offsets = [1, 2, 3] if format_opts.get('supports_empty_value', False) else [0, 1, 2] @@ -587,8 +604,8 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' # https://stackoverflow.com/a/57692111/1555175 -def describe_consumer_group(name): - client = BrokerConnection('localhost', 9092, socket.AF_INET) +def describe_consumer_group(kafka_cluster, name): + client = BrokerConnection('localhost', kafka_cluster.kafka_port, socket.AF_INET) client.connect_blocking() list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) @@ -619,10 +636,7 @@ def describe_consumer_group(name): @pytest.fixture(scope="module") def kafka_cluster(): try: - global kafka_id cluster.start() - kafka_id = instance.cluster.kafka_docker_id - print(("kafka_id is {}".format(kafka_id))) yield cluster finally: @@ -633,7 +647,7 @@ def kafka_cluster(): def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') wait_kafka_is_available() - # print("kafka is available - running test") + # logging.debug("kafka is available - running test") yield # run test @@ -661,7 +675,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('old', messages) + kafka_produce(kafka_cluster, 'old', messages) result = '' while True: @@ -671,9 +685,9 @@ kafka_topic_old old kafka_check_result(result, True) - members = describe_consumer_group('old') + members = describe_consumer_group(kafka_cluster, 'old') assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' - # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) + # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:{} --describe --members --group old --verbose".format(cluster.kafka_port))) @pytest.mark.timeout(180) @@ -693,16 +707,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) # Insert couple of malformed messages. - kafka_produce('new', ['}{very_broken_message,']) - kafka_produce('new', ['}another{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) result = '' while True: @@ -712,14 +726,14 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) - members = describe_consumer_group('new') + members = describe_consumer_group(kafka_cluster, 'new') assert members[0]['client_id'] == 'instance test 1234' @pytest.mark.timeout(180) def test_kafka_issue11308(kafka_cluster): # Check that matview does respect Kafka SETTINGS - kafka_produce('issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', + kafka_produce(kafka_cluster, 'issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', '{"t": 124, "e": {"x": "test"} }']) instance.query(''' @@ -768,7 +782,7 @@ def test_kafka_issue11308(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_issue4116(kafka_cluster): # Check that format_csv_delimiter parameter works now - as part of all available format settings. - kafka_produce('issue4116', ['1|foo', '2|bar', '42|answer', '100|multi\n101|row\n103|message']) + kafka_produce(kafka_cluster, 'issue4116', ['1|foo', '2|bar', '42|answer', '100|multi\n101|row\n103|message']) instance.query(''' CREATE TABLE test.kafka (a UInt64, b String) @@ -823,7 +837,7 @@ def test_kafka_consumer_hang(kafka_cluster): time.sleep(0.5) kafka_cluster.unpause_container('kafka1') - # print("Attempt to drop") + # logging.debug("Attempt to drop") instance.query('DROP TABLE test.kafka') # kafka_cluster.open_bash_shell('instance') @@ -899,7 +913,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) result = '' while True: @@ -925,7 +939,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) - kafka_produce('tsv', messages) + kafka_produce(kafka_cluster, 'tsv', messages) result = '' while True: @@ -965,12 +979,12 @@ def test_kafka_json_without_delimiter(kafka_cluster): messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('json', [messages]) + kafka_produce(kafka_cluster, 'json', [messages]) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('json', [messages]) + kafka_produce(kafka_cluster, 'json', [messages]) result = '' while True: @@ -993,9 +1007,9 @@ def test_kafka_protobuf(kafka_cluster): kafka_schema = 'kafka.proto:KeyValuePair'; ''') - kafka_produce_protobuf_messages('pb', 0, 20) - kafka_produce_protobuf_messages('pb', 20, 1) - kafka_produce_protobuf_messages('pb', 21, 29) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 0, 20) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 20, 1) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 21, 29) result = '' while True: @@ -1025,9 +1039,9 @@ SETTINGS SELECT * FROM test.kafka; ''') - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 21, 29) result = instance.query('SELECT * FROM test.kafka', ignore_error=True) expected = '''\ @@ -1096,9 +1110,9 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): kafka_schema = 'kafka.proto:KeyValuePair'; ''') - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 0, 20) - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 20, 1) - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 21, 29) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 0, 20) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 20, 1) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 21, 29) result = '' while True: @@ -1155,7 +1169,7 @@ def test_kafka_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mv', messages) + kafka_produce(kafka_cluster, 'mv', messages) while True: result = instance.query('SELECT * FROM test.view') @@ -1184,7 +1198,7 @@ def test_librdkafka_snappy_regression(kafka_cluster): """ # create topic with snappy compression - admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:9092'}) + admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:{}'.format(kafka_cluster.kafka_port)}) topic_snappy = admin.NewTopic(topic='snappy_regression', num_partitions=1, replication_factor=1, config={ 'compression.type': 'snappy', }) @@ -1215,14 +1229,14 @@ def test_librdkafka_snappy_regression(kafka_cluster): for i in range(number_of_messages): messages.append(json.dumps({'key': i, 'value': value})) expected.append(f'{i}\t{value}') - kafka_produce('snappy_regression', messages) + kafka_produce(kafka_cluster, 'snappy_regression', messages) expected = '\n'.join(expected) while True: result = instance.query('SELECT * FROM test.kafka') rows = len(result.strip('\n').split('\n')) - print(rows) + logging.debug(rows) if rows == number_of_messages: break @@ -1252,7 +1266,7 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mvsq', messages) + kafka_produce(kafka_cluster, 'mvsq', messages) while True: result = instance.query('SELECT * FROM test.view') @@ -1296,7 +1310,7 @@ def test_kafka_many_materialized_views(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mmv', messages) + kafka_produce(kafka_cluster, 'mmv', messages) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -1321,7 +1335,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): kafka_messages = 1000 batch_messages = 1000 messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(kafka_messages)] - kafka_produce('flush', messages) + kafka_produce(kafka_cluster, 'flush', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -1340,7 +1354,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): SELECT * FROM test.kafka; ''') - client = KafkaAdminClient(bootstrap_servers="localhost:9092") + client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) received = False while not received: try: @@ -1379,12 +1393,12 @@ def test_kafka_virtual_columns(kafka_cluster): messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('virt1', [messages], 0) + kafka_produce(kafka_cluster, 'virt1', [messages], 0) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('virt1', [messages], 0) + kafka_produce(kafka_cluster, 'virt1', [messages], 0) result = '' while True: @@ -1419,7 +1433,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('virt2', messages, 0) + kafka_produce(kafka_cluster, 'virt2', messages, 0) while True: result = instance.query('SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view') @@ -1563,7 +1577,7 @@ def test_kafka_commit_on_block_write(kafka_cluster): for _ in range(101): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - kafka_produce('block', messages) + kafka_produce(kafka_cluster, 'block', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() @@ -1608,7 +1622,7 @@ def test_kafka_commit_on_block_write(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_virtual_columns2(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="virt2_0", num_partitions=2, replication_factor=1)) topic_list.append(NewTopic(name="virt2_1", num_partitions=2, replication_factor=1)) @@ -1628,7 +1642,7 @@ def test_kafka_virtual_columns2(kafka_cluster): SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; ''') - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer, key_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(cluster.kafka_port), value_serializer=producer_serializer, key_serializer=producer_serializer) producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001, headers=[('content-encoding', b'base64')]) @@ -1655,7 +1669,7 @@ def test_kafka_virtual_columns2(kafka_cluster): time.sleep(10) - members = describe_consumer_group('virt2') + members = describe_consumer_group(kafka_cluster, 'virt2') # pprint.pprint(members) members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' @@ -1679,7 +1693,7 @@ def test_kafka_virtual_columns2(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="insert3", num_partitions=1, replication_factor=1)) @@ -1724,7 +1738,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - # print(result) + # logging.debug(result) expected = '''\ 1 1 k1 1577836801 k1 insert3 0 0 1577836801 @@ -1739,7 +1753,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): @pytest.mark.timeout(600) def test_kafka_flush_by_time(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="flush_by_time", num_partitions=1, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1770,7 +1784,7 @@ def test_kafka_flush_by_time(kafka_cluster): while not cancel.is_set(): messages = [] messages.append(json.dumps({'key': 0, 'value': 0})) - kafka_produce('flush_by_time', messages) + kafka_produce(kafka_cluster, 'flush_by_time', messages) time.sleep(0.8) kafka_thread = threading.Thread(target=produce) @@ -1806,7 +1820,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): while not cancel.is_set(): messages = [] messages.append(json.dumps({'key': 0, 'value': 0})) - kafka_produce('flush_by_block_size', messages) + kafka_produce(kafka_cluster, 'flush_by_block_size', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() @@ -1844,7 +1858,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") - # print(result) + # logging.debug(result) instance.query(''' DROP TABLE test.consumer; @@ -1859,7 +1873,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): @pytest.mark.timeout(600) def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions2", num_partitions=10, replication_factor=1)) @@ -1890,12 +1904,12 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): count = count + 1 rows.append(json.dumps({'key': count, 'value': count})) messages.append("\n".join(rows)) - kafka_produce('topic_with_multiple_partitions2', messages) + kafka_produce(kafka_cluster, 'topic_with_multiple_partitions2', messages) time.sleep(30) result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) assert TSV(result) == TSV('{0}\t{0}\t{0}'.format(count)) instance.query(''' @@ -1928,7 +1942,7 @@ def test_kafka_rebalance(kafka_cluster): # time.sleep(2) - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions", num_partitions=11, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1943,14 +1957,14 @@ def test_kafka_rebalance(kafka_cluster): for _ in range(59): messages.append(json.dumps({'key': msg_index[0], 'value': msg_index[0]})) msg_index[0] += 1 - kafka_produce('topic_with_multiple_partitions', messages) + kafka_produce(kafka_cluster, 'topic_with_multiple_partitions', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): table_name = 'kafka_consumer{}'.format(consumer_index) - print(("Setting up {}".format(table_name))) + logging.debug(("Setting up {}".format(table_name))) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -1977,21 +1991,21 @@ def test_kafka_rebalance(kafka_cluster): # kafka_cluster.open_bash_shell('instance') while int( instance.query("SELECT count() FROM test.destination WHERE _consumed_by='{}'".format(table_name))) == 0: - print(("Waiting for test.kafka_consumer{} to start consume".format(consumer_index))) + logging.debug(("Waiting for test.kafka_consumer{} to start consume".format(consumer_index))) time.sleep(1) cancel.set() # I leave last one working by intent (to finish consuming after all rebalances) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): - print(("Dropping test.kafka_consumer{}".format(consumer_index))) + logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) instance.query('DROP TABLE IF EXISTS test.kafka_consumer{}'.format(consumer_index)) while int(instance.query( "SELECT count() FROM system.tables WHERE database='test' AND name='kafka_consumer{}'".format( consumer_index))) == 1: time.sleep(1) - # print(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) + # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) # kafka_cluster.open_bash_shell('instance') while 1: @@ -1999,9 +2013,9 @@ def test_kafka_rebalance(kafka_cluster): if messages_consumed >= msg_index[0]: break time.sleep(1) - print(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) + logging.debug(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) - print((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) + logging.debug((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) # Some queries to debug... # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) @@ -2026,7 +2040,7 @@ def test_kafka_rebalance(kafka_cluster): result = int(instance.query('SELECT count() == uniqExact(key) FROM test.destination')) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - print(("kafka_consumer{}".format(consumer_index))) + logging.debug(("kafka_consumer{}".format(consumer_index))) table_name = 'kafka_consumer{}'.format(consumer_index) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -2045,7 +2059,7 @@ def test_kafka_rebalance(kafka_cluster): @pytest.mark.timeout(1200) def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] - kafka_produce('no_holes_when_write_suffix_failed', messages) + kafka_produce(kafka_cluster, 'no_holes_when_write_suffix_failed', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -2064,7 +2078,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): ''') messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('no_holes_when_write_suffix_failed', messages) + kafka_produce(kafka_cluster, 'no_holes_when_write_suffix_failed', messages) # init PartitionManager (it starts container) earlier pm = PartitionManager() @@ -2095,7 +2109,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): assert instance.contains_in_log("ZooKeeper session has been expired.: while write prefix to view") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) # kafka_cluster.open_bash_shell('instance') @@ -2143,7 +2157,7 @@ def test_exception_from_destructor(kafka_cluster): @pytest.mark.timeout(120) def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(1)] - kafka_produce('commits_of_unprocessed_messages_on_drop', messages) + kafka_produce(kafka_cluster, 'commits_of_unprocessed_messages_on_drop', messages) instance.query(''' DROP TABLE IF EXISTS test.destination; @@ -2181,7 +2195,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): ''') while int(instance.query("SELECT count() FROM test.destination")) == 0: - print("Waiting for test.kafka_consumer to start consume") + logging.debug("Waiting for test.kafka_consumer to start consume") time.sleep(1) cancel = threading.Event() @@ -2194,7 +2208,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): for _ in range(113): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - kafka_produce('commits_of_unprocessed_messages_on_drop', messages) + kafka_produce(kafka_cluster, 'commits_of_unprocessed_messages_on_drop', messages) time.sleep(1) kafka_thread = threading.Thread(target=produce) @@ -2222,7 +2236,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): # SELECT key, _timestamp, _offset FROM test.destination where runningDifference(key) <> 1 ORDER BY key; result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.destination') - print(result) + logging.debug(result) instance.query(''' DROP TABLE test.kafka_consumer; @@ -2236,7 +2250,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): @pytest.mark.timeout(120) def test_bad_reschedule(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce('test_bad_reschedule', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -2261,7 +2275,7 @@ def test_bad_reschedule(kafka_cluster): ''') while int(instance.query("SELECT count() FROM test.destination")) < 20000: - print("Waiting for consume") + logging.debug("Waiting for consume") time.sleep(1) assert int(instance.query("SELECT max(consume_ts) - min(consume_ts) FROM test.destination")) < 8 @@ -2270,7 +2284,7 @@ def test_bad_reschedule(kafka_cluster): @pytest.mark.timeout(300) def test_kafka_duplicates_when_commit_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] - kafka_produce('duplicates_when_commit_failed', messages) + kafka_produce(kafka_cluster, 'duplicates_when_commit_failed', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -2289,7 +2303,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): ''') messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('duplicates_when_commit_failed', messages) + kafka_produce(kafka_cluster, 'duplicates_when_commit_failed', messages) instance.query(''' CREATE TABLE test.view (key UInt64, value String) @@ -2325,7 +2339,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): assert instance.contains_in_log("All commit attempts failed") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) instance.query(''' DROP TABLE test.consumer; @@ -2374,7 +2388,7 @@ def test_premature_flush_on_eof(kafka_cluster): # next poll can return more data, and we should respect kafka_flush_interval_ms # and try to form bigger block messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(1)] - kafka_produce('premature_flush_on_eof', messages) + kafka_produce(kafka_cluster, 'premature_flush_on_eof', messages) instance.query(''' CREATE MATERIALIZED VIEW test.kafka_consumer TO test.destination AS @@ -2397,7 +2411,7 @@ def test_premature_flush_on_eof(kafka_cluster): # TODO: wait for messages in log: "Polled batch of 1 messages", followed by "Stalled" # produce more messages after delay - kafka_produce('premature_flush_on_eof', messages) + kafka_produce(kafka_cluster, 'premature_flush_on_eof', messages) # data was not flushed yet (it will be flushed 7.5 sec after creating MV) assert int(instance.query("SELECT count() FROM test.destination")) == 0 @@ -2417,7 +2431,7 @@ def test_premature_flush_on_eof(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_unavailable(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce('test_bad_reschedule', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) kafka_cluster.pause_container('kafka1') @@ -2451,7 +2465,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.unpause_container('kafka1') while int(instance.query("SELECT count() FROM test.destination")) < 20000: - print("Waiting for consume") + logging.debug("Waiting for consume") time.sleep(1) @@ -2503,7 +2517,7 @@ def test_kafka_csv_with_thread_per_consumer(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) result = '' while True: diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index 865afc8b162..accb855af10 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -24,7 +24,7 @@ instance = cluster.add_instance('instance', with_kerberized_kafka=True, clickhouse_path_dir="clickhouse_path" ) -kafka_id = '' # instance.cluster.kafka_docker_id +kafka_id = cluster.kerberized_kafka_docker_id # Helpers @@ -51,13 +51,14 @@ def wait_kafka_is_available(max_retries=50): else: retries += 1 if retries > max_retries: - raise "Kafka is not available" + raise Exception("Kafka is not available") print("Waiting for Kafka to start up") time.sleep(1) def producer_serializer(x): return x.encode() if isinstance(x, str) else x + def kafka_produce(topic, messages, timestamp=None): producer = KafkaProducer(bootstrap_servers="localhost:9093", value_serializer=producer_serializer) for message in messages: @@ -72,9 +73,7 @@ def kafka_produce(topic, messages, timestamp=None): @pytest.fixture(scope="module") def kafka_cluster(): try: - global kafka_id cluster.start() - kafka_id = instance.cluster.kerberized_kafka_docker_id print("kafka_id is {}".format(kafka_id)) yield cluster From 609c9b272fc3475df14828f8e5010e42019c9602 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 18 Feb 2021 16:42:31 +0300 Subject: [PATCH 026/652] fix --- tests/integration/helpers/cluster.py | 3 +-- tests/integration/test_storage_kafka/test.py | 8 ++++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0673deaa892..7409945d019 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -186,7 +186,6 @@ class ClickHouseCluster: self.kafka_host = "kafka1" self.kafka_port = get_open_port() self.kafka_docker_id = None - self.schema_registry_client = None self.schema_registry_host = "schema-registry" self.schema_registry_port = get_open_port() self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) @@ -805,7 +804,7 @@ class ClickHouseCluster: if self.with_kafka and self.base_kafka_cmd: print('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.schema_registry_client = self.wait_schema_registry_to_start(30) + self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: print('Setup kerberized kafka') diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index e1b5d1068a0..24afbc8ca4e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -42,7 +42,7 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], with_kafka=True, -# with_zookeeper=True, + with_zookeeper=True, # For Replicated Table macros={"kafka_broker":"kafka1", "kafka_topic_old":"old", "kafka_group_name_old":"old", @@ -130,7 +130,7 @@ def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messa def kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, topic, start_index, num_messages): data = '' - producer = KafkaProducer(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() msg.key = i @@ -1477,7 +1477,7 @@ def test_kafka_insert(kafka_cluster): messages = [] while True: - messages.extend(kafka_consume('insert1')) + messages.extend(kafka_consume(kafka_cluster, 'insert1')) if len(messages) == 50: break @@ -1942,7 +1942,7 @@ def test_kafka_rebalance(kafka_cluster): # time.sleep(2) - admin_client = KafkaAdminClient(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions", num_partitions=11, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) From 2df4317abad43a6408f4e32cbff298f32df73ebb Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 00:21:50 +0300 Subject: [PATCH 027/652] wip --- .../runner/compose/docker_compose_hdfs.yml | 8 +- .../docker_compose_kerberized_kafka.yml | 2 +- tests/integration/helpers/cluster.py | 214 ++++++++++-------- tests/integration/pytest.ini | 7 + tests/integration/test_storage_hdfs/test.py | 6 - tests/integration/test_storage_kafka/test.py | 1 - .../test_storage_kerberized_kafka/test.py | 28 ++- .../integration/test_storage_mongodb/test.py | 12 +- 8 files changed, 157 insertions(+), 121 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_hdfs.yml index b8cd7f64273..96384b0a7d4 100644 --- a/docker/test/integration/runner/compose/docker_compose_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_hdfs.yml @@ -5,6 +5,10 @@ services: hostname: hdfs1 restart: always ports: - - 50075:50075 - - 50070:50070 + - ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070 + - ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075 entrypoint: /etc/bootstrap.sh -d + volumes: + - type: ${HDFS_FS:-tmpfs} + source: ${HDFS_LOGS:-} + target: /usr/local/hadoop/logs \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 1c1ab837592..710f9dacf48 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -25,7 +25,7 @@ services: ports: - ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT} environment: - KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 + KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://0.0.0.0:${KERBERIZED_KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT} # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7409945d019..cb5c25c260e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -55,9 +55,9 @@ def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually - print('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) - print('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) - print('Env:\n{}\n'.format(env)) + logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) + logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) + logging.debug('Env:\n{}\n'.format(env)) raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 @@ -72,13 +72,13 @@ def get_open_port(): def subprocess_check_call(args): # Uncomment for debugging - print('run:', ' '.join(args)) + logging.info('run:' + ' '.join(args)) run_and_check(args) def subprocess_call(args): # Uncomment for debugging..; - # print('run:', ' ' . join(args)) + # logging.debug('run:', ' ' . join(args)) subprocess.call(args) def get_odbc_bridge_path(): @@ -100,7 +100,7 @@ def get_docker_compose_path(): if os.path.exists(os.path.dirname('/compose/')): return os.path.dirname('/compose/') # default in docker runner container else: - print(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) + logging.debug(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) return LOCAL_DOCKER_COMPOSE_DIR @@ -116,7 +116,7 @@ class ClickHouseCluster: def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): for param in list(os.environ.keys()): - print("ENV %40s %s" % (param, os.environ[param])) + logging.debug("ENV %40s %s" % (param, os.environ[param])) self.base_dir = p.dirname(base_path) self.name = name if name is not None else '' @@ -155,6 +155,7 @@ class ClickHouseCluster: self.base_kerberized_kafka_cmd = [] self.base_rabbitmq_cmd = [] self.base_cassandra_cmd = [] + self.base_redis_cmd = [] self.pre_zookeeper_commands = [] self.instances = {} self.with_zookeeper = False @@ -182,6 +183,13 @@ class ClickHouseCluster: self.minio_redirect_host = "proxy1" self.minio_redirect_port = 8080 + # available when with_hdfs == True + self.hdfs_host = "hdfs1" + self.hdfs_name_port = get_open_port() + self.hdfs_data_port = get_open_port() + self.hdfs_dir = p.abspath(p.join(self.instances_dir, "hdfs")) + self.hdfs_logs_dir = os.path.join(self.hdfs_dir, "logs") + # available when with_kafka == True self.kafka_host = "kafka1" self.kafka_port = get_open_port() @@ -194,7 +202,7 @@ class ClickHouseCluster: self.kerberized_kafka_host = "kerberized_kafka1" self.kerberized_kafka_port = get_open_port() self.kerberized_kafka_docker_id = None - self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kerberized_kafka_host) # available when with_mongo == True self.mongo_host = "mongo1" @@ -229,7 +237,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - print("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) + logging.debug("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) def get_client_cmd(self): cmd = self.client_bin_path @@ -259,6 +267,21 @@ class ClickHouseCluster: return self.base_mysql8_cmd + def setup_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_hdfs = True + env_variables['HDFS_HOST'] = self.hdfs_host + env_variables['HDFS_NAME_EXTERNAL_PORT'] = str(self.hdfs_name_port) + env_variables['HDFS_NAME_INTERNAL_PORT'] = "50070" + env_variables['HDFS_DATA_EXTERNAL_PORT'] = str(self.hdfs_data_port) + env_variables['HDFS_DATA_INTERNAL_PORT'] = "50075" + env_variables['HDFS_LOGS'] = self.hdfs_logs_dir + env_variables['HDFS_FS'] = "bind" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) + self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] + print("HDFS BASE CMD:{}".format(self.base_hdfs_cmd)) + return self.base_hdfs_cmd + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True env_variables['KAFKA_HOST'] = self.kafka_host @@ -266,9 +289,9 @@ class ClickHouseCluster: env_variables['SCHEMA_REGISTRY_EXTERNAL_PORT'] = str(self.schema_registry_port) env_variables['SCHEMA_REGISTRY_INTERNAL_PORT'] = "8081" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) - self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + self.base_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] - return self.base_redis_cmd + return self.base_kafka_cmd def setup_kerberized_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True @@ -276,9 +299,9 @@ class ClickHouseCluster: env_variables['KERBERIZED_KAFKA_HOST'] = self.kerberized_kafka_host env_variables['KERBERIZED_KAFKA_EXTERNAL_PORT'] = str(self.kerberized_kafka_port) self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + self.base_kerberized_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] - return self.base_redis_cmd + return self.base_kerberized_kafka_cmd def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True @@ -314,6 +337,16 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] return self.base_postgres_cmd + def setup_mongo_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mongo = True + env_variables['MONGO_HOST'] = self.mongo_host + env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) + env_variables['MONGO_INTERNAL_PORT'] = "27017" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) + self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] + return self.base_mongo_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -430,11 +463,7 @@ class ClickHouseCluster: cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) if with_hdfs and not self.with_hdfs: - self.with_hdfs = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) - self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] - cmds.append(self.base_hdfs_cmd) + cmds.append(self.setup_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kerberized_hdfs and not self.with_kerberized_hdfs: self.with_kerberized_hdfs = True @@ -445,14 +474,7 @@ class ClickHouseCluster: cmds.append(self.base_kerberized_hdfs_cmd) if with_mongo and not self.with_mongo: - self.with_mongo = True - env_variables['MONGO_HOST'] = self.mongo_host - env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) - env_variables['MONGO_INTERNAL_PORT'] = "27017" - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) - self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] - cmds.append(self.base_mongo_cmd) + cmds.append(self.setup_mongo_cmd(instance, env_variables, docker_compose_yml_dir)) if self.with_net_trics: for cmd in cmds: @@ -486,7 +508,7 @@ class ClickHouseCluster: self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] - print("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( + logging.debug("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( self.name, self.project_name, name, tag, self.base_cmd, docker_compose_yml_dir)) return instance @@ -525,10 +547,10 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["restart", service_name]) def get_instance_ip(self, instance_name): - print("get_instance_ip instance_name={}".format(instance_name)) + logging.debug("get_instance_ip instance_name={}".format(instance_name)) docker_id = self.get_instance_docker_id(instance_name) # for cont in self.docker_client.containers.list(): - # print("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) + # logging.debug("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) handle = self.docker_client.containers.get(docker_id) return list(handle.attrs['NetworkSettings']['Networks'].values())[0]['IPAddress'] @@ -550,16 +572,16 @@ class ClickHouseCluster: container_info = self.docker_client.api.inspect_container(container_id) image_id = container_info.get('Image') image_info = self.docker_client.api.inspect_image(image_id) - print(("Command failed in container {}: ".format(container_id))) - pprint.pprint(container_info) - print("") - print(("Container {} uses image {}: ".format(container_id, image_id))) - pprint.pprint(image_info) - print("") + logging.debug(("Command failed in container {}: ".format(container_id))) + pprint.plogging.debug(container_info) + logging.debug("") + logging.debug(("Container {} uses image {}: ".format(container_id, image_id))) + pprint.plogging.debug(image_info) + logging.debug("") message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output) if nothrow: - print(message) + logging.debug(message) else: raise Exception(message) if not detach: @@ -582,7 +604,7 @@ class ClickHouseCluster: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) conn.close() - print("Mysql Started") + logging.debug("Mysql Started") return except Exception as ex: errors += [str(ex)] @@ -598,10 +620,10 @@ class ClickHouseCluster: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql8_port) conn.close() - print("Mysql 8 Started") + logging.debug("Mysql 8 Started") return except Exception as ex: - print("Can't connect to MySQL 8 " + str(ex)) + logging.debug("Can't connect to MySQL 8 " + str(ex)) time.sleep(0.5) subprocess_call(['docker-compose', 'ps', '--services', '--all']) @@ -613,10 +635,10 @@ class ClickHouseCluster: try: conn = psycopg2.connect(host='127.0.0.1', port=self.postgres_port, user='postgres', password='mysecretpassword') conn.close() - print("Postgres Started") + logging.debug("Postgres Started") return except Exception as ex: - print("Can't connect to Postgres " + str(ex)) + logging.debug("Can't connect to Postgres " + str(ex)) time.sleep(0.5) raise Exception("Cannot wait Postgres container") @@ -628,10 +650,10 @@ class ClickHouseCluster: for instance in ['zoo1', 'zoo2', 'zoo3']: conn = self.get_kazoo_client(instance) conn.get_children('/') - print("All instances of ZooKeeper started") + logging.debug("All instances of ZooKeeper started") return except Exception as ex: - print("Can't connect to ZooKeeper " + str(ex)) + logging.debug("Can't connect to ZooKeeper " + str(ex)) time.sleep(0.5) raise Exception("Cannot wait ZooKeeper container") @@ -641,9 +663,9 @@ class ClickHouseCluster: keytab = p.abspath(p.join(self.instances['node1'].path, "secrets/clickhouse.keytab")) krb_conf = p.abspath(p.join(self.instances['node1'].path, "secrets/krb_long.conf")) hdfs_ip = self.get_instance_ip('kerberizedhdfs1') - # print("kerberizedhdfs1 ip ", hdfs_ip) + # logging.debug("kerberizedhdfs1 ip ", hdfs_ip) kdc_ip = self.get_instance_ip('hdfskerberos') - # print("kdc_ip ", kdc_ip) + # logging.debug("kdc_ip ", kdc_ip) self.hdfs_api = HDFSApi(user="root", timeout=timeout, kerberized=True, @@ -657,7 +679,7 @@ class ClickHouseCluster: hdfs_ip=hdfs_ip, kdc_ip=kdc_ip) else: - self.hdfs_api = HDFSApi(user="root", host="hdfs1") + self.hdfs_api = HDFSApi(user="root", host=self.hdfs_host) def wait_hdfs_to_start(self, timeout=60): @@ -665,10 +687,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: self.hdfs_api.write_data("/somefilewithrandomname222", "1") - print("Connected to HDFS and SafeMode disabled! ") + logging.debug("Connected to HDFS and SafeMode disabled! ") return except Exception as ex: - print("Can't connect to HDFS " + str(ex)) + logging.debug("Can't connect to HDFS " + str(ex)) time.sleep(1) raise Exception("Can't wait HDFS to start") @@ -681,10 +703,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: connection.list_database_names() - print("Connected to Mongo dbs:", connection.database_names()) + logging.debug("Connected to Mongo dbs:", connection.database_names()) return except Exception as ex: - print("Can't connect to Mongo " + str(ex)) + logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): @@ -697,7 +719,7 @@ class ClickHouseCluster: try: minio_client.list_buckets() - print("Connected to Minio.") + logging.debug("Connected to Minio.") buckets = [self.minio_bucket, self.minio_bucket_2] @@ -705,12 +727,12 @@ class ClickHouseCluster: if minio_client.bucket_exists(bucket): minio_client.remove_bucket(bucket) minio_client.make_bucket(bucket) - print("S3 bucket '%s' created", bucket) + logging.debug("S3 bucket '%s' created", bucket) self.minio_client = minio_client return except Exception as ex: - print("Can't connect to Minio: %s", str(ex)) + logging.debug("Can't connect to Minio: %s", str(ex)) time.sleep(1) raise Exception("Can't wait Minio to start") @@ -721,10 +743,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) - print("Connected to SchemaRegistry") + logging.debug("Connected to SchemaRegistry") return sr_client except Exception as ex: - print(("Can't connect to SchemaRegistry: %s", str(ex))) + logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) def wait_cassandra_to_start(self, timeout=30): @@ -740,27 +762,27 @@ class ClickHouseCluster: time.sleep(1) def start(self, destroy_dirs=True): - print("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) + logging.debug("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) if self.is_up: return # Just in case kill unstopped containers from previous launch try: - print("Trying to kill unstopped containers...") + logging.debug("Trying to kill unstopped containers...") if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) - print("Unstopped containers killed") + logging.debug("Unstopped containers killed") except: pass try: if destroy_dirs and p.exists(self.instances_dir): - print(("Removing instances dir %s", self.instances_dir)) + logging.debug(("Removing instances dir %s", self.instances_dir)) shutil.rmtree(self.instances_dir) for instance in list(self.instances.values()): - print(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) + logging.debug(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) instance.create_dir(destroy_dir=destroy_dirs) self.docker_client = docker.from_env(version=self.docker_api_version) @@ -768,7 +790,7 @@ class ClickHouseCluster: common_opts = ['up', '-d', '--force-recreate'] if self.with_zookeeper and self.base_zookeeper_cmd: - print('Setup ZooKeeper') + logging.debug('Setup ZooKeeper') env = os.environ.copy() if not self.zookeeper_use_tmpfs: env['ZK_FS'] = 'bind' @@ -787,51 +809,53 @@ class ClickHouseCluster: self.wait_zookeeper_to_start(120) if self.with_mysql and self.base_mysql_cmd: - print('Setup MySQL') + logging.debug('Setup MySQL') subprocess_check_call(self.base_mysql_cmd + common_opts) self.wait_mysql_to_start(120) if self.with_mysql8 and self.base_mysql8_cmd: - print('Setup MySQL 8') + logging.debug('Setup MySQL 8') subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start(120) if self.with_postgres and self.base_postgres_cmd: - print('Setup Postgres') + logging.debug('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(30) if self.with_kafka and self.base_kafka_cmd: - print('Setup Kafka') + logging.debug('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: - print('Setup kerberized kafka') + logging.debug('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') if self.with_hdfs and self.base_hdfs_cmd: - print('Setup HDFS') + logging.debug('Setup HDFS') + os.makedirs(self.hdfs_logs_dir) subprocess_check_call(self.base_hdfs_cmd + common_opts) self.make_hdfs_api() - self.wait_hdfs_to_start(120) + self.wait_hdfs_to_start(50) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: - print('Setup kerberized HDFS') + logging.debug('Setup kerberized HDFS') run_and_check(self.base_kerberized_hdfs_cmd + common_opts) self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(timeout=300) if self.with_mongo and self.base_mongo_cmd: - print('Setup Mongo') + logging.debug('Setup Mongo') run_and_check(self.base_mongo_cmd + common_opts) self.wait_mongo_to_start(30) if self.with_redis and self.base_redis_cmd: - print('Setup Redis') + logging.debug('Setup Redis') subprocess_check_call(self.base_redis_cmd + common_opts) time.sleep(10) @@ -850,28 +874,28 @@ class ClickHouseCluster: _create_env_file(os.path.join(self.env_file), self.env_variables) clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] - print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) + logging.debug(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) self.up_called = True subprocess.check_output(clickhouse_start_cmd) - print("ClickHouse instance created") + logging.debug("ClickHouse instance created") start_deadline = time.time() + 20.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) - print("Waiting for ClickHouse start...") + logging.debug("Waiting for ClickHouse start...") instance.wait_for_start(start_deadline) - print("ClickHouse started") + logging.debug("ClickHouse started") instance.client = Client(instance.ip_address, command=self.client_bin_path) self.is_up = True except BaseException as e: - print("Failed to start cluster: ") - print(str(e)) - print(traceback.print_exc()) + logging.debug("Failed to start cluster: ") + logging.debug(str(e)) + logging.debug(traceback.print_exc()) raise def shutdown(self, kill=True): @@ -881,7 +905,7 @@ class ClickHouseCluster: try: subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL except Exception as e: - print("Unable to get logs from docker.") + logging.debug("Unable to get logs from docker.") f.seek(0) for line in f: if SANITIZER_SIGN in line: @@ -892,14 +916,14 @@ class ClickHouseCluster: try: subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) except Exception as e: - print("Kill command failed during shutdown. {}".format(repr(e))) - print("Trying to kill forcefully") + logging.debug("Kill command failed during shutdown. {}".format(repr(e))) + logging.debug("Trying to kill forcefully") subprocess_check_call(self.base_cmd + ['kill']) try: subprocess_check_call(self.base_cmd + ['down', '--volumes']) except Exception as e: - print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) + logging.debug("Down + remove orphans failed durung shutdown. {}".format(repr(e))) self.is_up = False @@ -947,7 +971,7 @@ class ClickHouseCluster: kazoo_callback(self.get_kazoo_client(zoo_instance_name)) return except KazooException as e: - print(repr(e)) + logging.debug(repr(e)) time.sleep(sleep_for) kazoo_callback(self.get_kazoo_client(zoo_instance_name)) @@ -1112,7 +1136,7 @@ class ClickHouseInstance: return result time.sleep(sleep_time) except Exception as ex: - print("Retry {} got exception {}".format(i + 1, ex)) + logging.debug("Retry {} got exception {}".format(i + 1, ex)) time.sleep(sleep_time) if result is not None: @@ -1372,16 +1396,16 @@ class ClickHouseInstance: instance_config_dir = p.abspath(p.join(self.path, 'configs')) os.makedirs(instance_config_dir) - print("Copy common default production configuration from {}".format(self.base_config_dir)) + logging.debug("Copy common default production configuration from {}".format(self.base_config_dir)) shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml')) shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml')) - print("Create directory for configuration generated in this helper") + logging.debug("Create directory for configuration generated in this helper") # used by all utils with any config conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d')) os.mkdir(conf_d_dir) - print("Create directory for common tests configuration") + logging.debug("Create directory for common tests configuration") # used by server with main config.xml self.config_d_dir = p.abspath(p.join(instance_config_dir, 'config.d')) os.mkdir(self.config_d_dir) @@ -1390,14 +1414,14 @@ class ClickHouseInstance: dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) os.mkdir(dictionaries_dir) - print("Copy common configuration from helpers") + logging.debug("Copy common configuration from helpers") # The file is named with 0_ prefix to be processed before other configuration overloads. shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir) if len(self.custom_dictionaries_paths): shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir) - print("Generate and write macros file") + logging.debug("Generate and write macros file") macros = self.macros.copy() macros['instance'] = self.name with open(p.join(conf_d_dir, 'macros.xml'), 'w') as macros_config: @@ -1411,7 +1435,7 @@ class ClickHouseInstance: shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config.d configs - print("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) + logging.debug("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) for path in self.custom_main_config_paths: shutil.copy(path, self.config_d_dir) @@ -1424,16 +1448,16 @@ class ClickHouseInstance: shutil.copy(path, dictionaries_dir) db_dir = p.abspath(p.join(self.path, 'database')) - print("Setup database dir {}".format(db_dir)) + logging.debug("Setup database dir {}".format(db_dir)) if self.clickhouse_path_dir is not None: - print("Database files taken from {}".format(self.clickhouse_path_dir)) + logging.debug("Database files taken from {}".format(self.clickhouse_path_dir)) shutil.copytree(self.clickhouse_path_dir, db_dir) - print("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) + logging.debug("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) else: os.mkdir(db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) - print("Setup logs dir {}".format(logs_dir)) + logging.debug("Setup logs dir {}".format(logs_dir)) os.mkdir(logs_dir) depends_on = [] @@ -1478,7 +1502,7 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND - print("Entrypoint cmd: {}".format(entrypoint_cmd)) + logging.debug("Entrypoint cmd: {}".format(entrypoint_cmd)) networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name: diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index a7ca8c57da8..7d4a3ad1c29 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -4,3 +4,10 @@ norecursedirs = _instances timeout = 300 junit_duration_report = call junit_suite_name = integration +log_cli = 1 +log_cli_level = CRITICAL +log_cli_format = %(message)s +log_file = pytest.log +log_file_level = DEBUG +log_file_format = %(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) +log_file_date_format=%Y-%m-%d %H:%M:%S diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index a6c8b7e1ee9..ac3807908a8 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -4,7 +4,6 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.hdfs_api import HDFSApi -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_hdfs=True, main_configs=['configs/log_conf.xml']) @@ -13,12 +12,7 @@ node1 = cluster.add_instance('node1', with_hdfs=True, main_configs=['configs/log def started_cluster(): try: cluster.start() - yield cluster - - except Exception as ex: - print(ex) - raise ex finally: cluster.shutdown() diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 24afbc8ca4e..da5826665e5 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -638,7 +638,6 @@ def kafka_cluster(): try: cluster.start() yield cluster - finally: cluster.shutdown() diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index accb855af10..e21d18c2f49 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -37,7 +37,7 @@ def check_kafka_is_available(): kafka_id, '/usr/bin/kafka-broker-api-versions', '--bootstrap-server', - 'localhost:9093'), + 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), stdout=subprocess.PIPE) p.communicate() return p.returncode == 0 @@ -58,14 +58,26 @@ def wait_kafka_is_available(max_retries=50): def producer_serializer(x): return x.encode() if isinstance(x, str) else x + +def get_kafka_producer(port, serializer): + errors = [] + for _ in range(15): + try: + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) + logging.debug("Kafka Connection establised: localhost:{}".format(port)) + return producer + except Exception as e: + errors += [str(e)] + time.sleep(1) -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9093", value_serializer=producer_serializer) + raise Exception("Connection not establised, {}".format(errors)) + +def kafka_produce(kafka_cluster, topic, messages, timestamp=None): + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() - print ("Produced {} messages for topic {}".format(len(messages), topic)) - # Fixtures @@ -74,9 +86,7 @@ def kafka_produce(topic, messages, timestamp=None): def kafka_cluster(): try: cluster.start() - print("kafka_id is {}".format(kafka_id)) yield cluster - finally: cluster.shutdown() @@ -92,7 +102,7 @@ def kafka_setup_teardown(): @pytest.mark.timeout(180) # wait to build containers def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + kafka_produce(kafka_cluster, 'kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) instance.query(''' CREATE TABLE test.kafka (field String) @@ -116,7 +126,7 @@ def test_kafka_json_as_string(kafka_cluster): assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") def test_kafka_json_as_string_no_kdc(kafka_cluster): - kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + kafka_produce(kafka_cluster, 'kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) kafka_cluster.pause_container('kafka_kerberos') time.sleep(45) # wait for ticket expiration diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index f75a9aac237..75af909faec 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -13,20 +13,18 @@ node = cluster.add_instance('node', with_mongo=True) def started_cluster(): try: cluster.start() - yield cluster - finally: cluster.shutdown() -def get_mongo_connection(): - connection_str = 'mongodb://root:clickhouse@localhost:27018' +def get_mongo_connection(started_cluster): + connection_str = 'mongodb://root:clickhouse@localhost:{}'.format(started_cluster.mongo_port) return pymongo.MongoClient(connection_str) def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') simple_mongo_table = db['simple_table'] @@ -45,7 +43,7 @@ def test_simple_select(started_cluster): def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') incomplete_mongo_table = db['complex_table'] @@ -64,7 +62,7 @@ def test_complex_data_type(started_cluster): def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') strange_mongo_table = db['strange_table'] From ee955038c1139da47f80326459e26f47c7c426c2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 15:58:11 +0300 Subject: [PATCH 028/652] wip --- .../runner/compose/docker_compose_minio.yml | 2 +- tests/integration/helpers/cluster.py | 97 ++++++++++--------- tests/integration/helpers/hdfs_api.py | 97 ++++++------------- .../test_allowed_url_from_config/test.py | 6 +- .../test_redirect_url_storage/test.py | 18 ++-- tests/integration/test_storage_hdfs/test.py | 63 +++++++----- .../test_storage_kerberized_hdfs/test.py | 22 +++-- 7 files changed, 158 insertions(+), 147 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_minio.yml b/docker/test/integration/runner/compose/docker_compose_minio.yml index dbb29f9711a..73016075921 100644 --- a/docker/test/integration/runner/compose/docker_compose_minio.yml +++ b/docker/test/integration/runner/compose/docker_compose_minio.yml @@ -7,7 +7,7 @@ services: - data1-1:/data1 - ${MINIO_CERTS_DIR:-}:/certs ports: - - "9001:9001" + - ${MINIO_EXTERNAL_PORT}:${MINIO_INTERNAL_PORT} environment: MINIO_ACCESS_KEY: minio MINIO_SECRET_KEY: minio123 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index cb5c25c260e..302901159e1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -36,7 +36,7 @@ DEFAULT_ENV_NAME = '.env' SANITIZER_SIGN = "==================" - +# to create docker-compose env file def _create_env_file(path, variables): logging.debug("Env {} stored in {}".format(variables, path)) with open(path, 'w') as f: @@ -44,13 +44,6 @@ def _create_env_file(path, variables): f.write("=".join([var, value]) + "\n") return path -def env_to_compose_args(env): - args = [] - for key, value in env.items(): - args += ["-e", "{}={}".format(key, value)] - return args - - def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) if res.returncode != 0: @@ -174,7 +167,8 @@ class ClickHouseCluster: self.with_cassandra = False self.with_minio = False - self.minio_certs_dir = None + self.minio_dir = os.path.join(self.instances_dir, "minio") + self.minio_certs_dir = None # source for certificates self.minio_host = "minio1" self.minio_bucket = "root" self.minio_bucket_2 = "root2" @@ -347,6 +341,19 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] return self.base_mongo_cmd + def setup_minio_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_minio = True + cert_d = p.join(self.minio_dir, "certs") + env_variables['MINIO_CERTS_DIR'] = cert_d + env_variables['MINIO_EXTERNAL_PORT'] = self.minio_port + env_variables['MINIO_INTERNAL_PORT'] = "9001" + env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, cert_d, 'public.crt') + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) + self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] + return self.base_minio_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -484,21 +491,13 @@ class ClickHouseCluster: cmds.append(self.setup_redis_cmd(instance, env_variables, docker_compose_yml_dir)) if with_minio and not self.with_minio: - self.with_minio = True - self.minio_certs_dir = minio_certs_dir - if self.minio_certs_dir: - env_variables['MINIO_CERTS_DIR'] = p.join(self.base_dir, self.minio_certs_dir) - # Minio client (urllib3) uses SSL_CERT_FILE for certificate validation. - env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') + cmds.append(self.setup_minio_cmd(instance, env_variables, docker_compose_yml_dir)) + + if minio_certs_dir is not None: + if self.minio_certs_dir is None: + self.minio_certs_dir = minio_certs_dir else: - # Attach empty certificates directory to ensure non-secure mode. - minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') - os.makedirs(minio_certs_dir, exist_ok=True) - env_variables['MINIO_CERTS_DIR'] = minio_certs_dir - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) - self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] - cmds.append(self.base_minio_cmd) + raise Exception("Overwriting minio certs dir") if with_cassandra and not self.with_cassandra: self.with_cassandra = True @@ -659,6 +658,7 @@ class ClickHouseCluster: raise Exception("Cannot wait ZooKeeper container") def make_hdfs_api(self, timeout=60, kerberized=False): + hdfs_api = None if kerberized: keytab = p.abspath(p.join(self.instances['node1'].path, "secrets/clickhouse.keytab")) krb_conf = p.abspath(p.join(self.instances['node1'].path, "secrets/krb_long.conf")) @@ -666,27 +666,29 @@ class ClickHouseCluster: # logging.debug("kerberizedhdfs1 ip ", hdfs_ip) kdc_ip = self.get_instance_ip('hdfskerberos') # logging.debug("kdc_ip ", kdc_ip) - self.hdfs_api = HDFSApi(user="root", - timeout=timeout, - kerberized=True, - principal="root@TEST.CLICKHOUSE.TECH", - keytab=keytab, - krb_conf=krb_conf, - host="kerberizedhdfs1", - protocol="http", - proxy_port=50070, - data_port=1006, - hdfs_ip=hdfs_ip, - kdc_ip=kdc_ip) + hdfs_api = HDFSApi(user="root", + timeout=timeout, + kerberized=True, + principal="root@TEST.CLICKHOUSE.TECH", + keytab=keytab, + krb_conf=krb_conf, + host="kerberizedhdfs1", + protocol="http", + proxy_port=50070, + data_port=1006, + hdfs_ip=hdfs_ip, + kdc_ip=kdc_ip) + else: - self.hdfs_api = HDFSApi(user="root", host=self.hdfs_host) + logging.debug("Create HDFSApi host={}".format("localhost")) + hdfs_api = HDFSApi(user="root", host="localhost", data_port=self.hdfs_data_port, proxy_port=self.hdfs_name_port) + return hdfs_api - - def wait_hdfs_to_start(self, timeout=60): + def wait_hdfs_to_start(self, hdfs_api, timeout=60): start = time.time() while time.time() - start < timeout: try: - self.hdfs_api.write_data("/somefilewithrandomname222", "1") + hdfs_api.write_data("/somefilewithrandomname222", "1") logging.debug("Connected to HDFS and SafeMode disabled! ") return except Exception as ex: @@ -710,7 +712,7 @@ class ClickHouseCluster: time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): - minio_client = Minio('localhost:9001', + minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', secret_key='minio123', secure=secure) @@ -840,14 +842,14 @@ class ClickHouseCluster: logging.debug('Setup HDFS') os.makedirs(self.hdfs_logs_dir) subprocess_check_call(self.base_hdfs_cmd + common_opts) - self.make_hdfs_api() - self.wait_hdfs_to_start(50) + hdfs_api = self.make_hdfs_api() + self.wait_hdfs_to_start(hdfs_api, 120) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') run_and_check(self.base_kerberized_hdfs_cmd + common_opts) - self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(timeout=300) + hdfs_api = self.make_hdfs_api(kerberized=True) + self.wait_hdfs_to_start(hdfs_api, timeout=300) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') @@ -860,6 +862,13 @@ class ClickHouseCluster: time.sleep(10) if self.with_minio and self.base_minio_cmd: + # Copy minio certificates to minio/certs + os.mkdir(self.minio_dir) + if self.minio_certs_dir is None: + os.mkdir(os.path.join(self.minio_dir, 'certs')) + else: + shutil.copytree(self.minio_certs_dir, os.path.join(self.minio_dir, 'certs')) + minio_start_cmd = self.base_minio_cmd + common_opts logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd))) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index cb742662855..903a670aa57 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -11,15 +11,6 @@ import tempfile import logging import os -g_dns_hook = None - -def custom_getaddrinfo(*args): - # print("from custom_getaddrinfo g_dns_hook is None ", g_dns_hook is None) - ret = g_dns_hook.custom_getaddrinfo(*args) - # print("g_dns_hook.custom_getaddrinfo result", ret) - return ret - - class mk_krb_conf(object): def __init__(self, krb_conf, kdc_ip): self.krb_conf = krb_conf @@ -37,32 +28,6 @@ class mk_krb_conf(object): if self.amended_krb_conf is not None: self.amended_krb_conf.close() -# tweak dns resolution to connect to localhost where api_host is in URL -class dns_hook(object): - def __init__(self, hdfs_api): - # print("dns_hook.init ", hdfs_api.kerberized, hdfs_api.host, hdfs_api.data_port, hdfs_api.proxy_port) - self.hdfs_api = hdfs_api - def __enter__(self): - global g_dns_hook - g_dns_hook = self - # print("g_dns_hook is None ", g_dns_hook is None) - self.original_getaddrinfo = socket.getaddrinfo - socket.getaddrinfo = custom_getaddrinfo - return self - def __exit__(self, type, value, traceback): - global g_dns_hook - g_dns_hook = None - socket.getaddrinfo = self.original_getaddrinfo - def custom_getaddrinfo(self, *args): - (hostname, port) = args[:2] - # print("top of custom_getaddrinfo", hostname, port) - - if hostname == self.hdfs_api.host and (port == self.hdfs_api.data_port or port == self.hdfs_api.proxy_port): - # print("dns_hook substitute") - return [(socket.AF_INET, 1, 6, '', ("127.0.0.1", port))] - else: - return self.original_getaddrinfo(*args) - class HDFSApi(object): def __init__(self, user, timeout=100, kerberized=False, principal=None, keytab=None, krb_conf=None, @@ -83,9 +48,9 @@ class HDFSApi(object): # logging.basicConfig(level=logging.DEBUG) # logging.getLogger().setLevel(logging.DEBUG) - # requests_log = logging.getLogger("requests.packages.urllib3") - # requests_log.setLevel(logging.DEBUG) - # requests_log.propagate = True + requests_log = logging.getLogger("requests.packages.urllib3") + requests_log.setLevel(logging.DEBUG) + requests_log.propagate = True if kerberized: self._run_kinit() @@ -101,13 +66,13 @@ class HDFSApi(object): raise Exception("kerberos principal and keytab are required") with mk_krb_conf(self.krb_conf, self.kdc_ip) as instantiated_krb_conf: - # print("instantiated_krb_conf ", instantiated_krb_conf) + logging.debug("instantiated_krb_conf ", instantiated_krb_conf) os.environ["KRB5_CONFIG"] = instantiated_krb_conf cmd = "(kinit -R -t {keytab} -k {principal} || (sleep 5 && kinit -R -t {keytab} -k {principal})) ; klist".format(instantiated_krb_conf=instantiated_krb_conf, keytab=self.keytab, principal=self.principal) - # print(cmd) + logging.debug(cmd) start = time.time() @@ -123,17 +88,15 @@ class HDFSApi(object): raise Exception("Kinit running failure") def read_data(self, path, universal_newlines=True): - with dns_hook(self): - response = requests.get("{protocol}://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(protocol=self.protocol, host=self.host, port=self.proxy_port, path=path), headers={'host': 'localhost'}, allow_redirects=False, verify=False, auth=self.kerberos_auth) + logging.debug("read_data protocol:{} host:{} port:{} path: {}".format(self.protocol, self.host, self.proxy_port, path)) + response = requests.get("{protocol}://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(protocol=self.protocol, host=self.host, port=self.proxy_port, path=path), headers={'host': 'localhost'}, allow_redirects=False, verify=False, auth=self.kerberos_auth) if response.status_code != 307: response.raise_for_status() # additional_params = '&'.join(response.headers['Location'].split('&')[1:2]) - url = "{location}".format(location=response.headers['Location']) - # print("redirected to ", url) - with dns_hook(self): - response_data = requests.get(url, - headers={'host': 'localhost'}, - verify=False, auth=self.kerberos_auth) + url = "{location}".format(location=response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port))) + logging.debug("redirected to {}".format(url)) + response_data = requests.get(url, headers={'host': 'localhost'}, + verify=False, auth=self.kerberos_auth) if response_data.status_code != 200: response_data.raise_for_status() if universal_newlines: @@ -142,6 +105,7 @@ class HDFSApi(object): return response_data.content def write_data(self, path, content): + logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}".format(self.protocol, self.host, self.proxy_port, path, self.user)) named_file = NamedTemporaryFile(mode='wb+') fpath = named_file.name if isinstance(content, str): @@ -149,40 +113,41 @@ class HDFSApi(object): named_file.write(content) named_file.flush() - if self.kerberized: self._run_kinit() self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) - # print(self.kerberos_auth) + logging.debug(self.kerberos_auth) + + response = requests.put( + "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', + port=self.proxy_port, + path=path, user=self.user), + allow_redirects=False, + headers={'host': 'localhost'}, + params={'overwrite' : 'true'}, + verify=False, auth=self.kerberos_auth + ) + + logging.debug("HDFS api response:{}".format(response.headers)) - with dns_hook(self): - response = requests.put( - "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host=self.host, - port=self.proxy_port, - path=path, user=self.user), - allow_redirects=False, - headers={'host': 'localhost'}, - params={'overwrite' : 'true'}, - verify=False, auth=self.kerberos_auth - ) if response.status_code != 307: - # print(response.headers) response.raise_for_status() - additional_params = '&'.join( - response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) + # additional_params = '&'.join( + # response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) - with dns_hook(self), open(fpath, mode="rb") as fh: + with open(fpath, mode="rb") as fh: file_data = fh.read() protocol = "http" # self.protocol response = requests.put( - "{location}".format(location=response.headers['Location']), + "{location}".format(location=location), data=file_data, headers={'content-type':'text/plain', 'host': 'localhost'}, params={'file': path, 'user.name' : self.user}, allow_redirects=False, verify=False, auth=self.kerberos_auth ) - # print(response) + logging.debug(response) if response.status_code != 201: response.raise_for_status() diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 6442937c8f4..29844301dba 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -100,8 +100,10 @@ def test_table_function_remote(start_cluster): def test_redirect(start_cluster): - start_cluster.hdfs_api.write_data("/simple_storage", "1\t\n") - assert start_cluster.hdfs_api.read_data("/simple_storage") == "1\t\n" + hdfs_api = start_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\t\n") + assert hdfs_api.read_data("/simple_storage") == "1\t\n" node7.query( "CREATE TABLE table_test_7_1 (word String) ENGINE=URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', CSV)") assert "not allowed" in node7.query_and_get_error("SET max_http_get_redirects=1; SELECT * from table_test_7_1") diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index f2731794d43..736fb5c409c 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -17,8 +17,10 @@ def started_cluster(): def test_url_without_redirect(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access datanode port directly node1.query( @@ -27,8 +29,10 @@ def test_url_without_redirect(started_cluster): def test_url_with_redirect_not_allowed(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access proxy port without allowing redirects node1.query( @@ -38,8 +42,10 @@ def test_url_with_redirect_not_allowed(started_cluster): def test_url_with_redirect_allowed(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access proxy port with allowing redirects # http://localhost:50070/webhdfs/v1/b?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0 diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ac3807908a8..3fb12b233fa 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -18,14 +18,16 @@ def started_cluster(): def test_read_write_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/simple_storage', 'TSV')") node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" def test_read_write_storage_with_globs(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table HDFSStorageWithRange (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1..5}', 'TSV')") node1.query( @@ -36,8 +38,8 @@ def test_read_write_storage_with_globs(started_cluster): "create table HDFSStorageWithAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage*', 'TSV')") for i in ["1", "2", "3"]: - started_cluster.hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" + hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") + assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" assert node1.query("select count(*) from HDFSStorageWithRange") == "3\n" assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" @@ -67,23 +69,26 @@ def test_read_write_storage_with_globs(started_cluster): def test_read_write_table(started_cluster): - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_data("/simple_table_function") == data + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", data) + + assert hdfs_api.read_data("/simple_table_function") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data def test_write_table(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table OtherHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/other_storage', 'TSV')") node1.query("insert into OtherHDFSStorage values (10, 'tomas', 55.55), (11, 'jack', 32.54)") result = "10\ttomas\t55.55\n11\tjack\t32.54\n" - assert started_cluster.hdfs_api.read_data("/other_storage") == result + assert hdfs_api.read_data("/other_storage") == result assert node1.query("select * from OtherHDFSStorage order by id") == result @@ -110,12 +115,14 @@ def test_bad_hdfs_uri(started_cluster): @pytest.mark.timeout(800) def test_globs_in_read_table(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + some_data = "1\tSerialize\t555.222\n2\tData\t777.333\n" globs_dir = "/dir_for_test_with_globs/" files = ["dir1/dir_dir/file1", "dir2/file2", "simple_table_function", "dir/file", "some_dir/dir1/file", "some_dir/dir2/file", "some_dir/file", "table1_function", "table2_function", "table3_function"] for filename in files: - started_cluster.hdfs_api.write_data(globs_dir + filename, some_data) + hdfs_api.write_data(globs_dir + filename, some_data) test_requests = [("dir{1..5}/dir_dir/file1", 1, 1), ("*_table_functio?", 1, 1), @@ -141,58 +148,70 @@ def test_globs_in_read_table(started_cluster): def test_read_write_gzip_table(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data def test_read_write_gzip_table_with_parameter_gzip(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function", data) + + assert hdfs_api.read_gzip_data("/simple_table_function") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data def test_read_write_table_with_parameter_none(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_data("/simple_table_function.gz", data) + + assert hdfs_api.read_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data def test_write_gz_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query( "create table GZHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage.gz', 'TSV')") node1.query("insert into GZHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" + assert hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" def test_write_gzip_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query( "create table GZIPHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/gzip_storage', 'TSV', 'gzip')") node1.query("insert into GZIPHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" if __name__ == '__main__': diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index 1fffd7a8c12..bb4440a3e38 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -23,10 +23,12 @@ def started_cluster(): cluster.shutdown() def test_read_table(started_cluster): - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - api_read = started_cluster.hdfs_api.read_data("/simple_table_function") + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", data) + + api_read = hdfs_api.read_data("/simple_table_function") assert api_read == data select_read = node1.query("select * from hdfs('hdfs://kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") @@ -34,10 +36,12 @@ def test_read_table(started_cluster): def test_read_write_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query("create table SimpleHDFSStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage1', 'TSV')") node1.query("insert into SimpleHDFSStorage2 values (1, 'Mark', 72.53)") - api_read = started_cluster.hdfs_api.read_data("/simple_storage1") + api_read = hdfs_api.read_data("/simple_storage1") assert api_read == "1\tMark\t72.53\n" select_read = node1.query("select * from SimpleHDFSStorage2") @@ -45,12 +49,14 @@ def test_read_write_storage(started_cluster): def test_write_storage_not_expired(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query("create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')") time.sleep(45) # wait for ticket expiration node1.query("insert into SimpleHDFSStorageNotExpired values (1, 'Mark', 72.53)") - api_read = started_cluster.hdfs_api.read_data("/simple_storage_not_expired") + api_read = hdfs_api.read_data("/simple_storage_not_expired") assert api_read == "1\tMark\t72.53\n" select_read = node1.query("select * from SimpleHDFSStorageNotExpired") @@ -58,6 +64,8 @@ def test_write_storage_not_expired(started_cluster): def test_two_users(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query("create table HDFSStorOne (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/storage_user_one', 'TSV')") node1.query("insert into HDFSStorOne values (1, 'Real', 86.00)") @@ -69,8 +77,10 @@ def test_two_users(started_cluster): select_read_2 = node1.query("select * from hdfs('hdfs://suser@kerberizedhdfs1:9010/storage_user_one', 'TSV', 'id UInt64, text String, number Float64')") def test_read_table_expired(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function_relogin", data) + hdfs_api.write_data("/simple_table_function_relogin", data) started_cluster.pause_container('hdfskerberos') time.sleep(45) From d45cab4228da23261c725479430cfa5bea66bf21 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 17:42:43 +0300 Subject: [PATCH 029/652] wip --- .../docker_compose_kerberized_hdfs.yml | 15 ++++--- tests/integration/helpers/cluster.py | 40 ++++++++++++++----- tests/integration/helpers/hdfs_api.py | 39 ++++++++++++------ tests/integration/runner | 14 ++++++- 4 files changed, 77 insertions(+), 31 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml index f2a659bce58..34fa902bf93 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml @@ -8,13 +8,16 @@ services: hostname: kerberizedhdfs1 restart: always volumes: - - ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro - - ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf - - ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro + - ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro + - ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf + - ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro + - type: ${KERBERIZED_HDFS_FS:-tmpfs} + source: ${KERBERIZED_HDFS_LOGS:-} + target: /var/log/hadoop-hdfs ports: - - 1006:1006 - - 50070:50070 - - 9010:9010 + - ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070 + - ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006 + # - 9010:9010 depends_on: - hdfskerberos entrypoint: /etc/bootstrap.sh -d diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 302901159e1..bc9850d5176 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -184,6 +184,13 @@ class ClickHouseCluster: self.hdfs_dir = p.abspath(p.join(self.instances_dir, "hdfs")) self.hdfs_logs_dir = os.path.join(self.hdfs_dir, "logs") + # available when with_kerberized_hdfs == True + self.hdfs_kerberized_host = "kerberizedhdfs1" + self.hdfs_kerberized_name_port = get_open_port() + self.hdfs_kerberized_data_port = get_open_port() + self.hdfs_kerberized_dir = p.abspath(p.join(self.instances_dir, "kerberized_hdfs")) + self.hdfs_kerberized_logs_dir = os.path.join(self.hdfs_kerberized_dir, "logs") + # available when with_kafka == True self.kafka_host = "kafka1" self.kafka_port = get_open_port() @@ -276,6 +283,21 @@ class ClickHouseCluster: print("HDFS BASE CMD:{}".format(self.base_hdfs_cmd)) return self.base_hdfs_cmd + def setup_kerberized_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_kerberized_hdfs = True + env_variables['KERBERIZED_HDFS_HOST'] = self.hdfs_kerberized_host + env_variables['KERBERIZED_HDFS_NAME_EXTERNAL_PORT'] = str(self.hdfs_kerberized_name_port) + env_variables['KERBERIZED_HDFS_NAME_INTERNAL_PORT'] = "50070" + env_variables['KERBERIZED_HDFS_DATA_EXTERNAL_PORT'] = str(self.hdfs_kerberized_data_port) + env_variables['KERBERIZED_HDFS_DATA_INTERNAL_PORT'] = "1006" + env_variables['KERBERIZED_HDFS_LOGS'] = self.hdfs_kerberized_logs_dir + env_variables['KERBERIZED_HDFS_FS'] = "bind" + env_variables['KERBERIZED_HDFS_DIR'] = instance.path + '/' + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')]) + self.base_kerberized_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')] + return self.base_kerberized_hdfs_cmd + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True env_variables['KAFKA_HOST'] = self.kafka_host @@ -345,7 +367,7 @@ class ClickHouseCluster: self.with_minio = True cert_d = p.join(self.minio_dir, "certs") env_variables['MINIO_CERTS_DIR'] = cert_d - env_variables['MINIO_EXTERNAL_PORT'] = self.minio_port + env_variables['MINIO_EXTERNAL_PORT'] = str(self.minio_port) env_variables['MINIO_INTERNAL_PORT'] = "9001" env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, cert_d, 'public.crt') @@ -473,12 +495,7 @@ class ClickHouseCluster: cmds.append(self.setup_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kerberized_hdfs and not self.with_kerberized_hdfs: - self.with_kerberized_hdfs = True - env_variables['KERBERIZED_HDFS_DIR'] = instance.path + '/' - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')]) - self.base_kerberized_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')] - cmds.append(self.base_kerberized_hdfs_cmd) + cmds.append(self.setup_kerberized_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) if with_mongo and not self.with_mongo: cmds.append(self.setup_mongo_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -672,10 +689,10 @@ class ClickHouseCluster: principal="root@TEST.CLICKHOUSE.TECH", keytab=keytab, krb_conf=krb_conf, - host="kerberizedhdfs1", + host="localhost", protocol="http", - proxy_port=50070, - data_port=1006, + proxy_port=self.hdfs_kerberized_name_port, + data_port=self.hdfs_kerberized_data_port, hdfs_ip=hdfs_ip, kdc_ip=kdc_ip) @@ -847,9 +864,10 @@ class ClickHouseCluster: if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') + os.makedirs(self.hdfs_kerberized_logs_dir) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(hdfs_api, timeout=300) + self.wait_hdfs_to_start(hdfs_api, timeout=30) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 903a670aa57..60b8979fce2 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -10,7 +10,6 @@ import socket import tempfile import logging import os - class mk_krb_conf(object): def __init__(self, krb_conf, kdc_ip): self.krb_conf = krb_conf @@ -51,6 +50,9 @@ class HDFSApi(object): requests_log = logging.getLogger("requests.packages.urllib3") requests_log.setLevel(logging.DEBUG) requests_log.propagate = True + kerb_log = logging.getLogger("requests_kerberos") + kerb_log.setLevel(logging.DEBUG) + kerb_log.propagate = True if kerberized: self._run_kinit() @@ -66,23 +68,28 @@ class HDFSApi(object): raise Exception("kerberos principal and keytab are required") with mk_krb_conf(self.krb_conf, self.kdc_ip) as instantiated_krb_conf: - logging.debug("instantiated_krb_conf ", instantiated_krb_conf) + logging.debug("instantiated_krb_conf {}".format(instantiated_krb_conf)) os.environ["KRB5_CONFIG"] = instantiated_krb_conf cmd = "(kinit -R -t {keytab} -k {principal} || (sleep 5 && kinit -R -t {keytab} -k {principal})) ; klist".format(instantiated_krb_conf=instantiated_krb_conf, keytab=self.keytab, principal=self.principal) - logging.debug(cmd) - start = time.time() while time.time() - start < self.timeout: try: - subprocess.call(cmd, shell=True) - print("KDC started, kinit successfully run") + res = subprocess.run(cmd, shell=True) + if res.returncode != 0: + # check_call(...) from subprocess does not print stderr, so we do it manually + logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) + logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) + logging.debug('Env:\n{}\n'.format(env)) + raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) + + logging.debug("KDC started, kinit successfully run") return except Exception as ex: - print("Can't run kinit ... waiting {}".format(str(ex))) + logging.debug("Can't run kinit ... waiting {}".format(str(ex))) time.sleep(1) raise Exception("Kinit running failure") @@ -93,9 +100,13 @@ class HDFSApi(object): if response.status_code != 307: response.raise_for_status() # additional_params = '&'.join(response.headers['Location'].split('&')[1:2]) - url = "{location}".format(location=response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port))) - logging.debug("redirected to {}".format(url)) - response_data = requests.get(url, headers={'host': 'localhost'}, + location = None + if self.kerberized: + location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port)) + else: + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) + logging.debug("redirected to {}".format(location)) + response_data = requests.get(location, headers={'host': 'localhost'}, verify=False, auth=self.kerberos_auth) if response_data.status_code != 200: response_data.raise_for_status() @@ -116,7 +127,6 @@ class HDFSApi(object): if self.kerberized: self._run_kinit() self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) - logging.debug(self.kerberos_auth) response = requests.put( "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', @@ -135,8 +145,11 @@ class HDFSApi(object): # additional_params = '&'.join( # response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) - location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) - + if self.kerberized: + location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port)) + else: + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) + with open(fpath, mode="rb") as fh: file_data = fh.read() protocol = "http" # self.protocol diff --git a/tests/integration/runner b/tests/integration/runner index 6dca7663310..ead57913217 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -142,12 +142,23 @@ if __name__ == "__main__": action="append", help="Set non-default tags for images used in docker compose recipes(yandex/my_container:my_tag)") + parser.add_argument( + "-n", "--parallel", + action="store", + dest="parallel", + help="Parallelism") + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() check_args_and_update_paths(args) + parallel_args = "" + if args.parallel: + parallel_args += "--dist=loadfile" + parallel_args += "-n {}".format(args.parallel) + net = "" if not args.disable_net_host: net = "--net=host" @@ -188,7 +199,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, @@ -197,6 +208,7 @@ if __name__ == "__main__": cases_dir=args.cases_dir, src_dir=args.src_dir, env_tags=env_tags, + parallel=parallel_args, opts=' '.join(args.pytest_args), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, From ecf928737b8adfeff05329780016c735608e6ce0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 18:08:38 +0300 Subject: [PATCH 030/652] fix --- tests/integration/helpers/cluster.py | 2 +- tests/integration/helpers/hdfs_api.py | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index bc9850d5176..88169425856 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -867,7 +867,7 @@ class ClickHouseCluster: os.makedirs(self.hdfs_kerberized_logs_dir) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(hdfs_api, timeout=30) + self.wait_hdfs_to_start(hdfs_api, timeout=300) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 60b8979fce2..9bd4bf66f92 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -116,7 +116,8 @@ class HDFSApi(object): return response_data.content def write_data(self, path, content): - logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}".format(self.protocol, self.host, self.proxy_port, path, self.user)) + logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}, principal:{}".format( + self.protocol, self.host, self.proxy_port, path, self.user, self.principal)) named_file = NamedTemporaryFile(mode='wb+') fpath = named_file.name if isinstance(content, str): @@ -126,7 +127,9 @@ class HDFSApi(object): if self.kerberized: self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) + self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, + hostname_override="kerberizedhdfs1", + principal=self.principal) response = requests.put( "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', From 34fd15ca407fbc9f2619ed2e9e10f176fed75c9f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 18:10:38 +0300 Subject: [PATCH 031/652] trivial --- tests/integration/test_storage_kerberized_hdfs/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index bb4440a3e38..fa520752d1c 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -23,7 +23,7 @@ def started_cluster(): cluster.shutdown() def test_read_table(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) data = "1\tSerialize\t555.222\n2\tData\t777.333\n" hdfs_api.write_data("/simple_table_function", data) @@ -36,7 +36,7 @@ def test_read_table(started_cluster): def test_read_write_storage(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) node1.query("create table SimpleHDFSStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage1', 'TSV')") node1.query("insert into SimpleHDFSStorage2 values (1, 'Mark', 72.53)") @@ -49,7 +49,7 @@ def test_read_write_storage(started_cluster): def test_write_storage_not_expired(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) node1.query("create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')") @@ -64,7 +64,7 @@ def test_write_storage_not_expired(started_cluster): def test_two_users(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) node1.query("create table HDFSStorOne (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/storage_user_one', 'TSV')") node1.query("insert into HDFSStorOne values (1, 'Real', 86.00)") @@ -77,7 +77,7 @@ def test_two_users(started_cluster): select_read_2 = node1.query("select * from hdfs('hdfs://suser@kerberizedhdfs1:9010/storage_user_one', 'TSV', 'id UInt64, text String, number Float64')") def test_read_table_expired(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) data = "1\tSerialize\t555.222\n2\tData\t777.333\n" hdfs_api.write_data("/simple_table_function_relogin", data) From b5f59446b792e47ca117ffb1947ec962847c09c2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 18:39:27 +0300 Subject: [PATCH 032/652] fix kerberos --- tests/integration/helpers/hdfs_api.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 9bd4bf66f92..8a69f6baffd 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -56,7 +56,7 @@ class HDFSApi(object): if kerberized: self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) + self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override="kerberizedhdfs1", principal=self.principal) #principal=self.principal, #hostname_override=self.host, principal=self.principal) # , mutual_authentication=reqkerb.REQUIRED, force_preemptive=True) @@ -106,6 +106,7 @@ class HDFSApi(object): else: location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) logging.debug("redirected to {}".format(location)) + response_data = requests.get(location, headers={'host': 'localhost'}, verify=False, auth=self.kerberos_auth) if response_data.status_code != 200: @@ -125,12 +126,6 @@ class HDFSApi(object): named_file.write(content) named_file.flush() - if self.kerberized: - self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, - hostname_override="kerberizedhdfs1", - principal=self.principal) - response = requests.put( "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', port=self.proxy_port, From fc89b4fd3528c6aa467e1c3f643c92c97d199153 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sat, 20 Feb 2021 17:59:39 +0300 Subject: [PATCH 033/652] better --- tests/integration/helpers/cluster.py | 2 +- tests/integration/runner | 2 +- tests/integration/test_storage_s3/test.py | 193 ++++++++---------- .../test_storage_s3/test_redirect.py | 125 ++++++++++++ 4 files changed, 214 insertions(+), 108 deletions(-) create mode 100644 tests/integration/test_storage_s3/test_redirect.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 668838afb5c..5443659eac9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -172,7 +172,7 @@ class ClickHouseCluster: self.minio_host = "minio1" self.minio_bucket = "root" self.minio_bucket_2 = "root2" - self.minio_port = 9001 + self.minio_port = get_open_port() self.minio_client = None # type: Minio self.minio_redirect_host = "proxy1" self.minio_redirect_port = 8080 diff --git a/tests/integration/runner b/tests/integration/runner index ead57913217..99987839f56 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -157,7 +157,7 @@ if __name__ == "__main__": parallel_args = "" if args.parallel: parallel_args += "--dist=loadfile" - parallel_args += "-n {}".format(args.parallel) + parallel_args += " -n {}".format(args.parallel) net = "" if not args.disable_net_host: diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 1f445feb5a0..b9de57366bf 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -14,9 +14,10 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) +MINIO_INTERNAL_PORT = 9001 # Creates S3 bucket for tests and allows anonymous read-write access to it. -def prepare_s3_bucket(cluster): +def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. bucket_read_write_policy = {"Version": "2012-10-17", "Statement": [ @@ -50,26 +51,26 @@ def prepare_s3_bucket(cluster): } ]} - minio_client = cluster.minio_client - minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy(started_cluster.minio_bucket, json.dumps(bucket_read_write_policy)) - cluster.minio_restricted_bucket = "{}-with-auth".format(cluster.minio_bucket) - if minio_client.bucket_exists(cluster.minio_restricted_bucket): - minio_client.remove_bucket(cluster.minio_restricted_bucket) + started_cluster.minio_restricted_bucket = "{}-with-auth".format(started_cluster.minio_bucket) + if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): + minio_client.remove_bucket(started_cluster.minio_restricted_bucket) - minio_client.make_bucket(cluster.minio_restricted_bucket) + minio_client.make_bucket(started_cluster.minio_restricted_bucket) -def put_s3_file_content(cluster, bucket, filename, data): +def put_s3_file_content(started_cluster, bucket, filename, data): buf = io.BytesIO(data) - cluster.minio_client.put_object(bucket, filename, buf, len(data)) + started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) # Returns content of given S3 file as string. -def get_s3_file_content(cluster, bucket, filename, decode=True): +def get_s3_file_content(started_cluster, bucket, filename, decode=True): # type: (ClickHouseCluster, str) -> str - data = cluster.minio_client.get_object(bucket, filename) + data = started_cluster.minio_client.get_object(bucket, filename) data_str = b"" for chunk in data.stream(): data_str += chunk @@ -79,7 +80,7 @@ def get_s3_file_content(cluster, bucket, filename, decode=True): @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("restricted_dummy", main_configs=["configs/config_for_test_remote_host_filter.xml"], @@ -115,17 +116,17 @@ def run_query(instance, query, stdin=None, settings=None): ("'minio','minio123',", True), ("'wrongid','wrongkey',", False) ]) -def test_put(cluster, maybe_auth, positive): +def test_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, filename, maybe_auth, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, maybe_auth, table_format, values) try: run_query(instance, put_query) @@ -134,18 +135,18 @@ def test_put(cluster, maybe_auth, positive): raise else: assert positive - assert values_csv == get_s3_file_content(cluster, bucket, filename) + assert values_csv == get_s3_file_content(started_cluster, bucket, filename) # Test put no data to S3. @pytest.mark.parametrize("auth", [ "'minio','minio123'," ]) -def test_empty_put(cluster, auth): +def test_empty_put(started_cluster, auth): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" create_empty_table_query = """ @@ -158,13 +159,13 @@ def test_empty_put(cluster, auth): filename = "empty_put_test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') select * from empty_table".format( - cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, auth, table_format) run_query(instance, put_query) try: run_query(instance, "select count(*) from s3('http://{}:{}/{}/{}', {}'CSV', '{}')".format( - cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format)) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, auth, table_format)) assert False, "Query should be failed." except helpers.client.QueryRuntimeException as e: @@ -177,15 +178,15 @@ def test_empty_put(cluster, auth): ("'minio','minio123',", True), ("'wrongid','wrongkey',", False) ]) -def test_put_csv(cluster, maybe_auth, positive): +def test_put_csv(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( - cluster.minio_host, cluster.minio_port, bucket, filename, maybe_auth, table_format) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, maybe_auth, table_format) csv_data = "8,9,16\n11,18,13\n22,14,2\n" try: @@ -195,27 +196,27 @@ def test_put_csv(cluster, maybe_auth, positive): raise else: assert positive - assert csv_data == get_s3_file_content(cluster, bucket, filename) + assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # Test put and get with S3 server redirect. -def test_put_get_with_redirect(cluster): +def test_put_get_with_redirect(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" values_csv = "1,1,1\n1,1,1\n11,11,11\n" filename = "test.csv" query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format, values) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) run_query(instance, query) - assert values_csv == get_s3_file_content(cluster, bucket, filename) + assert values_csv == get_s3_file_content(started_cluster, bucket, filename) query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/{}', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format) stdout = run_query(instance, query) assert list(map(str.split, stdout.splitlines())) == [ @@ -226,23 +227,23 @@ def test_put_get_with_redirect(cluster): # Test put with restricted S3 server redirect. -def test_put_with_zero_redirect(cluster): +def test_put_with_zero_redirect(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["s3_max_redirects"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["s3_max_redirects"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" filename = "test.csv" # Should work without redirect query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, filename, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, table_format, values) run_query(instance, query) # Should not work with redirect query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format, values) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) exception_raised = False try: run_query(instance, query) @@ -253,11 +254,11 @@ def test_put_with_zero_redirect(cluster): assert exception_raised -def test_put_get_with_globs(cluster): +def test_put_get_with_globs(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" for i in range(10): @@ -266,11 +267,11 @@ def test_put_get_with_globs(cluster): max_path = max(path, max_path) values = "({},{},{})".format(i, j, i + j) query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, path, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, path, table_format, values) run_query(instance, query) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from s3('http://{}:{}/{}/*_{{a,b,c,d}}/%3f.csv', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, table_format) assert run_query(instance, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format(bucket=bucket, max_path=max_path)] @@ -281,11 +282,11 @@ def test_put_get_with_globs(cluster): # ("'minio','minio123',",True), Redirect with credentials not working with nginx. ("'wrongid','wrongkey',", False) ]) -def test_multipart_put(cluster, maybe_auth, positive): +def test_multipart_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" # Minimum size of part is 5 Mb for Minio. @@ -303,7 +304,7 @@ def test_multipart_put(cluster, maybe_auth, positive): filename = "test_multipart.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, maybe_auth, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, maybe_auth, table_format) try: run_query(instance, put_query, stdin=csv_data, settings={'s3_min_upload_part_size': min_part_size_bytes, @@ -315,23 +316,23 @@ def test_multipart_put(cluster, maybe_auth, positive): assert positive # Use proxy access logs to count number of parts uploaded to Minio. - proxy_logs = cluster.get_container_logs("proxy1") # type: str + proxy_logs = started_cluster.get_container_logs("proxy1") # type: str assert proxy_logs.count("PUT /{}/{}".format(bucket, filename)) >= 2 - assert csv_data == get_s3_file_content(cluster, bucket, filename) + assert csv_data == get_s3_file_content(started_cluster, bucket, filename) -def test_remote_host_filter(cluster): - instance = cluster.instances["restricted_dummy"] +def test_remote_host_filter(started_cluster): + instance = started_cluster.instances["restricted_dummy"] format = "column1 UInt32, column2 UInt32, column3 UInt32" query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format( - "invalid_host", cluster.minio_port, cluster.minio_bucket, format) + "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format) assert "not allowed in config.xml" in instance.query_and_get_error(query) other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format( - "invalid_host", cluster.minio_port, cluster.minio_bucket, format, other_values) + "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format, other_values) assert "not allowed in config.xml" in instance.query_and_get_error(query) @@ -339,8 +340,8 @@ def test_remote_host_filter(cluster): "''", # 1 arguments "'','','','','',''" # 6 arguments ]) -def test_wrong_s3_syntax(cluster, s3_storage_args): - instance = cluster.instances["dummy"] # type: ClickHouseInstance +def test_wrong_s3_syntax(started_cluster, s3_storage_args): + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance expected_err_msg = "Code: 42" # NUMBER_OF_ARGUMENTS_DOESNT_MATCH query = "create table test_table_s3_syntax (id UInt32) ENGINE = S3({})".format(s3_storage_args) @@ -348,9 +349,9 @@ def test_wrong_s3_syntax(cluster, s3_storage_args): # https://en.wikipedia.org/wiki/One_Thousand_and_One_Nights -def test_s3_glob_scheherazade(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance +def test_s3_glob_scheherazade(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" values = "(1, 1, 1)" @@ -361,7 +362,7 @@ def test_s3_glob_scheherazade(cluster): for i in range(start, end): path = "night_{}/tale.csv".format(i) query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, path, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, path, table_format, values) run_query(instance, query) jobs.append(threading.Thread(target=add_tales, args=(night, min(night + nights_per_job, 1001)))) @@ -371,20 +372,20 @@ def test_s3_glob_scheherazade(cluster): job.join() query = "select count(), sum(column1), sum(column2), sum(column3) from s3('http://{}:{}/{}/night_*/tale.csv', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, table_format) assert run_query(instance, query).splitlines() == ["1001\t1001\t1001\t1001"] -def run_s3_mock(cluster): +def run_s3_mock(started_cluster): logging.info("Starting s3 mock") - container_id = cluster.get_container_id('resolver') + container_id = started_cluster.get_container_id('resolver') current_dir = os.path.dirname(__file__) - cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") - cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) + started_cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") + started_cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) # Wait for S3 mock start for attempt in range(10): - ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), + ping_response = started_cluster.exec_in_container(started_cluster.get_container_id('resolver'), ["curl", "-s", "http://resolver:8080/"], nothrow=True) if ping_response != 'OK': if attempt == 9: @@ -397,25 +398,25 @@ def run_s3_mock(cluster): logging.info("S3 mock started") -def test_custom_auth_headers(cluster): +def test_custom_auth_headers(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=cluster.minio_restricted_bucket, + bucket=started_cluster.minio_restricted_bucket, file=filename, table_format=table_format) - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance result = run_query(instance, get_query) assert result == '1\t2\t3\n' -def test_custom_auth_headers_exclusion(cluster): +def test_custom_auth_headers_exclusion(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" - get_query = f"SELECT * FROM s3('http://resolver:8080/{cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" + get_query = f"SELECT * FROM s3('http://resolver:8080/{started_cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance with pytest.raises(helpers.client.QueryRuntimeException) as ei: result = run_query(instance, get_query) print(result) @@ -423,33 +424,13 @@ def test_custom_auth_headers_exclusion(cluster): assert ei.value.returncode == 243 assert '403 Forbidden' in ei.value.stderr - -def test_infinite_redirect(cluster): - bucket = "redirected" - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - filename = "test.csv" - get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=bucket, - file=filename, - table_format=table_format) - instance = cluster.instances["dummy"] # type: ClickHouseInstance - exception_raised = False - try: - run_query(instance, get_query) - except Exception as e: - assert str(e).find("Too many redirects while trying to access") != -1 - exception_raised = True - finally: - assert exception_raised - - @pytest.mark.parametrize("extension,method", [ ("bin", "gzip"), ("gz", "auto") ]) -def test_storage_s3_get_gzip(cluster, extension, method): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_get_gzip(started_cluster, extension, method): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = f"test_get_gzip.{extension}" name = "test_get_gzip" data = [ @@ -474,11 +455,11 @@ def test_storage_s3_get_gzip(cluster, extension, method): compressed = gzip.GzipFile(fileobj=buf, mode="wb") compressed.write(("\n".join(data)).encode()) compressed.close() - put_s3_file_content(cluster, bucket, filename, buf.getvalue()) + put_s3_file_content(started_cluster, bucket, filename, buf.getvalue()) try: run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( - 'http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', + 'http://{started_cluster.minio_host}:{MINIO_INTERNAL_PORT}/{bucket}/{filename}', 'CSV', '{method}')""") @@ -488,9 +469,9 @@ def test_storage_s3_get_gzip(cluster, extension, method): run_query(instance, f"DROP TABLE {name}") -def test_storage_s3_put_uncompressed(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_put_uncompressed(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = "test_put_uncompressed.bin" name = "test_put_uncompressed" data = [ @@ -512,13 +493,13 @@ def test_storage_s3_put_uncompressed(cluster): ] try: run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV')".format( - name, cluster.minio_host, cluster.minio_port, bucket, filename)) + name, started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename)) run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["753"] - uncompressed_content = get_s3_file_content(cluster, bucket, filename) + uncompressed_content = get_s3_file_content(started_cluster, bucket, filename) assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 753 finally: run_query(instance, f"DROP TABLE {name}") @@ -528,9 +509,9 @@ def test_storage_s3_put_uncompressed(cluster): ("bin", "gzip"), ("gz", "auto") ]) -def test_storage_s3_put_gzip(cluster, extension, method): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_put_gzip(started_cluster, extension, method): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = f"test_put_gzip.{extension}" name = "test_put_gzip" data = [ @@ -552,7 +533,7 @@ def test_storage_s3_put_gzip(cluster, extension, method): ] try: run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( - 'http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', + 'http://{started_cluster.minio_host}:{MINIO_INTERNAL_PORT}/{bucket}/{filename}', 'CSV', '{method}')""") @@ -560,7 +541,7 @@ def test_storage_s3_put_gzip(cluster, extension, method): run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["708"] - buf = io.BytesIO(get_s3_file_content(cluster, bucket, filename, decode=False)) + buf = io.BytesIO(get_s3_file_content(started_cluster, bucket, filename, decode=False)) f = gzip.GzipFile(fileobj=buf, mode="rb") uncompressed_content = f.read().decode() assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 diff --git a/tests/integration/test_storage_s3/test_redirect.py b/tests/integration/test_storage_s3/test_redirect.py new file mode 100644 index 00000000000..ab677f46d9a --- /dev/null +++ b/tests/integration/test_storage_s3/test_redirect.py @@ -0,0 +1,125 @@ +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + +# Creates S3 bucket for tests and allows anonymous read-write access to it. +def prepare_s3_bucket(cluster): + # Allows read-write access for bucket without authorization. + bucket_read_write_policy = {"Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*" + } + ]} + + minio_client = cluster.minio_client + minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) + + cluster.minio_restricted_bucket = "{}-with-auth".format(cluster.minio_bucket) + if minio_client.bucket_exists(cluster.minio_restricted_bucket): + minio_client.remove_bucket(cluster.minio_restricted_bucket) + + minio_client.make_bucket(cluster.minio_restricted_bucket) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__, name="redirect") + cluster.add_instance("dummy", with_minio=True, main_configs=["configs/defaultS3.xml"]) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + run_s3_mock(cluster) + + yield cluster + finally: + cluster.shutdown() + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + +def run_s3_mock(cluster): + logging.info("Starting s3 mock") + container_id = cluster.get_container_id('resolver') + current_dir = os.path.dirname(__file__) + cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") + cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) + + # Wait for S3 mock start + for attempt in range(10): + ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), + ["curl", "-s", "http://resolver:8080/"], nothrow=True) + if ping_response != 'OK': + if attempt == 9: + assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) + else: + time.sleep(1) + else: + break + + logging.info("S3 mock started") + +def test_infinite_redirect(started_cluster): + bucket = "redirected" + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + filename = "test.csv" + get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( + bucket=bucket, + file=filename, + table_format=table_format) + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + exception_raised = False + try: + run_query(instance, get_query) + except Exception as e: + assert str(e).find("Too many redirects while trying to access") != -1 + exception_raised = True + finally: + assert exception_raised \ No newline at end of file From 775f31f65eb932a10b3f90b87c7143f03e86c831 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sat, 20 Feb 2021 18:23:09 +0300 Subject: [PATCH 034/652] fix --- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/runner | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5443659eac9..872d34d657c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -589,10 +589,10 @@ class ClickHouseCluster: image_id = container_info.get('Image') image_info = self.docker_client.api.inspect_image(image_id) logging.debug(("Command failed in container {}: ".format(container_id))) - pprint.plogging.debug(container_info) + pprint.pprint(container_info) logging.debug("") logging.debug(("Container {} uses image {}: ".format(container_id, image_id))) - pprint.plogging.debug(image_info) + pprint.pprint(image_info) logging.debug("") message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output) diff --git a/tests/integration/runner b/tests/integration/runner index 99987839f56..f03554f179d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -199,7 +199,8 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=120 \ + {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, From 905cc7a2bd96540114e6b9de1da2361a6d9ce871 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Feb 2021 14:46:58 +0300 Subject: [PATCH 035/652] fixes --- tests/integration/helpers/cluster.py | 87 ++++++++++++------- tests/integration/runner | 12 +-- .../test_alter_on_mixed_type_cluster/test.py | 1 - .../test_format_avro_confluent/test.py | 11 +-- tests/integration/test_s3_with_https/test.py | 2 +- tests/integration/test_storage_kafka/test.py | 40 ++------- .../test_storage_kerberized_kafka/test.py | 38 +------- 7 files changed, 79 insertions(+), 112 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 872d34d657c..d69321268e5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -15,17 +15,19 @@ import traceback import urllib.parse import cassandra.cluster -import docker import psycopg2 import pymongo import pymysql import requests +from confluent_kafka.avro.cached_schema_registry_client import \ + CachedSchemaRegistryClient from dict2xml import dict2xml -from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient from kazoo.client import KazooClient from kazoo.exceptions import KazooException from minio import Minio +import docker + from .client import Client from .hdfs_api import HDFSApi @@ -38,7 +40,7 @@ SANITIZER_SIGN = "==================" # to create docker-compose env file def _create_env_file(path, variables): - logging.debug("Env {} stored in {}".format(variables, path)) + logging.debug(f"Env {variables} stored in {path}") with open(path, 'w') as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") @@ -48,20 +50,19 @@ def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually - logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) - logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) - logging.debug('Env:\n{}\n'.format(env)) - raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) + logging.debug(f"Stderr:\n{res.stderr.decode('utf-8')}\n") + logging.debug(f"Stdout:\n{res.stdout.decode('utf-8')}\n") + logging.debug(f"Env:\n{env}\n") + raise Exception(f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}") # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 def get_open_port(): - import socket - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - s.bind(("",0)) - s.listen(1) - port = s.getsockname()[1] - s.close() - return port + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + s.bind(("",0)) + s.listen(1) + port = s.getsockname()[1] + s.close() + return port def subprocess_check_call(args): # Uncomment for debugging @@ -93,10 +94,23 @@ def get_docker_compose_path(): if os.path.exists(os.path.dirname('/compose/')): return os.path.dirname('/compose/') # default in docker runner container else: - logging.debug(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) + logging.debug(f"Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {LOCAL_DOCKER_COMPOSE_DIR}") return LOCAL_DOCKER_COMPOSE_DIR +def check_kafka_is_available(kafka_id, kafka_port): + p = subprocess.Popen(('docker', + 'exec', + '-i', + kafka_id, + '/usr/bin/kafka-broker-api-versions', + '--bootstrap-server', + f'INSIDE://localhost:{kafka_port}'), + stdout=subprocess.PIPE, stderr=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -202,7 +216,6 @@ class ClickHouseCluster: # available when with_kerberozed_kafka == True self.kerberized_kafka_host = "kerberized_kafka1" self.kerberized_kafka_port = get_open_port() - self.kerberized_kafka_docker_id = None self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kerberized_kafka_host) # available when with_mongo == True @@ -238,7 +251,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - logging.debug("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) + logging.debug(f"CLUSTER INIT base_config_dir:{self.base_config_dir}") def get_client_cmd(self): cmd = self.client_bin_path @@ -299,7 +312,7 @@ class ClickHouseCluster: return self.base_kerberized_hdfs_cmd def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): - self.with_redis = True + self.with_kafka = True env_variables['KAFKA_HOST'] = self.kafka_host env_variables['KAFKA_EXTERNAL_PORT'] = str(self.kafka_port) env_variables['SCHEMA_REGISTRY_EXTERNAL_PORT'] = str(self.schema_registry_port) @@ -310,7 +323,7 @@ class ClickHouseCluster: return self.base_kafka_cmd def setup_kerberized_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): - self.with_redis = True + self.with_kerberized_kafka = True env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' env_variables['KERBERIZED_KAFKA_HOST'] = self.kerberized_kafka_host env_variables['KERBERIZED_KAFKA_EXTERNAL_PORT'] = str(self.kerberized_kafka_port) @@ -694,13 +707,25 @@ class ClickHouseCluster: proxy_port=self.hdfs_kerberized_name_port, data_port=self.hdfs_kerberized_data_port, hdfs_ip=hdfs_ip, - kdc_ip=kdc_ip) - + kdc_ip=kdc_ip) else: logging.debug("Create HDFSApi host={}".format("localhost")) hdfs_api = HDFSApi(user="root", host="localhost", data_port=self.hdfs_data_port, proxy_port=self.hdfs_name_port) return hdfs_api + def wait_kafka_is_available(self, kafka_docker_id, kafka_port, max_retries=50): + retries = 0 + while True: + if check_kafka_is_available(kafka_docker_id, kafka_port): + break + else: + retries += 1 + if retries > max_retries: + raise Exception("Kafka is not available") + logging.debug("Waiting for Kafka to start up") + time.sleep(1) + + def wait_hdfs_to_start(self, hdfs_api, timeout=60): start = time.time() while time.time() - start < timeout: @@ -722,13 +747,14 @@ class ClickHouseCluster: while time.time() - start < timeout: try: connection.list_database_names() - logging.debug("Connected to Mongo dbs:", connection.database_names()) + logging.debug("Connected to Mongo dbs: {}", connection.database_names()) return except Exception as ex: logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): + os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', secret_key='minio123', @@ -757,7 +783,7 @@ class ClickHouseCluster: raise Exception("Can't wait Minio to start") def wait_schema_registry_to_start(self, timeout=10): - sr_client = CachedSchemaRegistryClient('http://localhost:{}'.format(cluster.schema_registry_port)) + sr_client = CachedSchemaRegistryClient('http://localhost:{}'.format(self.schema_registry_port)) start = time.time() while time.time() - start < timeout: try: @@ -845,11 +871,13 @@ class ClickHouseCluster: if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: logging.debug('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.wait_kafka_is_available(self.kerberized_kafka_docker_id, self.kerberized_kafka_port, 100) if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) @@ -885,7 +913,7 @@ class ClickHouseCluster: if self.minio_certs_dir is None: os.mkdir(os.path.join(self.minio_dir, 'certs')) else: - shutil.copytree(self.minio_certs_dir, os.path.join(self.minio_dir, 'certs')) + shutil.copytree(os.path.join(self.base_dir, self.minio_certs_dir), os.path.join(self.minio_dir, 'certs')) minio_start_cmd = self.base_minio_cmd + common_opts @@ -1307,6 +1335,7 @@ class ClickHouseInstance: user='root') self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid())) from helpers.test_tools import assert_eq_with_retry + # wait start assert_eq_with_retry(self, "select 1", "1", retry_count=retries) @@ -1462,7 +1491,7 @@ class ClickHouseInstance: shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config.d configs - logging.debug("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) + logging.debug(f"Copy custom test config files {self.custom_main_config_paths} to {self.config_d_dir}") for path in self.custom_main_config_paths: shutil.copy(path, self.config_d_dir) @@ -1475,16 +1504,16 @@ class ClickHouseInstance: shutil.copy(path, dictionaries_dir) db_dir = p.abspath(p.join(self.path, 'database')) - logging.debug("Setup database dir {}".format(db_dir)) + logging.debug(f"Setup database dir {db_dir}") if self.clickhouse_path_dir is not None: - logging.debug("Database files taken from {}".format(self.clickhouse_path_dir)) + logging.debug(f"Database files taken from {self.clickhouse_path_dir}") shutil.copytree(self.clickhouse_path_dir, db_dir) - logging.debug("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) + logging.debug(f"Database copied from {self.clickhouse_path_dir} to {db_dir}") else: os.mkdir(db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) - logging.debug("Setup logs dir {}".format(logs_dir)) + logging.debug(f"Setup logs dir {logs_dir}") os.mkdir(logs_dir) depends_on = [] diff --git a/tests/integration/runner b/tests/integration/runner index f03554f179d..af29466e4fa 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -27,7 +27,7 @@ def check_args_and_update_paths(args): else: CLICKHOUSE_ROOT = args.clickhouse_root else: - logging.info("ClickHouse root is not set. Will use {}".format(DEFAULT_CLICKHOUSE_ROOT)) + logging.info(f"ClickHouse root is not set. Will use {DEFAULT_CLICKHOUSE_ROOT}") CLICKHOUSE_ROOT = DEFAULT_CLICKHOUSE_ROOT if not os.path.isabs(args.binary): @@ -43,23 +43,23 @@ def check_args_and_update_paths(args): args.base_configs_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.base_configs_dir)) else: args.base_configs_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, CONFIG_DIR_IN_REPO)) - logging.info("Base configs dir is not set. Will use {}".format(args.base_configs_dir)) + logging.info(f"Base configs dir is not set. Will use {args.base_configs_dir}") if args.cases_dir: if not os.path.isabs(args.cases_dir): args.cases_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.cases_dir)) else: args.cases_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, INTERGATION_DIR_IN_REPO)) - logging.info("Cases dir is not set. Will use {}".format(args.cases_dir)) + logging.info(f"Cases dir is not set. Will use {args.cases_dir}") if args.src_dir: if not os.path.isabs(args.src_dir): args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.src_dir)) else: args.src_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, SRC_DIR_IN_REPO)) - logging.info("src dir is not set. Will use {}".format(args.src_dir)) + logging.info(f"src dir is not set. Will use {args.src_dir}") - logging.info("base_configs_dir: {}, binary: {}, cases_dir: {} ".format(args.base_configs_dir, args.binary, args.cases_dir)) + logging.info(f"base_configs_dir: {args.base_configs_dir}, binary: {args.binary}, cases_dir: {args.cases_dir} ") for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: if not os.path.exists(path): @@ -183,7 +183,7 @@ if __name__ == "__main__": elif image == "yandex/clickhouse-kerberos-kdc": env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: - logging.info("Unknown image {}".format(image)) + logging.info(f"Unknown image {image}") # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 269072f239a..5e516176c0b 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -1,7 +1,6 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index ec2ed875075..49bc7baeb39 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -3,6 +3,7 @@ import logging import avro.schema import pytest +from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance @@ -11,7 +12,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("dummy", with_kafka=True) @@ -37,10 +38,10 @@ def run_query(instance, query, data=None, settings=None): return result -def test_select(cluster): +def test_select(started_cluster): # type: (ClickHouseCluster) -> None - schema_registry_client = cluster.schema_registry_client + schema_registry_client = CachedSchemaRegistryClient('http://localhost:{}'.format(started_cluster.schema_registry_port)) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object({ @@ -62,9 +63,9 @@ def test_select(cluster): buf.write(message) data = buf.getvalue() - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( - cluster.schema_registry_host, + started_cluster.schema_registry_host, 8081 ) diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 58e3b3c2a3b..1a5c6e2ce12 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -20,7 +20,7 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/log_conf.xml", "configs/config.d/ssl.xml"], with_minio=True, - minio_certs_dir='minio_certs') + minio_certs_dir="minio_certs") logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index da5826665e5..98c3855ddd9 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -24,13 +24,12 @@ from kafka.admin import NewTopic from kafka.protocol.admin import DescribeGroupsRequest_v1 from kafka.protocol.group import MemberAssignment -""" -protoc --version -libprotoc 3.0.0 -# to create kafka_pb2.py -protoc --python_out=. kafka.proto -""" +# protoc --version +# libprotoc 3.0.0 +# # to create kafka_pb2.py +# protoc --python_out=. kafka.proto + from . import kafka_pb2 from . import social_pb2 @@ -51,36 +50,8 @@ instance = cluster.add_instance('instance', "kafka_client_id":"instance", "kafka_format_json_each_row":"JSONEachRow"}, clickhouse_path_dir='clickhouse_path') -kafka_id = cluster.kafka_docker_id -# Helpers - -def check_kafka_is_available(): - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), - stdout=subprocess.PIPE, stderr=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise Exception("Kafka is not available") - logging.debug("Waiting for Kafka to start up") - time.sleep(1) - def get_kafka_producer(port, serializer): errors = [] for _ in range(15): @@ -645,7 +616,6 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() # logging.debug("kafka is available - running test") yield # run test diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index e21d18c2f49..94d2156c2ff 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -3,6 +3,7 @@ import random import threading import time import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -24,37 +25,6 @@ instance = cluster.add_instance('instance', with_kerberized_kafka=True, clickhouse_path_dir="clickhouse_path" ) -kafka_id = cluster.kerberized_kafka_docker_id - -# Helpers - -def check_kafka_is_available(): - - # plaintext - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise Exception("Kafka is not available") - print("Waiting for Kafka to start up") - time.sleep(1) - def producer_serializer(x): return x.encode() if isinstance(x, str) else x @@ -73,8 +43,8 @@ def get_kafka_producer(port, serializer): raise Exception("Connection not establised, {}".format(errors)) def kafka_produce(kafka_cluster, topic, messages, timestamp=None): - logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) - producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer) + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kerberized_kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kerberized_kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() @@ -94,8 +64,6 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() - print("kafka is available - running test") yield # run test # Tests From 74f6ab798f71b015dc5fd2bd06af9f9c360f0d60 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 14:52:39 +0300 Subject: [PATCH 036/652] Add __restrict modifier to AggregateFunctionSequenceNextNode --- .../AggregateFunctionSequenceNextNode.h | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index be672974ac6..c406fee557e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -177,7 +177,7 @@ public: [[maybe_unused]] auto a = new (place) Data; } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { Node * node = Node::allocate(*columns[1], row_num, arena); @@ -202,7 +202,7 @@ public: data(place).value.push_back(node, arena); } - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { if (data(rhs).value.empty()) return; @@ -241,7 +241,7 @@ public: data(place).sorted = true; } - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { writeBinary(data(place).sorted, buf); @@ -251,7 +251,7 @@ public: node->write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override { readBinary(data(place).sorted, buf); @@ -305,7 +305,7 @@ public: return 0; } - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { auto & value = data(place).value; @@ -359,7 +359,7 @@ public: a.value.push_back(v->clone(arena), arena); } - void create(AggregateDataPtr place) const override + void create(AggregateDataPtr __restrict place) const override { [[maybe_unused]] auto a = new (place) Data; } @@ -369,7 +369,7 @@ public: return Descending ? lhs_timestamp < rhs_timestamp : lhs_timestamp > rhs_timestamp; } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { bool is_first = true; auto & value = data(place).value; @@ -394,7 +394,7 @@ public: } } - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { auto & a = data(place).value; auto & b = data(rhs).value; @@ -415,7 +415,7 @@ public: } } - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { writeBinary(data(place).sorted, buf); @@ -425,7 +425,7 @@ public: node->write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override { readBinary(data(place).sorted, buf); @@ -442,7 +442,7 @@ public: value[i] = Node::read(buf, arena); } - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { auto & value = data(place).value; From 6f28b6c4a6a61824ff97df55b0f9f880f81b5d7f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 14:53:24 +0300 Subject: [PATCH 037/652] Minor changes in AggregateFunctionSequenceNextNode --- .../parametric-functions.md | 16 +++--- .../AggregateFunctionSequenceNextNode.cpp | 50 +++++++++++-------- .../AggregateFunctionSequenceNextNode.h | 38 +++++++------- .../0_stateless/01656_sequence_next_node.sql | 2 +- 4 files changed, 57 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 3bb22461e5d..96e0013db19 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -513,21 +513,21 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, event3, ...) +sequenceNextNode(descending_order)(timestamp, event_column, event1, event2, event3, ...) ``` **Parameters** - `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. -- `timestamp` — Name of the column containing the timestamp. Data types supported: Date, DateTime and other unsigned integer types. -- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: String and Nullable(String) -- `cond` — Conditions describing the chain of events. UInt8. + +**Arguments** +- `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. +- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `cond` — Conditions describing the chain of events. `UInt8` **Returned value** -Nullable(String). - -- event_column[next_index] - if the pattern is matched and next value exists. -- null - if the pattern isn’t matched or next value doesn't exist. +- `event_column[next_index]` - if the pattern is matched and next value exists. +- `NULL` - if the pattern isn’t matched or next value doesn't exist. Type: `Nullable(String)`. diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index af90c80de61..1513e5b3213 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -6,7 +6,6 @@ #include #include #include -#include "registerAggregateFunctions.h" namespace DB @@ -22,36 +21,37 @@ namespace ErrorCodes namespace { -template +template inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) { if (argument_types.size() == 2) { // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. if (descending_order) - return std::make_shared>(data_type); + return std::make_shared>(data_type); else - return std::make_shared>(data_type); + return std::make_shared>(data_type); } else { if (descending_order) - return std::make_shared>(data_type, argument_types); + return std::make_shared>(data_type, argument_types); else - return std::make_shared>(data_type, argument_types); + return std::make_shared>(data_type, argument_types); } } -template -AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) +AggregateFunctionPtr +createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, const DataTypes & argument_types, const Array & parameters) { bool descending_order = false; if (parameters.size() == 1) { auto type = parameters[0].getType(); - if (type != Field::Types::Int64 && type != Field::Types::UInt64) - throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); + bool is_correct_type = type == Field::Types::Int64 || type == Field::Types::UInt64; + if (!is_correct_type || (parameters[0].get() != 0 && parameters[0].get() != 1)) + throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); descending_order = parameters[0].get(); } @@ -60,9 +60,13 @@ AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & nam ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (argument_types.size() < 2) - throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > MaxArgs) - throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 31 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Aggregate function " + name + " requires at least two arguments.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > max_args + 2) + throw Exception("Aggregate function " + name + " requires at most " + + std::to_string(max_args + 2) + + " (timestamp, value_column, " + std::to_string(max_args) + " events) arguments.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) { @@ -73,15 +77,11 @@ AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & nam } if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) - throw Exception{"Illegal type " + argument_types.front().get()->getName() + throw Exception{"Illegal type " + argument_types[1].get()->getName() + " of second argument of aggregate function " + name + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - DataTypePtr data_type; - if (typeid_cast(argument_types[1].get())) - data_type = argument_types[1]; - else - data_type = std::make_shared(argument_types[1]); + DataTypePtr data_type = makeNullable(argument_types[1]); WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) @@ -102,13 +102,21 @@ AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & nam ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } +auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_args) +{ + return [max_args](const std::string & name, const DataTypes & argument_types, const Array & parameters) + { + return createAggregateFunctionSequenceNode(name, max_args, argument_types, parameters); + }; +} + } void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode<2 + 31>, properties }); - factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNode<2>, properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(31), properties }); + factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c406fee557e..69381b2aa95 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -32,10 +32,10 @@ namespace DB template struct NodeBase { - UInt64 size; // size of payload + UInt64 size; /// size of payload DataTypeDateTime::FieldType event_time; - UInt32 events_bitset; // Bitsets of UInt32 are easy to compare. (< operator on bitsets) + UInt32 events_bitset; /// Bitsets of UInt32 are easy to compare. (< operator on bitsets) char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -95,7 +95,7 @@ struct NodeString : public NodeBase }; /// TODO : Expends SequenceNextNodeGeneralData to support other types -template +template struct SequenceNextNodeGeneralData { using Allocator = MixedAlignedArenaAllocator; @@ -130,9 +130,9 @@ struct SequenceNextNodeGeneralData /// Implementation of sequenceNextNode template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { - using Data = SequenceNextNodeGeneralData; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } @@ -142,7 +142,7 @@ class SequenceNextNodeImpl final public: SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) , events_size(arguments.size() - 2) @@ -158,11 +158,11 @@ public: const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties & /*properties*/) const override { - // This aggregate function sets insertion_requires_nullable_column on. - // Even though some values are mapped to aggregating key, it could return nulls for the below case. - // aggregated events: [A -> B -> C] - // events to find: [C -> D] - // [C -> D] is not matched to 'A -> B -> C' so that it returns null. + /// This aggregate function sets insertion_requires_nullable_column on. + /// Even though some values are mapped to aggregating key, it could return nulls for the below case. + /// aggregated events: [A -> B -> C] + /// events to find: [C -> D] + /// [C -> D] is not matched to 'A -> B -> C' so that it returns null. return std::make_shared>(nested_function, arguments, params); } @@ -174,7 +174,7 @@ public: void create(AggregateDataPtr place) const override { - [[maybe_unused]] auto a = new (place) Data; + new (place) Data; } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override @@ -218,8 +218,8 @@ public: for (UInt64 i = 0; i < new_elems; ++i) a.push_back(b[i]->clone(arena), arena); - /// either sort whole container or do so partially merging ranges afterwards - using Comparator = typename SequenceNextNodeGeneralData::Comparator; + /// Either sort whole container or do so partially merging ranges afterwards + using Comparator = typename SequenceNextNodeGeneralData::Comparator; if (!data(place).sorted && !data(rhs).sorted) std::stable_sort(std::begin(a), std::end(a), Comparator{}); @@ -326,17 +326,17 @@ public: /// Implementation of sequenceFirstNode template class SequenceFirstNodeImpl final - : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> + : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> { - using Data = SequenceNextNodeGeneralData; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } DataTypePtr & data_type; public: - SequenceFirstNodeImpl(const DataTypePtr & data_type_) - : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( + explicit SequenceFirstNodeImpl(const DataTypePtr & data_type_) + : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) { @@ -361,7 +361,7 @@ public: void create(AggregateDataPtr __restrict place) const override { - [[maybe_unused]] auto a = new (place) Data; + new (place) Data; } bool compare(const T lhs_timestamp, const T rhs_timestamp) const diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 9af59d5c8e2..c2270bc10ac 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; -CREATE TABLE iF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; +CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); From bc8db50142c02f53cabfc0a2010e72b267ed941d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Feb 2021 18:14:21 +0300 Subject: [PATCH 038/652] no python3 in CI by defalult --- tests/integration/runner | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index af29466e4fa..5ad37726821 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -27,7 +27,7 @@ def check_args_and_update_paths(args): else: CLICKHOUSE_ROOT = args.clickhouse_root else: - logging.info(f"ClickHouse root is not set. Will use {DEFAULT_CLICKHOUSE_ROOT}") + logging.info("ClickHouse root is not set. Will use %s" % (DEFAULT_CLICKHOUSE_ROOT)) CLICKHOUSE_ROOT = DEFAULT_CLICKHOUSE_ROOT if not os.path.isabs(args.binary): @@ -43,23 +43,23 @@ def check_args_and_update_paths(args): args.base_configs_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.base_configs_dir)) else: args.base_configs_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, CONFIG_DIR_IN_REPO)) - logging.info(f"Base configs dir is not set. Will use {args.base_configs_dir}") + logging.info("Base configs dir is not set. Will use %s" % (args.base_configs_dir)) if args.cases_dir: if not os.path.isabs(args.cases_dir): args.cases_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.cases_dir)) else: args.cases_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, INTERGATION_DIR_IN_REPO)) - logging.info(f"Cases dir is not set. Will use {args.cases_dir}") + logging.info("Cases dir is not set. Will use %s" % (args.cases_dir)) if args.src_dir: if not os.path.isabs(args.src_dir): args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.src_dir)) else: args.src_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, SRC_DIR_IN_REPO)) - logging.info(f"src dir is not set. Will use {args.src_dir}") + logging.info("src dir is not set. Will use %s" % (args.src_dir)) - logging.info(f"base_configs_dir: {args.base_configs_dir}, binary: {args.binary}, cases_dir: {args.cases_dir} ") + logging.info("base_configs_dir: %s , binary: %s, cases_dir: %s " % (args.base_configs_dir, args.binary, args.cases_dir)) for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: if not os.path.exists(path): @@ -183,7 +183,7 @@ if __name__ == "__main__": elif image == "yandex/clickhouse-kerberos-kdc": env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: - logging.info(f"Unknown image {image}") + logging.info("Unknown image %s" % (image)) # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time From f8f475e83fc5a08a113417ff385eaacd1292ec2b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Feb 2021 21:24:16 +0300 Subject: [PATCH 039/652] fix --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e2e57e92187..0347eee1719 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -754,7 +754,7 @@ class ClickHouseCluster: time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): - os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') + os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', secret_key='minio123', From 7dc4b6d58e26462ffa5ff19ea9f86e887c7bc5eb Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 25 Feb 2021 19:31:20 +0900 Subject: [PATCH 040/652] Add comments and fix AggregateFunctionSequenceNextNode --- .../AggregateFunctionSequenceNextNode.h | 43 ++++++++++++++++--- 1 file changed, 36 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 69381b2aa95..9bde44be06e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,6 +27,19 @@ namespace DB { +/** + * When sorting the list of events the EMPTY_EVENTS_BITSET will be moved to the last. + * In the case of events, + * dt action + * 2020-01-01 00:00:01 'D' + * 2020-01-01 00:00:01 'A' + * 2020-01-01 00:00:01 'B' + * 2020-01-01 00:00:01 'C' + * The next node of a chain of events 'A' -> 'B' -> 'C' is expected to be the 'D'. + * Because EMPTY_EVENTS_BITSET is 0x80000000 the order of the sorted events is ['A", 'B', 'C', 'D']. The result value of this aggregation is 'D'. + * If EMPTY_EVENTS_BITSET is 0 hen the order of the sorted events is ['D', 'A', 'B', 'C']. This time, the result value is NULL. + */ +static const UInt32 EMPTY_EVENTS_BITSET = 0x80000000; /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template @@ -36,6 +49,8 @@ struct NodeBase DataTypeDateTime::FieldType event_time; UInt32 events_bitset; /// Bitsets of UInt32 are easy to compare. (< operator on bitsets) + /// Nodes in the list must be sorted in order to find a chain of events at the method getNextNodeIndex(). + /// While sorting, events_bitset is one of sorting criteria. char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -92,6 +107,11 @@ struct NodeString : public NodeBase { assert_cast(column).insertData(data(), size); } + + bool compare(const Node * rhs) const + { + return strcmp(data(), rhs->data()) < 0; + } }; /// TODO : Expends SequenceNextNodeGeneralData to support other types @@ -110,10 +130,12 @@ struct SequenceNextNodeGeneralData { if constexpr (Descending) return lhs->event_time == rhs->event_time ? - lhs->events_bitset < rhs->events_bitset : lhs->event_time > rhs->event_time; + (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) + : lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? - lhs->events_bitset < rhs->events_bitset : lhs->event_time < rhs->event_time; + (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) + : lhs->event_time < rhs->event_time; } }; @@ -163,7 +185,7 @@ public: /// aggregated events: [A -> B -> C] /// events to find: [C -> D] /// [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const @@ -194,7 +216,7 @@ public: for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) events_bitset += (1 << i); - if (events_bitset == 0) events_bitset = 0x80000000; // Any events are not matched. + if (events_bitset == 0) events_bitset = EMPTY_EVENTS_BITSET; // Any events are not matched. node->event_time = timestamp; node->events_bitset = events_bitset; @@ -278,7 +300,12 @@ public: } /// This method returns an index of next node that matched the events. - /// It is one as referring Boyer-Moore-Algorithm. + /// It is one as referring Boyer-Moore-Algorithm(https://en.wikipedia.org/wiki/Boyer%E2%80%93Moore_string-search_algorithm). + /// But, there are some differences. + /// In original Boyer-Moore-Algorithm compares strings, but this algorithm compares events_bits. + /// events_bitset consists of events_bits. + /// matched events in the chain of events are represented as a bitmask of UInt32. + /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { if (data.value.size() <= events_size) @@ -292,6 +319,8 @@ public: UInt32 j = 0; /// It checks whether the chain of events are matched or not. for (; j < events_size; ++j) + /// It compares each matched events. + /// The lower bitmask is the former matched event. if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) break; @@ -350,7 +379,7 @@ public: const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties &) const override { - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const @@ -388,7 +417,7 @@ public: { Node * node = Node::allocate(*columns[1], row_num, arena); node->event_time = timestamp; - node->events_bitset = 0x80000000; + node->events_bitset = EMPTY_EVENTS_BITSET; data(place).value.push_back(node, arena); } From 66e9098e4e8100626fd8ef3a8428b69a4272e254 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 26 Feb 2021 14:56:59 +0900 Subject: [PATCH 041/652] Fix comparator of NodeString --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 9bde44be06e..e188ae58045 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -110,7 +110,8 @@ struct NodeString : public NodeBase bool compare(const Node * rhs) const { - return strcmp(data(), rhs->data()) < 0; + auto cmp = strncmp(data(), rhs->data(), std::min(size, rhs->size)); + return (cmp == 0) ? size <= rhs->size : cmp < 0; } }; From ec3d63bfe471f25bf5fbe09c30cdfca7fddbcf0a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 3 Mar 2021 11:09:44 +0300 Subject: [PATCH 042/652] add check for fatal --- tests/integration/helpers/cluster.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0347eee1719..09b3731e2bb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -955,6 +955,7 @@ class ClickHouseCluster: def shutdown(self, kill=True): sanitizer_assert_instance = None + fatal_log = None if self.up_called: with open(self.docker_logs_path, "w+") as f: try: @@ -967,6 +968,15 @@ class ClickHouseCluster: sanitizer_assert_instance = line.split('|')[0].strip() break + for instance in list(self.instances.values()): + if instance.contains_in_log(SANITIZER_SIGN): + sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) + logging.ERROR(f"Sanitizer in instance {instance.name} fatal log {fatal_log}") + + if instance.contains_in_log("Fatal"): + fatal_log = instance.grep_in_log("Fatal") + logging.ERROR(f"Crash in instance {instance.name} fatal log {fatal_log}") + if kill: try: subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) @@ -1002,6 +1012,7 @@ class ClickHouseCluster: raise Exception( "Sanitizer assert found in {} for instance {}".format(self.docker_logs_path, sanitizer_assert_instance)) + def pause_container(self, instance_name): subprocess_check_call(self.base_cmd + ['pause', instance_name]) @@ -1287,6 +1298,11 @@ class ClickHouseInstance: ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) return len(result) > 0 + def grep_in_log(self, substring): + result = self.exec_in_container( + ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) + return result + def file_exists(self, path): return self.exec_in_container( ["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n' From 34609445f75ce10117b3692d99417380a149d99d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 12:06:39 +0300 Subject: [PATCH 043/652] fix --- tests/integration/test_storage_kafka/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 1a3a6ce136a..b9635ecd738 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -50,9 +50,9 @@ instance = cluster.add_instance('instance', clickhouse_path_dir='clickhouse_path') -def get_kafka_producer(port, serializer): +def get_kafka_producer(port, serializer, retries): errors = [] - for _ in range(15): + for _ in range(retries): try: producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) logging.debug("Kafka Connection establised: localhost:{}".format(port)) @@ -66,7 +66,7 @@ def get_kafka_producer(port, serializer): def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=2): +def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer, retries) for message in messages: From 8b6a64e8b8b5ccee6c0a0f9092509f881ca86248 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 12:07:59 +0300 Subject: [PATCH 044/652] typo --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 24153b4236d..2f70d77eb08 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1320,7 +1320,7 @@ class ClickHouseInstance: wait_duration = time.time() - start_time logging.debug('{} log line matching "{}" appeared in a {} seconds'.format(repetitions, regexp, wait_duration)) - return wait_duratio + return wait_duration def file_exists(self, path): return self.exec_in_container( From 8122003c98cfa99e36a8703c12da932630a029e5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 13:53:50 +0300 Subject: [PATCH 045/652] more fixes --- tests/integration/test_storage_kafka/test.py | 44 ++++++++++---------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b9635ecd738..f72b2b105bf 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -19,7 +19,9 @@ from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection +from kafka.protocol.admin import DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment from kafka.admin import NewTopic @@ -75,7 +77,7 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): ## just to ensure the python client / producer is working properly def kafka_producer_send_heartbeat_msg(max_retries=50): - kafka_produce('test_heartbeat_topic', ['test'], retries=max_retries) + kafka_produce(kafka_cluster, 'test_heartbeat_topic', ['test'], retries=max_retries) def kafka_consume(kafka_cluster, topic): consumer = KafkaConsumer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest") @@ -154,8 +156,8 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' else: return TSV(result) == TSV(reference) -def describe_consumer_group(name): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") +def describe_consumer_group(kafka_cluster, name): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) consumer_groups = admin_client.describe_consumer_groups([name]) res = [] for member in consumer_groups[0].members: @@ -216,7 +218,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('old', messages) + kafka_produce(kafka_cluster, 'old', messages) result = '' while True: @@ -248,16 +250,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) # Insert couple of malformed messages. - kafka_produce('new', ['}{very_broken_message,']) - kafka_produce('new', ['}another{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) result = '' while True: @@ -907,7 +909,7 @@ def test_kafka_issue4116(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="consumer_hang", num_partitions=8, replication_factor=1)) @@ -959,7 +961,7 @@ def test_kafka_consumer_hang(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang2(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="consumer_hang2", num_partitions=1, replication_factor=1)) @@ -1010,7 +1012,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -1035,7 +1037,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) - kafka_produce('tsv', messages) + kafka_produce(kafka_cluster, 'tsv', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -1057,7 +1059,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_select_empty(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="empty", num_partitions=1, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1107,9 +1109,9 @@ def test_kafka_json_without_delimiter(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_protobuf(kafka_cluster): - kafka_produce_protobuf_messages('pb', 0, 20) - kafka_produce_protobuf_messages('pb', 20, 1) - kafka_produce_protobuf_messages('pb', 21, 29) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 0, 20) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 20, 1) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 21, 29) instance.query(''' CREATE TABLE test.kafka (key UInt64, value String) @@ -1133,9 +1135,9 @@ def test_kafka_protobuf(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_string_field_on_first_position_in_protobuf(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/12615 - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 21, 29) instance.query(''' CREATE TABLE test.kafka ( @@ -1349,7 +1351,7 @@ def test_librdkafka_compression(kafka_cluster): SELECT * FROM test.kafka; '''.format(topic_name=topic_name) ) - kafka_produce(topic_name, messages) + kafka_produce(kafka_cluster, topic_name, messages) instance.wait_for_log_line("Committed offset {}".format(number_of_messages)) From f03cf47a8264ea9970fd0a614d87277b9ec16f48 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 14:22:19 +0300 Subject: [PATCH 046/652] fix --- tests/integration/test_storage_kafka/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index f72b2b105bf..115f1e926bc 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2525,7 +2525,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.pause_container('kafka1') instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) + CREATE TABLE test.test_kafka_unavailable (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'test_kafka_unavailable', @@ -2534,7 +2534,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_max_block_size = 1000, kafka_flush_interval_ms = 1000; - CREATE MATERIALIZED VIEW test.destination Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_kafka_unavailable Engine=Log AS SELECT key, now() as consume_ts, @@ -2544,10 +2544,10 @@ def test_kafka_unavailable(kafka_cluster): _offset, _partition, _timestamp - FROM test.kafka; + FROM test.test_kafka_unavailable; ''') - instance.query("SELECT * FROM test.kafka") + instance.query("SELECT * FROM test.test_kafka_unavailable") instance.wait_for_log_line('brokers are down') instance.wait_for_log_line('stalled. Reschedule', repetitions=2) @@ -2555,7 +2555,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.unpause_container('kafka1') instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination")) == 2000 + assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 2000 time.sleep(5) # needed to give time for kafka client in python test to recovery @pytest.mark.timeout(180) From 298824c9262b78df3c140d3f29ea81b6d0df3e3a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 14:52:56 +0300 Subject: [PATCH 047/652] fix --- tests/integration/test_storage_kafka/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 115f1e926bc..e0c337775a4 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2525,6 +2525,8 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.pause_container('kafka1') instance.query(''' + DROP TABLE IF EXISTS test.destination_kafka_unavailable; + DROP TABLE IF EXISTS test.test_kafka_unavailable; CREATE TABLE test.test_kafka_unavailable (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -2555,7 +2557,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.unpause_container('kafka1') instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 2000 + assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 5000 time.sleep(5) # needed to give time for kafka client in python test to recovery @pytest.mark.timeout(180) From fad1a4ccb3c79958699b8292da6aecc63efcca5e Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Fri, 5 Mar 2021 21:46:09 +0900 Subject: [PATCH 048/652] Fix comparator of sequenceNextNode for more deterministic sorting --- .../AggregateFunctionSequenceNextNode.cpp | 2 +- .../AggregateFunctionSequenceNextNode.h | 49 ++++++------------- .../01656_sequence_next_node.reference | 16 +++--- .../0_stateless/01656_sequence_next_node.sql | 37 ++++++++------ 4 files changed, 45 insertions(+), 59 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 1513e5b3213..9d1d8aaa075 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -115,7 +115,7 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_args) void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(31), properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(MAX_EVENTS_SIZE), properties }); factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index e188ae58045..b8cf7ef5f5d 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,19 +27,7 @@ namespace DB { -/** - * When sorting the list of events the EMPTY_EVENTS_BITSET will be moved to the last. - * In the case of events, - * dt action - * 2020-01-01 00:00:01 'D' - * 2020-01-01 00:00:01 'A' - * 2020-01-01 00:00:01 'B' - * 2020-01-01 00:00:01 'C' - * The next node of a chain of events 'A' -> 'B' -> 'C' is expected to be the 'D'. - * Because EMPTY_EVENTS_BITSET is 0x80000000 the order of the sorted events is ['A", 'B', 'C', 'D']. The result value of this aggregation is 'D'. - * If EMPTY_EVENTS_BITSET is 0 hen the order of the sorted events is ['D', 'A', 'B', 'C']. This time, the result value is NULL. - */ -static const UInt32 EMPTY_EVENTS_BITSET = 0x80000000; +const UInt32 MAX_EVENTS_SIZE = 64; /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template @@ -48,9 +36,7 @@ struct NodeBase UInt64 size; /// size of payload DataTypeDateTime::FieldType event_time; - UInt32 events_bitset; /// Bitsets of UInt32 are easy to compare. (< operator on bitsets) - /// Nodes in the list must be sorted in order to find a chain of events at the method getNextNodeIndex(). - /// While sorting, events_bitset is one of sorting criteria. + std::bitset events_bitset; char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -68,7 +54,8 @@ struct NodeBase buf.write(data(), size); writeBinary(event_time, buf); - writeBinary(events_bitset, buf); + UInt64 ulong_bitset = events_bitset.to_ulong(); + writeBinary(ulong_bitset, buf); } static Node * read(ReadBuffer & buf, Arena * arena) @@ -81,7 +68,9 @@ struct NodeBase buf.read(node->data(), size); readBinary(node->event_time, buf); - readBinary(node->events_bitset, buf); + UInt64 ulong_bitset; + readBinary(ulong_bitset, buf); + node->events_bitset = ulong_bitset; return node; } @@ -130,13 +119,9 @@ struct SequenceNextNodeGeneralData bool operator()(const Node * lhs, const Node * rhs) const { if constexpr (Descending) - return lhs->event_time == rhs->event_time ? - (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) - : lhs->event_time > rhs->event_time; + return lhs->event_time == rhs->event_time ? !lhs->compare(rhs) : lhs->event_time > rhs->event_time; else - return lhs->event_time == rhs->event_time ? - (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) - : lhs->event_time < rhs->event_time; + return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; } }; @@ -179,7 +164,7 @@ public: AggregateFunctionPtr getOwnNullAdapter( const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, - const AggregateFunctionProperties & /*properties*/) const override + const AggregateFunctionProperties &) const override { /// This aggregate function sets insertion_requires_nullable_column on. /// Even though some values are mapped to aggregating key, it could return nulls for the below case. @@ -213,14 +198,11 @@ public: /// 0x00000000 /// + 1 (bit of event1) /// + 4 (bit of event3) - UInt32 events_bitset = 0; + node->events_bitset.reset(); for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) - events_bitset += (1 << i); - if (events_bitset == 0) events_bitset = EMPTY_EVENTS_BITSET; // Any events are not matched. - + node->events_bitset.set(i); node->event_time = timestamp; - node->events_bitset = events_bitset; data(place).value.push_back(node, arena); } @@ -295,7 +277,7 @@ public: { UInt32 k = 0; for (; k < events_size - j; ++k) - if (data.value[i - j]->events_bitset & (1 << (events_size - 1 - j - k))) + if (data.value[i - j]->events_bitset.test(events_size - 1 - j - k)) return k; return k; } @@ -322,7 +304,7 @@ public: for (; j < events_size; ++j) /// It compares each matched events. /// The lower bitmask is the former matched event. - if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) + if (data.value[i - j]->events_bitset.test(events_size - 1 - j) == false) break; /// If the chain of events are matched returns the index of result value. @@ -413,12 +395,11 @@ public: is_first = false; } - if (is_first) { Node * node = Node::allocate(*columns[1], row_num, arena); node->event_time = timestamp; - node->events_bitset = EMPTY_EVENTS_BITSET; + node->events_bitset.reset(); data(place).value.push_back(node, arena); } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 50755232cb9..da6ec2d97bf 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -124,14 +124,14 @@ (0, A) id >= 10 10 B (0, A) id >= 10 10 B (0, A) id >= 10 10 A -(0, A) id = 11 0 -(0, C) id = 11 0 -(0, B->C) id = 11 0 -(0, A->B->C) id = 11 0 -(0, A) id = 11 0 -(0, C) id = 11 0 -(0, C->B) id = 11 0 -(0, C->B->A) id = 11 0 +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, B->C) id = 11 1 +(0, A->B->C) id = 11 1 +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, C->B) id = 11 1 +(0, C->B->A) id = 11 1 (0) id < 10 1 A (0) id < 10 2 A (0) id < 10 3 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index c2270bc10ac..2c16f33aa0e 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -61,14 +61,19 @@ SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS n SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); + +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; @@ -146,14 +151,14 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; From 534500ef84dff4b3bb66a9d726f7658dc28433a3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 5 Mar 2021 16:39:51 +0300 Subject: [PATCH 049/652] wip --- .../runner/compose/docker_compose_mysql.yml | 20 +++++- tests/integration/helpers/cluster.py | 63 ++++++++++++++----- tests/integration/helpers/external_sources.py | 4 ++ tests/integration/test_mysql_protocol/test.py | 10 +-- 4 files changed, 73 insertions(+), 24 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 3075abdc181..c9b45af6563 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -5,6 +5,20 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - ${MYSQL_EXTERNAL_PORT}:${MYSQL_INTERNAL_PORT} - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_PORT} + command: --server_id=100 + --log-bin='mysql-bin-1.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/error.log + --general-log=ON + --general-log-file=/mysql/general.log + volumes: + - type: ${MYSQL_LOGS_FS:-tmpfs} + source: ${MYSQL_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 2f70d77eb08..588b2599a34 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3,6 +3,7 @@ import errno import http.client import logging import os +import stat import os.path as p import pprint import pwd @@ -111,7 +112,6 @@ def check_kafka_is_available(kafka_id, kafka_port): p.communicate() return p.returncode == 0 - class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -242,11 +242,16 @@ class ClickHouseCluster: # available when with_mysql == True self.mysql_host = "mysql57" - self.mysql_port = get_open_port() + self.mysql_port = 3306 + self.mysql_ip = None + self.mysql_dir = p.abspath(p.join(self.instances_dir, "mysql")) + self.mysql_logs_dir = os.path.join(self.mysql_dir, "logs") # available when with_mysql8 == True self.mysql8_host = "mysql80" self.mysql8_port = get_open_port() + self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8")) + self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs") self.zookeeper_use_tmpfs = True @@ -263,8 +268,10 @@ class ClickHouseCluster: def setup_mysql_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_mysql = True env_variables['MYSQL_HOST'] = self.mysql_host - env_variables['MYSQL_EXTERNAL_PORT'] = str(self.mysql_port) - env_variables['MYSQL_INTERNAL_PORT'] = "3306" + env_variables['MYSQL_PORT'] = str(self.mysql_port) + env_variables['MYSQL_ROOT_HOST'] = '%' + env_variables['MYSQL_LOGS'] = self.mysql_logs_dir + env_variables['MYSQL_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] @@ -276,6 +283,8 @@ class ClickHouseCluster: env_variables['MYSQL8_HOST'] = self.mysql8_host env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port) env_variables['MYSQL8_INTERNAL_PORT'] = "3306" + env_variables['MYSQL8_LOGS'] = self.mysql8_logs_dir + env_variables['MYSQL8_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')]) self.base_mysql8_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')] @@ -627,12 +636,13 @@ class ClickHouseCluster: ["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)], user='root') - def wait_mysql_to_start(self, timeout=60): + def wait_mysql_to_start(self, timeout=30): + self.mysql_ip = self.get_instance_ip('mysql57') start = time.time() errors = [] while time.time() - start < timeout: try: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=self.mysql_ip, port=self.mysql_port) conn.close() logging.debug("Mysql Started") return @@ -812,6 +822,7 @@ class ClickHouseCluster: if self.is_up: return + # Just in case kill unstopped containers from previous launch try: logging.debug("Trying to kill unstopped containers...") @@ -822,6 +833,15 @@ class ClickHouseCluster: except: pass + # # Just in case remove unused networks + # try: + # logging.debug("Trying to prune unused networks...") + + # subprocess_call(['docker', 'network', 'prune', '-f']) + # logging.debug("Networks pruned") + # except: + # pass + try: if destroy_dirs and p.exists(self.instances_dir): logging.debug(("Removing instances dir %s", self.instances_dir)) @@ -831,7 +851,7 @@ class ClickHouseCluster: logging.debug(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) instance.create_dir(destroy_dir=destroy_dirs) - self.docker_client = docker.from_env(version=self.docker_api_version) + self.docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180) common_opts = ['up', '-d', '--force-recreate'] @@ -856,11 +876,14 @@ class ClickHouseCluster: if self.with_mysql and self.base_mysql_cmd: logging.debug('Setup MySQL') + os.makedirs(self.mysql_logs_dir) + os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(120) + self.wait_mysql_to_start(60) if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') + os.makedirs(self.mysql8_logs_dir) subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start(120) @@ -935,7 +958,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 20.0 # seconds + start_deadline = time.time() + 30.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -970,13 +993,19 @@ class ClickHouseCluster: break for instance in list(self.instances.values()): - if instance.contains_in_log(SANITIZER_SIGN): - sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) - logging.ERROR(f"Sanitizer in instance {instance.name} fatal log {fatal_log}") + try: + if not instance.is_up: + continue + if instance.contains_in_log(SANITIZER_SIGN): + sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) + logging.ERROR(f"Sanitizer in instance {instance.name} log {sanitizer_assert_instance}") - if instance.contains_in_log("Fatal"): - fatal_log = instance.grep_in_log("Fatal") - logging.ERROR(f"Crash in instance {instance.name} fatal log {fatal_log}") + if instance.contains_in_log("Fatal"): + fatal_log = instance.grep_in_log("Fatal") + name = instance.name + logging.ERROR(f"Crash in instance {name} fatal log {fatal_log}") + except Exception as e: + logging.error(f"Failed to check fails in logs: {e}") if kill: try: @@ -1177,6 +1206,7 @@ class ClickHouseInstance: self.ipv6_address = ipv6_address self.with_installed_binary = with_installed_binary self.env_file = os.path.join(os.path.dirname(self.docker_compose_path), DEFAULT_ENV_NAME) + self.is_up = False def is_built_with_thread_sanitizer(self): @@ -1342,7 +1372,7 @@ class ClickHouseInstance: return None return None - def restart_with_latest_version(self, stop_start_wait_sec=10, callback_onstop=None, signal=15): + def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=15): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root') @@ -1410,6 +1440,7 @@ class ClickHouseInstance: sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) sock.settimeout(time_left) sock.connect((self.ip_address, 9000)) + self.is_up = True return except socket.timeout: continue diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index fe4c5269a56..66cd45583a7 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -9,6 +9,7 @@ import cassandra.cluster import pymongo import pymysql.cursors import redis +import logging from tzlocal import get_localzone @@ -59,6 +60,7 @@ class SourceMySQL(ExternalSource): } def create_mysql_conn(self): + logging.debug(f"pymysql connect {self.user}, {self.password}, {self.internal_hostname}, {self.internal_port}") self.connection = pymysql.connect( user=self.user, password=self.password, @@ -98,6 +100,8 @@ class SourceMySQL(ExternalSource): ) def prepare(self, structure, table_name, cluster): + if self.internal_hostname is None: + self.internal_hostname = cluster.mysql_ip self.create_mysql_conn() self.execute_mysql_query("create database if not exists test default character set 'utf8'") self.execute_mysql_query("drop table if exists test.{}".format(table_name)) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 7f7d59674bc..bc7cb6024c0 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -36,7 +36,7 @@ def mysql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') @pytest.fixture(scope='module') @@ -63,7 +63,7 @@ def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') @@ -71,7 +71,7 @@ def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_php1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') @@ -79,7 +79,7 @@ def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') @@ -87,7 +87,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_java1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') def test_mysql_client(mysql_client, server_address): From a6c7ddbcdf253a314d386b3029e0b03739f9410d Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Sun, 7 Mar 2021 22:52:23 +0900 Subject: [PATCH 050/652] Fix comparator --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index b8cf7ef5f5d..0f642542daf 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -100,7 +100,7 @@ struct NodeString : public NodeBase bool compare(const Node * rhs) const { auto cmp = strncmp(data(), rhs->data(), std::min(size, rhs->size)); - return (cmp == 0) ? size <= rhs->size : cmp < 0; + return (cmp == 0) ? size < rhs->size : cmp < 0; } }; From e6f90298bba0cc11a5de7730906b01d36a3e8ab5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 9 Mar 2021 10:32:10 +0300 Subject: [PATCH 051/652] wip --- docker/test/integration/runner/Dockerfile | 1 - ...ompose_mysql_5_7_for_materialize_mysql.yml | 7 ++++ .../compose/docker_compose_mysql_8_0.yml | 14 ++++++-- .../integration/runner/dockerd-entrypoint.sh | 2 +- tests/integration/helpers/cluster.py | 18 ++++++---- tests/integration/helpers/network.py | 4 +-- tests/integration/runner | 2 +- .../configs/merge_tree_settings.xml | 2 +- .../test_adaptive_granularity/test.py | 5 +++ tests/integration/test_cluster_copier/test.py | 36 +++++++++---------- .../test_cluster_copier/trivial_test.py | 22 ++++++------ .../test_mysql.py | 4 +-- .../test.py | 2 +- .../test_mysql_database_engine/test.py | 16 ++++----- tests/integration/test_mysql_protocol/test.py | 5 +-- .../test_postgresql_protocol/test.py | 4 +-- 16 files changed, 86 insertions(+), 58 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 0a1ab0662a8..6b6651e5e3f 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -40,7 +40,6 @@ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV DOCKER_CHANNEL stable -ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml index 5aa13ba91c7..ba693fd9fb4 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml @@ -12,3 +12,10 @@ services: --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 + --log-error=/var/log/mysqld/error.log + --general-log=ON + --general-log-file=/var/log/mysqld/general.log + volumes: + - type: ${MYSQL_LOGS_FS:-tmpfs} + source: ${MYSQL_LOGS:-} + target: /var/log/mysqld/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml index a7fb5d85d8b..e13076c4e2e 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -5,9 +5,19 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - ${MYSQL8_EXTERNAL_PORT}:${MYSQL8_INTERNAL_PORT} + MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL8_PORT} command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/error.log + --general-log=ON + --general-log-file=/mysql/general.log + volumes: + - type: ${MYSQL8_LOGS_FS:-tmpfs} + source: ${MYSQL8_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c0255d3d706..9f04dde720d 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,7 +1,7 @@ #!/bin/bash set -e -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.0.0.0/8,size=24 &>/var/log/somefile & set +e reties=0 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 588b2599a34..eb6ecc8a39f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -249,7 +249,8 @@ class ClickHouseCluster: # available when with_mysql8 == True self.mysql8_host = "mysql80" - self.mysql8_port = get_open_port() + self.mysql8_port = 3306 + self.mysql8_ip = None self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8")) self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs") @@ -281,8 +282,8 @@ class ClickHouseCluster: def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_mysql8 = True env_variables['MYSQL8_HOST'] = self.mysql8_host - env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port) - env_variables['MYSQL8_INTERNAL_PORT'] = "3306" + env_variables['MYSQL8_PORT'] = str(self.mysql8_port) + env_variables['MYSQL8_ROOT_HOST'] = '%' env_variables['MYSQL8_LOGS'] = self.mysql8_logs_dir env_variables['MYSQL8_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')]) @@ -578,7 +579,7 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - start_deadline = time.time() + 20.0 # seconds + start_deadline = time.time() + 60.0 # seconds node.wait_for_start(start_deadline) return node @@ -876,6 +877,8 @@ class ClickHouseCluster: if self.with_mysql and self.base_mysql_cmd: logging.debug('Setup MySQL') + if os.path.exists(self.mysql_dir): + shutil.rmtree(self.mysql_dir) os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) @@ -883,7 +886,10 @@ class ClickHouseCluster: if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') + if os.path.exists(self.mysql8_dir): + shutil.rmtree(self.mysql8_dir) os.makedirs(self.mysql8_logs_dir) + os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start(120) @@ -958,7 +964,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 30.0 # seconds + start_deadline = time.time() + 60.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -1372,7 +1378,7 @@ class ClickHouseInstance: return None return None - def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=15): + def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=60): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root') diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index aa697c63d72..120ec9e873d 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -159,12 +159,12 @@ class _NetworkManager: def __init__( self, - container_expire_timeout=50, container_exit_timeout=60): + container_expire_timeout=50, container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION")): self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout - self._docker_client = docker.from_env(version=os.environ.get("DOCKER_API_VERSION")) + self._docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=docker_api_version, timeout=180) self._container = None diff --git a/tests/integration/runner b/tests/integration/runner index 5ad37726821..f3309f983a3 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -199,7 +199,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=120 \ + --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=180 \ {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, diff --git a/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml b/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml index 31ede39c318..2ee416c156e 100644 --- a/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml +++ b/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 12bfc22d7d9..47209908e5c 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -268,6 +268,11 @@ def test_version_single_node_update(start_dynamic_cluster, n, tables): ] ) def test_mixed_granularity_single_node(start_dynamic_cluster, node): + assert node.name == "node9" or node.name == "node10" + assert_eq_with_retry(node, + "SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'", + '0') # check that enable_mixed_granularity_parts is off by default + node.query( "INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)") node.query( diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index d87969630cd..abeb8739b7d 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -252,10 +252,10 @@ class Task_non_partitioned_table: instance.query("DROP TABLE copier_test1_1") -def execute_task(task, cmd_options): +def execute_task(started_cluster, task, cmd_options): task.start() - zk = cluster.get_kazoo_client('zoo1') + zk = started_cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) try: @@ -268,7 +268,7 @@ def execute_task(task, cmd_options): zk.create(zk_task_path + "/description", task.copier_task_config.encode()) # Run cluster-copier processes on each node - docker_api = docker.from_env().api + docker_api = started_cluster.docker_client.api copiers_exec_ids = [] cmd = ['/usr/bin/clickhouse', 'copier', @@ -280,7 +280,7 @@ def execute_task(task, cmd_options): copiers = random.sample(list(cluster.instances.keys()), 3) for instance_name in copiers: - instance = cluster.instances[instance_name] + instance = started_cluster.instances[instance_name] container = instance.get_docker_handle() instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") @@ -293,7 +293,7 @@ def execute_task(task, cmd_options): # Wait for copiers stopping and check their return codes for exec_id, instance_name in zip(copiers_exec_ids, copiers): - instance = cluster.instances[instance_name] + instance = started_cluster.instances[instance_name] while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -319,9 +319,9 @@ def execute_task(task, cmd_options): ) def test_copy_simple(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--experimental-use-sample-offset', '1']) + execute_task(started_cluster, Task1(started_cluster), ['--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), []) + execute_task(started_cluster, Task1(started_cluster), []) @pytest.mark.parametrize( @@ -333,10 +333,10 @@ def test_copy_simple(started_cluster, use_sample_offset): ) def test_copy_with_recovering(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.parametrize( @@ -348,40 +348,40 @@ def test_copy_with_recovering(started_cluster, use_sample_offset): ) def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster, "test1"), []) + execute_task(started_cluster, Task2(started_cluster, "test1"), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): - execute_task(Task_test_block_size(started_cluster), []) + execute_task(started_cluster, Task_test_block_size(started_cluster), []) def test_no_index(started_cluster): - execute_task(Task_no_index(started_cluster), []) + execute_task(started_cluster, Task_no_index(started_cluster), []) def test_no_arg(started_cluster): - execute_task(Task_no_arg(started_cluster), []) + execute_task(started_cluster, Task_no_arg(started_cluster), []) def test_non_partitioned_table(started_cluster): - execute_task(Task_non_partitioned_table(started_cluster), []) + execute_task(started_cluster, Task_non_partitioned_table(started_cluster), []) if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 8a43440ac90..717ff9d8d34 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -79,10 +79,10 @@ class TaskTrivial: node.query("DROP TABLE trivial") -def execute_task(task, cmd_options): +def execute_task(started_cluster, task, cmd_options): task.start() - zk = cluster.get_kazoo_client('zoo1') + zk = started_cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) zk_task_path = task.zk_task_path @@ -90,7 +90,7 @@ def execute_task(task, cmd_options): zk.create(zk_task_path + "/description", task.copier_task_config) # Run cluster-copier processes on each node - docker_api = docker.from_env().api + docker_api = started_cluster.docker_client.api copiers_exec_ids = [] cmd = ['/usr/bin/clickhouse', 'copier', @@ -101,7 +101,7 @@ def execute_task(task, cmd_options): print(cmd) - for instance_name, instance in cluster.instances.items(): + for instance_name, instance in started_cluster.instances.items(): container = instance.get_docker_handle() exec_id = docker_api.exec_create(container.id, cmd, stderr=True) docker_api.exec_start(exec_id, detach=True) @@ -110,7 +110,7 @@ def execute_task(task, cmd_options): print("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) # Wait for copiers stopping and check their return codes - for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + for exec_id, instance in zip(copiers_exec_ids, iter(started_cluster.instances.values())): while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -137,10 +137,10 @@ def execute_task(task, cmd_options): ) def test_trivial_copy(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) + execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) else: print("AAAAA") - execute_task(TaskTrivial(started_cluster, use_sample_offset), []) + execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), []) @pytest.mark.parametrize( @@ -152,10 +152,10 @@ def test_trivial_copy(started_cluster, use_sample_offset): ) def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.parametrize( @@ -167,10 +167,10 @@ def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): ) def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) if __name__ == '__main__': diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 119709bf229..7feba20f3a1 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -26,7 +26,7 @@ def setup_module(module): cluster = ClickHouseCluster(__file__, name=test_name) - SOURCE = SourceMySQL("MySQL", "localhost", cluster.mysql_port, cluster.mysql_host, "3306", "root", "clickhouse") + SOURCE = SourceMySQL("MySQL", None, cluster.mysql_port, cluster.mysql_host, cluster.mysql_port, "root", "clickhouse") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() @@ -55,7 +55,7 @@ def teardown_module(module): def started_cluster(): try: cluster.start() - + simple_tester.prepare(cluster) complex_tester.prepare(cluster) ranged_tester.prepare(cluster) diff --git a/tests/integration/test_match_process_uid_against_data_owner/test.py b/tests/integration/test_match_process_uid_against_data_owner/test.py index c9f87e683c8..ed37227ab15 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -20,7 +20,7 @@ def test_different_user(): cluster.start() - docker_api = docker.from_env().api + docker_api = cluster.docker_client.api container = node.get_docker_handle() container.stop() container.start() diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index f5fb9be337d..07a4a987b59 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -21,7 +21,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): + def __init__(self, user, password, hostname, port): self.user = user self.port = port self.hostname = hostname @@ -55,7 +55,7 @@ class MySQLNodeInstance: def test_mysql_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") @@ -89,7 +89,7 @@ def test_mysql_ddl_for_mysql_database(started_cluster): def test_clickhouse_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;') @@ -114,7 +114,7 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster): def test_clickhouse_dml_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `i``d` int(11) NOT NULL, PRIMARY KEY (`i``d`)) ENGINE=InnoDB;') @@ -132,7 +132,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster): def test_clickhouse_join_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test.t1_mysql_local (" "pays VARCHAR(55) DEFAULT 'FRA' NOT NULL," @@ -155,7 +155,7 @@ def test_clickhouse_join_for_mysql_database(started_cluster): def test_bad_arguments_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( @@ -165,7 +165,7 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster): def test_data_types_support_level_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) @@ -307,7 +307,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m else: return [do_execute(q) for q in query] - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node: execute_query(mysql_node, [ "DROP DATABASE IF EXISTS ${mysql_db}", "CREATE DATABASE ${mysql_db} DEFAULT CHARACTER SET 'utf8'", diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index bc7cb6024c0..1e0d3652a91 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -5,6 +5,7 @@ import math import os import time +import logging import docker import pymysql.connections import pytest @@ -36,7 +37,7 @@ def mysql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') @pytest.fixture(scope='module') @@ -96,7 +97,7 @@ def test_mysql_client(mysql_client, server_address): mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba -e "SELECT 1;" '''.format(host=server_address, port=server_port), demux=True) - + logging.debug(f"test_mysql_client code:{code} stdout:{stdout}, stderr:{stderr}") assert stdout.decode() == '\n'.join(['1', '1', '']) code, (stdout, stderr) = mysql_client.exec_run(''' diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 633c02ef408..1404cf409f3 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.from_env().containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') @@ -65,7 +65,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.from_env().containers.get(cluster.project_name + '_java_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') def test_psql_is_ready(psql_server): From 07dc7e81e324072e460cb0a76ff306e457905136 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Mar 2021 16:39:06 +0300 Subject: [PATCH 052/652] Remove MAX_EVENTS_SIZE from AggregateFunctionSequenceNextNode.h --- .../AggregateFunctionSequenceNextNode.cpp | 12 ++++++++---- .../AggregateFunctionSequenceNextNode.h | 17 ++++++++--------- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 9d1d8aaa075..9c865f3371a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes namespace { +constexpr size_t MAX_EVENTS_SIZE = 64; + template inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) { @@ -28,22 +30,24 @@ inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTy { // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. if (descending_order) - return std::make_shared>(data_type); + return std::make_shared, true>>(data_type); else - return std::make_shared>(data_type); + return std::make_shared, false>>(data_type); } else { if (descending_order) - return std::make_shared>(data_type, argument_types); + return std::make_shared, true>>(data_type, argument_types); else - return std::make_shared>(data_type, argument_types); + return std::make_shared, false>>(data_type, argument_types); } } AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, const DataTypes & argument_types, const Array & parameters) { + assert(max_args <= MAX_EVENTS_SIZE); + bool descending_order = false; if (parameters.size() == 1) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 0f642542daf..9efc78aa4c5 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,16 +27,14 @@ namespace DB { -const UInt32 MAX_EVENTS_SIZE = 64; - /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl -template +template struct NodeBase { UInt64 size; /// size of payload DataTypeDateTime::FieldType event_time; - std::bitset events_bitset; + std::bitset events_bitset; char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -77,9 +75,10 @@ struct NodeBase }; /// It stores String, timestamp, bitset of matched events. -struct NodeString : public NodeBase +template +struct NodeString : public NodeBase, MaxEventsSize> { - using Node = NodeString; + using Node = NodeString; static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) { @@ -94,13 +93,13 @@ struct NodeString : public NodeBase void insertInto(IColumn & column) { - assert_cast(column).insertData(data(), size); + assert_cast(column).insertData(this->data(), this->size); } bool compare(const Node * rhs) const { - auto cmp = strncmp(data(), rhs->data(), std::min(size, rhs->size)); - return (cmp == 0) ? size < rhs->size : cmp < 0; + auto cmp = strncmp(this->data(), rhs->data(), std::min(this->size, rhs->size)); + return (cmp == 0) ? this->size < rhs->size : cmp < 0; } }; From 5808bf24e2795554d0dc55d9306c3a2852710e5f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Mar 2021 16:40:32 +0300 Subject: [PATCH 053/652] Fix strict weak ordering, return always lower string in sequenceNextNode --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 9efc78aa4c5..bfece896809 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -118,7 +118,7 @@ struct SequenceNextNodeGeneralData bool operator()(const Node * lhs, const Node * rhs) const { if constexpr (Descending) - return lhs->event_time == rhs->event_time ? !lhs->compare(rhs) : lhs->event_time > rhs->event_time; + return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; } From f32a388fed0bed5f5f1b5017b018e6fcba0da5fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Mar 2021 16:56:29 +0300 Subject: [PATCH 054/652] Minor changes, add comments --- .../AggregateFunctionSequenceNextNode.h | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index bfece896809..a04f589f1e8 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -191,7 +191,6 @@ public: const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; /// The events_bitset variable stores matched events in the form of bitset. - /// It uses UInt32 instead of std::bitset because bitsets of UInt32 are easy to compare. (< operator on bitsets) /// Each Nth-bit indicates that the Nth-event are matched. /// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005. /// 0x00000000 @@ -272,12 +271,23 @@ public: value[i] = Node::read(buf, arena); } + /// Calculate position of current event in target chain and shift to corresponding offset + /// Lets consider case where we search chain 'ABCD': + /// - If current event is 'X' we can skip it and perform next step from position after this 'X' + /// - If current event is 'A' we will start from this position + /// - If current event is 'B' then second position in our chain should match this 'B'. + /// And we perform next step from position one before 'B'. + /// - And so on... inline UInt32 calculateJump(const Data & data, const UInt32 i, const UInt32 j) const { - UInt32 k = 0; + /// Fast check if value is zero, not in sequence + if (data.value[i - j]->events_bitset.none()) + return events_size - j; + + UInt32 k = 1; for (; k < events_size - j; ++k) if (data.value[i - j]->events_bitset.test(events_size - 1 - j - k)) - return k; + break; return k; } @@ -285,8 +295,7 @@ public: /// It is one as referring Boyer-Moore-Algorithm(https://en.wikipedia.org/wiki/Boyer%E2%80%93Moore_string-search_algorithm). /// But, there are some differences. /// In original Boyer-Moore-Algorithm compares strings, but this algorithm compares events_bits. - /// events_bitset consists of events_bits. - /// matched events in the chain of events are represented as a bitmask of UInt32. + /// Matched events in the chain of events are represented as a bitmask. /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { @@ -299,14 +308,16 @@ public: while (i < data.value.size()) { UInt32 j = 0; - /// It checks whether the chain of events are matched or not. + /// Try to match chain of events starting from the end of this chain. for (; j < events_size; ++j) + { /// It compares each matched events. /// The lower bitmask is the former matched event. - if (data.value[i - j]->events_bitset.test(events_size - 1 - j) == false) + if (!data.value[i - j]->events_bitset.test(events_size - 1 - j)) break; + } - /// If the chain of events are matched returns the index of result value. + /// Chain of events are matched, return the index of result value. if (j == events_size) return i + 1; From d8caf62c1b4f583e655eee574fd82f69129c0881 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 Mar 2021 12:06:05 +0300 Subject: [PATCH 055/652] Update test sequence_next_node, choose lowest string for same timestamps --- .../01656_sequence_next_node.reference | 8 ++++---- .../0_stateless/01656_sequence_next_node.sql | 16 ++++++++-------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index da6ec2d97bf..94a55f2d954 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -130,8 +130,8 @@ (0, A->B->C) id = 11 1 (0, A) id = 11 1 (0, C) id = 11 1 -(0, C->B) id = 11 1 -(0, C->B->A) id = 11 1 +(0, B->C) id = 11 1 +(0, B->C->D) id = 11 1 (0) id < 10 1 A (0) id < 10 2 A (0) id < 10 3 A @@ -288,8 +288,8 @@ (0, A->B->C) id = 11 1 (0, A) id = 11 1 (0, C) id = 11 1 -(0, C->B) id = 11 1 -(0, C->B->A) id = 11 1 +(0, B->C) id = 11 1 +(0, B->C->D) id = 11 1 (0) id < 10 1 A (0) id < 10 2 A (0) id < 10 3 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 2c16f33aa0e..57b0d7b1d8e 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -70,10 +70,10 @@ SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; @@ -155,10 +155,10 @@ SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; From e41c731f0f8e368bef8b46ab33e11e0fb72f801f Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Wed, 10 Mar 2021 20:22:44 +0900 Subject: [PATCH 056/652] Change the behavior of sequenceNextNode. --- .../parametric-functions.md | 152 +++- .../AggregateFunctionSequenceNextNode.cpp | 84 +- .../AggregateFunctionSequenceNextNode.h | 301 +++---- .../01656_sequence_next_node.reference | 765 +++++++++++------- .../0_stateless/01656_sequence_next_node.sql | 185 +++-- 5 files changed, 866 insertions(+), 621 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 33e5766089b..dcc63ebd960 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -513,11 +513,19 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode(descending_order)(timestamp, event_column, event1, event2, event3, ...) +sequenceNextNode(direction, base)(timestamp, event_column, event1, event2, event3, ...) ``` **Parameters** -- `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. +- `direction` - Used to navigate to directions. + - forward : Moving forward + - backward: Moving backward + +- `base` - Used to set the base point. + - head : Set the base point to the first event + - tail : Set the base point to the last event + - first_match : Set the base point to the first matched event1 + - last_match : Set the base point to the last matched event1 **Arguments** - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. @@ -535,26 +543,158 @@ Type: `Nullable(String)`. It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E. -The query statement searching the event following B->C : +The query statement searching the event following A->B : ``` sql CREATE TABLE test_flow ( dt DateTime, id int, - action String) + page String) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(dt) ORDER BY id; INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); -SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') as next_flow FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; ``` Result: ``` text ┌─id─┬─next_flow─┐ -│ 1 │ E │ +│ 1 │ C │ └────┴───────────┘ ``` + +**Behavior for `forward` and `head`** + +```SQL +ALTER TABLE test_flow DELETE WHERE 1 = 1 settings mutations_sync = 1; + +INSERT INTO test_flow VALUES (1, 1, 'Home') (2, 1, 'Gift') (3, 1, 'Exit'); +INSERT INTO test_flow VALUES (1, 2, 'Home') (2, 2, 'Home') (3, 2, 'Gift') (4, 2, 'Basket'); +INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, 'Basket'); +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page + 1970-01-01 09:00:01 1 Home // Base point, Matched with Home + 1970-01-01 09:00:02 1 Gift // Matched with Gift + 1970-01-01 09:00:03 1 Exit // The result + + 1970-01-01 09:00:01 3 Home // Base point, Matched with Home + 1970-01-01 09:00:02 3 Home // Unmatched with Gift + 1970-01-01 09:00:03 3 Gift + 1970-01-01 09:00:04 3 Basket + + 1970-01-01 09:00:01 4 Gift // Base point, Unmatched with Home + 1970-01-01 09:00:02 4 Home + 1970-01-01 09:00:03 4 Gift + 1970-01-01 09:00:04 4 Basket +``` + +**Behavior for `backward` and `tail`** + +```SQL +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift +1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Matched with Gift +1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket + +1970-01-01 09:00:01 4 Gift +1970-01-01 09:00:02 4 Home // The result +1970-01-01 09:00:03 4 Gift // Base point, Matched with Gift +1970-01-01 09:00:04 4 Basket // Base point, Matched with Basket +``` + + +**Behavior for `forward` and `first_match`** + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit // The result + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket The result + +1970-01-01 09:00:01 4 Gift // Base point +1970-01-01 09:00:02 4 Home // Thre result +1970-01-01 09:00:03 4 Gift +1970-01-01 09:00:04 4 Basket +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit // Unmatched with Home + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket // Unmatched with Home + +1970-01-01 09:00:01 4 Gift // Base point +1970-01-01 09:00:02 4 Home // Matched with Home +1970-01-01 09:00:03 4 Gift // The result +1970-01-01 09:00:04 4 Basket +``` + + +**Behavior for `backward` and `last_match`** + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home // The result +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket + +1970-01-01 09:00:01 4 Gift +1970-01-01 09:00:02 4 Home // The result +1970-01-01 09:00:03 4 Gift // Base point +1970-01-01 09:00:04 4 Basket +``` + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home // Matched with Home, the result is null +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit + +1970-01-01 09:00:01 3 Home // The result +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket + +1970-01-01 09:00:01 4 Gift // The result +1970-01-01 09:00:02 4 Home // Matched with Home +1970-01-01 09:00:03 4 Gift // Base point +1970-01-01 09:00:04 4 Basket +``` diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 9c865f3371a..274b75b383e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -11,6 +11,8 @@ namespace DB { +constexpr size_t MAX_EVENTS_SIZE = 64; + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -21,26 +23,26 @@ namespace ErrorCodes namespace { -constexpr size_t MAX_EVENTS_SIZE = 64; +template +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl2(const DataTypePtr data_type, const DataTypes & argument_types, SeqBase base) +{ + if (base == HEAD) + return std::make_shared, Direction, HEAD>>(data_type, argument_types); + else if (base == TAIL) + return std::make_shared, Direction, TAIL>>(data_type, argument_types); + else if (base == FIRST_MATCH) + return std::make_shared, Direction, FIRST_MATCH>>(data_type, argument_types); + else + return std::make_shared, Direction, LAST_MATCH>>(data_type, argument_types); +} template -inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl1(const DataTypePtr data_type, const DataTypes & argument_types, SeqDirection direction, SeqBase base) { - if (argument_types.size() == 2) - { - // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. - if (descending_order) - return std::make_shared, true>>(data_type); - else - return std::make_shared, false>>(data_type); - } + if (direction == FORWARD) + return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); else - { - if (descending_order) - return std::make_shared, true>>(data_type, argument_types); - else - return std::make_shared, false>>(data_type, argument_types); - } + return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); } AggregateFunctionPtr @@ -48,22 +50,32 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c { assert(max_args <= MAX_EVENTS_SIZE); - bool descending_order = false; - - if (parameters.size() == 1) - { - auto type = parameters[0].getType(); - bool is_correct_type = type == Field::Types::Int64 || type == Field::Types::UInt64; - if (!is_correct_type || (parameters[0].get() != 0 && parameters[0].get() != 1)) - throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); - - descending_order = parameters[0].get(); - } + String param_dir = parameters.at(0).safeGet(); + SeqDirection direction; + if (param_dir.compare("forward") == 0) + direction = FORWARD; + else if (param_dir.compare("backward") == 0) + direction = BACKWARD; else - throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS}; - if (argument_types.size() < 2) + String param_base = parameters.at(1).safeGet(); + SeqBase base; + if (param_base.compare("head") == 0) + base = HEAD; + else if (param_base.compare("tail") == 0) + base = TAIL; + else if (param_base.compare("first_match") == 0) + base = FIRST_MATCH; + else if (param_base.compare("last_match") == 0) + base = LAST_MATCH; + else + throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; + + if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < 3) + throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() < 2) throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); else if (argument_types.size() > max_args + 2) @@ -89,17 +101,17 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt16) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt32) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt64) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.isDate()) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.isDateTime()) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); throw Exception{"Illegal type " + argument_types.front().get()->getName() + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index a04f589f1e8..78d19beedc4 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,6 +27,20 @@ namespace DB { +enum SeqDirection +{ + FORWARD = 0, + BACKWARD = 1 +}; + +enum SeqBase +{ + HEAD = 0, + TAIL = 1, + FIRST_MATCH = 2, + LAST_MATCH = 3 +}; + /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template struct NodeBase @@ -104,7 +118,7 @@ struct NodeString : public NodeBase, MaxEventsSize> }; /// TODO : Expends SequenceNextNodeGeneralData to support other types -template +template struct SequenceNextNodeGeneralData { using Allocator = MixedAlignedArenaAllocator; @@ -117,10 +131,7 @@ struct SequenceNextNodeGeneralData { bool operator()(const Node * lhs, const Node * rhs) const { - if constexpr (Descending) - return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time > rhs->event_time; - else - return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; + return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; } }; @@ -134,12 +145,12 @@ struct SequenceNextNodeGeneralData } }; -/// Implementation of sequenceNextNode -template +/// Implementation of sequenceFirstNode +template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { - using Data = SequenceNextNodeGeneralData; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } @@ -149,7 +160,7 @@ class SequenceNextNodeImpl final public: SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) , events_size(arguments.size() - 2) @@ -165,12 +176,11 @@ public: const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties &) const override { - /// This aggregate function sets insertion_requires_nullable_column on. /// Even though some values are mapped to aggregating key, it could return nulls for the below case. /// aggregated events: [A -> B -> C] /// events to find: [C -> D] /// [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const @@ -222,7 +232,7 @@ public: a.push_back(b[i]->clone(arena), arena); /// Either sort whole container or do so partially merging ranges afterwards - using Comparator = typename SequenceNextNodeGeneralData::Comparator; + using Comparator = typename SequenceNextNodeGeneralData::Comparator; if (!data(place).sorted && !data(rhs).sorted) std::stable_sort(std::begin(a), std::end(a), Comparator{}); @@ -246,12 +256,35 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { + // Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node. + this->data(const_cast(place)).sort(); + writeBinary(data(place).sorted, buf); auto & value = data(place).value; - writeVarUInt(value.size(), buf); - for (auto & node : value) - node->write(buf); + + UInt64 size = std::max(static_cast(events_size + 1), value.size()); + switch (Base) + { + case HEAD: + writeVarUInt(size, buf); + for (UInt64 i = 0; i < size; ++i) + value[i]->write(buf); + break; + + case TAIL: + writeVarUInt(size, buf); + for (UInt64 i = value.size() - 1; i >= size; --i) + value[i]->write(buf); + break; + + case FIRST_MATCH: + case LAST_MATCH: + writeVarUInt(value.size(), buf); + for (auto & node : value) + node->write(buf); + break; + } } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override @@ -271,60 +304,83 @@ public: value[i] = Node::read(buf, arena); } - /// Calculate position of current event in target chain and shift to corresponding offset - /// Lets consider case where we search chain 'ABCD': - /// - If current event is 'X' we can skip it and perform next step from position after this 'X' - /// - If current event is 'A' we will start from this position - /// - If current event is 'B' then second position in our chain should match this 'B'. - /// And we perform next step from position one before 'B'. - /// - And so on... - inline UInt32 calculateJump(const Data & data, const UInt32 i, const UInt32 j) const + inline UInt32 getBaseIndex(Data & data, bool & exist) const { - /// Fast check if value is zero, not in sequence - if (data.value[i - j]->events_bitset.none()) - return events_size - j; + switch (Base) + { + case HEAD: + exist = true; + return 0; - UInt32 k = 1; - for (; k < events_size - j; ++k) - if (data.value[i - j]->events_bitset.test(events_size - 1 - j - k)) + case TAIL: + exist = true; + return data.value.size() - 1; + + case FIRST_MATCH: + for (UInt64 i = 0; i < data.value.size(); ++i) + if (data.value[i]->events_bitset.test(0)) + { + exist = true; + return i; + } break; - return k; + + case LAST_MATCH: + for (UInt64 i = 0; i < data.value.size(); ++i) + { + auto reversed_i = data.value.size() - i - 1; + if (data.value[reversed_i]->events_bitset.test(0)) + { + exist = true; + return reversed_i; + } + } + break; + } + + exist = false; + return 0; } /// This method returns an index of next node that matched the events. - /// It is one as referring Boyer-Moore-Algorithm(https://en.wikipedia.org/wiki/Boyer%E2%80%93Moore_string-search_algorithm). - /// But, there are some differences. - /// In original Boyer-Moore-Algorithm compares strings, but this algorithm compares events_bits. - /// Matched events in the chain of events are represented as a bitmask. + /// matched events in the chain of events are represented as a bitmask. /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { + const UInt32 unmatched = data.value.size(); + if (data.value.size() <= events_size) - return 0; + return unmatched; data.sort(); - UInt32 i = events_size - 1; - while (i < data.value.size()) + bool base_existence; + UInt32 base = getBaseIndex(data, base_existence); + if (!base_existence) + return unmatched; + + if (events_size == 0) { - UInt32 j = 0; - /// Try to match chain of events starting from the end of this chain. - for (; j < events_size; ++j) - { - /// It compares each matched events. - /// The lower bitmask is the former matched event. - if (!data.value[i - j]->events_bitset.test(events_size - 1 - j)) - break; - } - - /// Chain of events are matched, return the index of result value. - if (j == events_size) - return i + 1; - - i += calculateJump(data, i, j); + return data.value.size() > 0 ? base : unmatched; } + else + { + UInt32 i = 0; + switch (Direction) + { + case FORWARD: + for (i = 0; i < events_size && base + i < data.value.size(); ++i) + if (data.value[base + i]->events_bitset.test(i) == false) + break; + return (i == events_size) ? base + i : unmatched; - return 0; + case BACKWARD: + for (i = 0; i < events_size && i < base; ++i) + if (data.value[base - i]->events_bitset.test(i) == false) + break; + return (i == events_size) ? base - i : unmatched; + } + } } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override @@ -332,7 +388,7 @@ public: auto & value = data(place).value; UInt32 event_idx = getNextNodeIndex(this->data(place)); - if (event_idx != 0 && event_idx < value.size()) + if (event_idx < value.size()) { ColumnNullable & to_concrete = assert_cast(to); value[event_idx]->insertInto(to_concrete.getNestedColumn()); @@ -345,139 +401,4 @@ public: bool allocatesMemoryInArena() const override { return true; } }; -/// Implementation of sequenceFirstNode -template -class SequenceFirstNodeImpl final - : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> -{ - using Data = SequenceNextNodeGeneralData; - static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } - static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - - DataTypePtr & data_type; - -public: - explicit SequenceFirstNodeImpl(const DataTypePtr & data_type_) - : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( - {data_type_}, {}) - , data_type(this->argument_types[0]) - { - } - - String getName() const override { return "sequenceFirstNode"; } - - DataTypePtr getReturnType() const override { return data_type; } - - AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, - const AggregateFunctionProperties &) const override - { - return std::make_shared>(nested_function, arguments, params); - } - - void insert(Data & a, const Node * v, Arena * arena) const - { - ++a.total_values; - a.value.push_back(v->clone(arena), arena); - } - - void create(AggregateDataPtr __restrict place) const override - { - new (place) Data; - } - - bool compare(const T lhs_timestamp, const T rhs_timestamp) const - { - return Descending ? lhs_timestamp < rhs_timestamp : lhs_timestamp > rhs_timestamp; - } - - void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override - { - bool is_first = true; - auto & value = data(place).value; - const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; - - if (value.size() != 0) - { - if (compare(value[0]->event_time, timestamp)) - value.pop_back(); - else - is_first = false; - } - - if (is_first) - { - Node * node = Node::allocate(*columns[1], row_num, arena); - node->event_time = timestamp; - node->events_bitset.reset(); - - data(place).value.push_back(node, arena); - } - } - - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override - { - auto & a = data(place).value; - auto & b = data(rhs).value; - - if (b.empty()) - return; - - if (a.empty()) - { - a.push_back(b[0]->clone(arena), arena); - return; - } - - if (compare(a[0]->event_time, b[0]->event_time)) - { - data(place).value.pop_back(); - a.push_back(b[0]->clone(arena), arena); - } - } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override - { - writeBinary(data(place).sorted, buf); - - auto & value = data(place).value; - writeVarUInt(value.size(), buf); - for (auto & node : value) - node->write(buf); - } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override - { - readBinary(data(place).sorted, buf); - - UInt64 size; - readVarUInt(size, buf); - - if (unlikely(size == 0)) - return; - - auto & value = data(place).value; - - value.resize(size, arena); - for (UInt64 i = 0; i < size; ++i) - value[i] = Node::read(buf, arena); - } - - void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - { - auto & value = data(place).value; - - if (value.size() > 0) - { - ColumnNullable & to_concrete = assert_cast(to); - value[0]->insertInto(to_concrete.getNestedColumn()); - to_concrete.getNullMapData().push_back(0); - } - else - to.insertDefault(); - } - - bool allocatesMemoryInArena() const override { return true; } -}; - } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 94a55f2d954..9309641f3bf 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -1,316 +1,461 @@ -(0, A) 1 B -(0, A) 2 B -(0, A) 3 B -(0, A) 4 A -(0, A) 5 B -(0, A) 6 B -(0, B) 1 C -(0, B) 2 D -(0, B) 3 \N -(0, B) 4 C -(0, B) 5 A -(0, B) 6 A -(0, C) 1 D -(0, C) 2 \N -(0, C) 3 \N -(0, C) 4 \N -(0, C) 5 \N -(0, C) 6 \N -(0, D) 1 \N -(0, D) 2 C -(0, D) 3 \N -(0, D) 4 \N -(0, D) 5 \N -(0, D) 6 \N -(0, E) 1 \N -(0, E) 2 \N -(0, E) 3 \N -(0, E) 4 \N -(0, E) 5 \N -(0, E) 6 \N -(1, A) 1 \N -(1, A) 2 \N -(1, A) 3 \N -(1, A) 4 A -(1, A) 5 B -(1, A) 6 B -(1, B) 1 A -(1, B) 2 A -(1, B) 3 A -(1, B) 4 A -(1, B) 5 A -(1, B) 6 A -(1, C) 1 B -(1, C) 2 D -(1, C) 3 \N -(1, C) 4 B -(1, C) 5 A -(1, C) 6 B -(1, D) 1 C -(1, D) 2 B -(1, D) 3 \N -(1, D) 4 \N -(1, D) 5 \N -(1, D) 6 \N -(1, E) 1 \N -(1, E) 2 \N -(1, E) 3 \N -(1, E) 4 \N -(1, E) 5 \N -(1, E) 6 \N -(0, A->B) 1 C -(0, A->B) 2 D -(0, A->B) 3 \N -(0, A->B) 4 C -(0, A->B) 5 A -(0, A->B) 6 A -(0, A->C) 1 \N -(0, A->C) 2 \N -(0, A->C) 3 \N -(0, A->C) 4 \N -(0, A->C) 5 \N -(0, A->C) 6 \N -(0, B->A) 1 \N -(0, B->A) 2 \N -(0, B->A) 3 \N -(0, B->A) 4 \N -(0, B->A) 5 C -(0, B->A) 6 B -(1, A->B) 1 \N -(1, A->B) 2 \N -(1, A->B) 3 \N -(1, A->B) 4 \N -(1, A->B) 5 A -(1, A->B) 6 A -(1, A->C) 1 \N -(1, A->C) 2 \N -(1, A->C) 3 \N -(1, A->C) 4 \N -(1, A->C) 5 \N -(1, A->C) 6 \N -(1, B->A) 1 \N -(1, B->A) 2 \N -(1, B->A) 3 \N -(1, B->A) 4 A -(1, B->A) 5 \N -(1, B->A) 6 B -(0, A->A->B) 1 \N -(0, A->A->B) 2 \N -(0, A->A->B) 3 \N -(0, A->A->B) 4 C -(0, A->A->B) 5 \N -(0, A->A->B) 6 \N -(0, B->A->A) 1 \N -(0, B->A->A) 2 \N -(0, B->A->A) 3 \N -(0, B->A->A) 4 \N -(0, B->A->A) 5 \N -(0, B->A->A) 6 \N -(1, A->A->B) 1 \N -(1, A->A->B) 2 \N -(1, A->A->B) 3 \N -(1, A->A->B) 4 \N -(1, A->A->B) 5 \N -(1, A->A->B) 6 \N -(1, B->A->A) 1 \N -(1, B->A->A) 2 \N -(1, B->A->A) 3 \N -(1, B->A->A) 4 A -(1, B->A->A) 5 \N -(1, B->A->A) 6 \N -(0, A) id >= 10 10 B -(0, A) id >= 10 10 C -(0, A) id >= 10 10 D -(0, A) id >= 10 10 B -(0, A) id >= 10 10 B -(0, A) id >= 10 10 A +(forward, head, A) 1 B +(forward, head, A) 2 \N +(forward, head, A) 3 B +(forward, head, A) 4 A +(forward, head, A) 5 B +(forward, head, A) 6 B +(forward, head, B) 1 \N +(forward, head, B) 2 B +(forward, head, B) 3 \N +(forward, head, B) 4 \N +(forward, head, B) 5 \N +(forward, head, B) 6 \N +(forward, head, C) 1 \N +(forward, head, C) 2 \N +(forward, head, C) 3 \N +(forward, head, C) 4 \N +(forward, head, C) 5 \N +(forward, head, C) 6 \N +(forward, head, D) 1 \N +(forward, head, D) 2 \N +(forward, head, D) 3 \N +(forward, head, D) 4 \N +(forward, head, D) 5 \N +(forward, head, D) 6 \N +(forward, head, E) 1 \N +(forward, head, E) 2 \N +(forward, head, E) 3 \N +(forward, head, E) 4 \N +(forward, head, E) 5 \N +(forward, head, E) 6 \N +(backward, tail, A) 1 \N +(backward, tail, A) 2 \N +(backward, tail, A) 3 \N +(backward, tail, A) 4 \N +(backward, tail, A) 5 \N +(backward, tail, A) 6 \N +(backward, tail, B) 1 \N +(backward, tail, B) 2 \N +(backward, tail, B) 3 A +(backward, tail, B) 4 \N +(backward, tail, B) 5 \N +(backward, tail, B) 6 \N +(backward, tail, C) 1 \N +(backward, tail, C) 2 D +(backward, tail, C) 3 \N +(backward, tail, C) 4 B +(backward, tail, C) 5 A +(backward, tail, C) 6 B +(backward, tail, D) 1 C +(backward, tail, D) 2 \N +(backward, tail, D) 3 \N +(backward, tail, D) 4 \N +(backward, tail, D) 5 \N +(backward, tail, D) 6 \N +(backward, tail, E) 1 \N +(backward, tail, E) 2 \N +(backward, tail, E) 3 \N +(backward, tail, E) 4 \N +(backward, tail, E) 5 \N +(backward, tail, E) 6 \N +(forward, head, A->B) 1 C +(forward, head, A->B) 2 \N +(forward, head, A->B) 3 \N +(forward, head, A->B) 4 \N +(forward, head, A->B) 5 A +(forward, head, A->B) 6 A +(forward, head, A->C) 1 \N +(forward, head, A->C) 2 \N +(forward, head, A->C) 3 \N +(forward, head, A->C) 4 \N +(forward, head, A->C) 5 \N +(forward, head, A->C) 6 \N +(forward, head, B->A) 1 \N +(forward, head, B->A) 2 \N +(forward, head, B->A) 3 \N +(forward, head, B->A) 4 \N +(forward, head, B->A) 5 \N +(forward, head, B->A) 6 \N +(backward, tail, A->B) 1 \N +(backward, tail, A->B) 2 \N +(backward, tail, A->B) 3 \N +(backward, tail, A->B) 4 \N +(backward, tail, A->B) 5 \N +(backward, tail, A->B) 6 \N +(backward, tail, A->C) 1 \N +(backward, tail, A->C) 2 \N +(backward, tail, A->C) 3 \N +(backward, tail, A->C) 4 \N +(backward, tail, A->C) 5 \N +(backward, tail, A->C) 6 \N +(backward, tail, B->A) 1 \N +(backward, tail, B->A) 2 \N +(backward, tail, B->A) 3 \N +(backward, tail, B->A) 4 \N +(backward, tail, B->A) 5 \N +(backward, tail, B->A) 6 \N +(forward, head, A->A->B) 1 \N +(forward, head, A->A->B) 2 \N +(forward, head, A->A->B) 3 \N +(forward, head, A->A->B) 4 \N +(forward, head, A->A->B) 5 \N +(forward, head, A->A->B) 6 \N +(forward, head, B->A->A) 1 \N +(forward, head, B->A->A) 2 \N +(forward, head, B->A->A) 3 \N +(forward, head, B->A->A) 4 \N +(forward, head, B->A->A) 5 \N +(forward, head, B->A->A) 6 \N +(backward, tail, A->A->B) 1 \N +(backward, tail, A->A->B) 2 \N +(backward, tail, A->A->B) 3 \N +(backward, tail, A->A->B) 4 \N +(backward, tail, A->A->B) 5 \N +(backward, tail, A->A->B) 6 \N +(backward, tail, B->A->A) 1 \N +(backward, tail, B->A->A) 2 \N +(backward, tail, B->A->A) 3 \N +(backward, tail, B->A->A) 4 \N +(backward, tail, B->A->A) 5 \N +(backward, tail, B->A->A) 6 \N +(forward, head, A) id >= 10 10 B +(forward, head, A) id >= 10 10 C +(forward, head, A) id >= 10 10 \N +(forward, head, A) id >= 10 10 \N +(backward, tail, A) id >= 10 10 B +(backward, tail, A) id >= 10 10 A (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 +(0, C) id = 11 0 +(0, B->C) id = 11 0 (0, A->B->C) id = 11 1 (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 -(0, B->C->D) id = 11 1 -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C -(0, A) 1 B -(0, A) 2 B -(0, A) 3 B -(0, A) 4 A -(0, A) 5 B -(0, A) 6 B -(0, B) 1 C -(0, B) 2 D -(0, B) 3 \N -(0, B) 4 C -(0, B) 5 A -(0, B) 6 A -(0, C) 1 D -(0, C) 2 \N -(0, C) 3 \N -(0, C) 4 \N -(0, C) 5 \N -(0, C) 6 \N -(0, D) 1 \N -(0, D) 2 C -(0, D) 3 \N -(0, D) 4 \N -(0, D) 5 \N -(0, D) 6 \N -(0, E) 1 \N -(0, E) 2 \N -(0, E) 3 \N -(0, E) 4 \N -(0, E) 5 \N -(0, E) 6 \N -(1, A) 1 \N -(1, A) 2 \N -(1, A) 3 \N -(1, A) 4 A -(1, A) 5 B -(1, A) 6 B -(1, B) 1 A -(1, B) 2 A -(1, B) 3 A -(1, B) 4 A -(1, B) 5 A -(1, B) 6 A -(1, C) 1 B -(1, C) 2 D -(1, C) 3 \N -(1, C) 4 B -(1, C) 5 A -(1, C) 6 B -(1, D) 1 C -(1, D) 2 B -(1, D) 3 \N -(1, D) 4 \N -(1, D) 5 \N -(1, D) 6 \N -(1, E) 1 \N -(1, E) 2 \N -(1, E) 3 \N -(1, E) 4 \N -(1, E) 5 \N -(1, E) 6 \N -(0, A->B) 1 C -(0, A->B) 2 D -(0, A->B) 3 \N -(0, A->B) 4 C -(0, A->B) 5 A -(0, A->B) 6 A -(0, A->C) 1 \N -(0, A->C) 2 \N -(0, A->C) 3 \N -(0, A->C) 4 \N -(0, A->C) 5 \N -(0, A->C) 6 \N -(0, B->A) 1 \N -(0, B->A) 2 \N -(0, B->A) 3 \N -(0, B->A) 4 \N -(0, B->A) 5 C -(0, B->A) 6 B -(1, A->B) 1 \N -(1, A->B) 2 \N -(1, A->B) 3 \N -(1, A->B) 4 \N -(1, A->B) 5 A -(1, A->B) 6 A -(1, A->C) 1 \N -(1, A->C) 2 \N -(1, A->C) 3 \N -(1, A->C) 4 \N -(1, A->C) 5 \N -(1, A->C) 6 \N -(1, B->A) 1 \N -(1, B->A) 2 \N -(1, B->A) 3 \N -(1, B->A) 4 A -(1, B->A) 5 \N -(1, B->A) 6 B -(0, A->A->B) 1 \N -(0, A->A->B) 2 \N -(0, A->A->B) 3 \N -(0, A->A->B) 4 C -(0, A->A->B) 5 \N -(0, A->A->B) 6 \N -(0, B->A->A) 1 \N -(0, B->A->A) 2 \N -(0, B->A->A) 3 \N -(0, B->A->A) 4 \N -(0, B->A->A) 5 \N -(0, B->A->A) 6 \N -(1, A->A->B) 1 \N -(1, A->A->B) 2 \N -(1, A->A->B) 3 \N -(1, A->A->B) 4 \N -(1, A->A->B) 5 \N -(1, A->A->B) 6 \N -(1, B->A->A) 1 \N -(1, B->A->A) 2 \N -(1, B->A->A) 3 \N -(1, B->A->A) 4 A -(1, B->A->A) 5 \N -(1, B->A->A) 6 \N -(0, A) id = 10 10 B -(0, A->B) id = 10 10 C -(0, B->C) id = 10 10 D -(0, C) id = 10 10 B -(0, D->C) id = 10 10 B -(0, C->B) id = 10 10 A +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 1 +(forward, head) id < 10 1 A +(forward, head) id < 10 2 B +(forward, head) id < 10 3 A +(forward, head) id < 10 4 A +(forward, head) id < 10 5 A +(forward, head) id < 10 6 A +(backward, tail) id < 10 1 D +(backward, tail) id < 10 2 C +(backward, tail) id < 10 3 B +(backward, tail) id < 10 4 C +(backward, tail) id < 10 5 C +(backward, tail) id < 10 6 C +(forward, first_match, A) 1 B +(forward, first_match, A) 2 \N +(forward, first_match, A) 3 B +(forward, first_match, A) 4 A +(forward, first_match, A) 5 B +(forward, first_match, A) 6 B +(forward, first_match, A) 10 B +(forward, first_match, A) 11 B +(forward, first_match, A->B) 1 C +(forward, first_match, A->B) 2 \N +(forward, first_match, A->B) 3 \N +(forward, first_match, A->B) 4 \N +(forward, first_match, A->B) 5 A +(forward, first_match, A->B) 6 A +(forward, first_match, A->B) 10 C +(forward, first_match, A->B) 11 C +(forward, first_match, A->B->C) 1 D +(forward, first_match, A->B->C) 2 \N +(forward, first_match, A->B->C) 3 \N +(forward, first_match, A->B->C) 4 \N +(forward, first_match, A->B->C) 5 \N +(forward, first_match, A->B->C) 6 \N +(forward, first_match, A->B->C) 10 D +(forward, first_match, A->B->C) 11 D +(forward, first_match, B) 1 C +(forward, first_match, B) 2 B +(forward, first_match, B) 3 \N +(forward, first_match, B) 4 C +(forward, first_match, B) 5 A +(forward, first_match, B) 6 A +(forward, first_match, B) 10 C +(forward, first_match, B) 11 C +(forward, first_match, B->B) 1 \N +(forward, first_match, B->B) 2 D +(forward, first_match, B->B) 3 \N +(forward, first_match, B->B) 4 \N +(forward, first_match, B->B) 5 \N +(forward, first_match, B->B) 6 \N +(forward, first_match, B->B) 10 \N +(forward, first_match, B->B) 11 \N +(forward, first_match, B->A) 1 \N +(forward, first_match, B->A) 2 \N +(forward, first_match, B->A) 3 \N +(forward, first_match, B->A) 4 \N +(forward, first_match, B->A) 5 C +(forward, first_match, B->A) 6 B +(forward, first_match, B->A) 10 \N +(forward, first_match, B->A) 11 \N +(backward, first_match, A) 1 \N +(backward, first_match, A) 2 \N +(backward, first_match, A) 3 \N +(backward, first_match, A) 4 \N +(backward, first_match, A) 5 \N +(backward, first_match, A) 6 \N +(backward, first_match, A) 10 \N +(backward, first_match, A) 11 \N +(backward, first_match, B) 1 A +(backward, first_match, B) 2 \N +(backward, first_match, B) 3 A +(backward, first_match, B) 4 A +(backward, first_match, B) 5 A +(backward, first_match, B) 6 A +(backward, first_match, B) 10 A +(backward, first_match, B) 11 A +(backward, first_match, B->A) 1 \N +(backward, first_match, B->A) 2 \N +(backward, first_match, B->A) 3 \N +(backward, first_match, B->A) 4 A +(backward, first_match, B->A) 5 \N +(backward, first_match, B->A) 6 \N +(backward, first_match, B->A) 10 \N +(backward, first_match, B->A) 11 \N +(backward, first_match, B->B) 1 \N +(backward, first_match, B->B) 2 \N +(backward, first_match, B->B) 3 \N +(backward, first_match, B->B) 4 \N +(backward, first_match, B->B) 5 \N +(backward, first_match, B->B) 6 \N +(backward, first_match, B->B) 10 \N +(backward, first_match, B->B) 11 \N +(forward, head, A) 1 B +(forward, head, A) 2 \N +(forward, head, A) 3 B +(forward, head, A) 4 A +(forward, head, A) 5 B +(forward, head, A) 6 B +(forward, head, B) 1 \N +(forward, head, B) 2 B +(forward, head, B) 3 \N +(forward, head, B) 4 \N +(forward, head, B) 5 \N +(forward, head, B) 6 \N +(forward, head, C) 1 \N +(forward, head, C) 2 \N +(forward, head, C) 3 \N +(forward, head, C) 4 \N +(forward, head, C) 5 \N +(forward, head, C) 6 \N +(forward, head, D) 1 \N +(forward, head, D) 2 \N +(forward, head, D) 3 \N +(forward, head, D) 4 \N +(forward, head, D) 5 \N +(forward, head, D) 6 \N +(forward, head, E) 1 \N +(forward, head, E) 2 \N +(forward, head, E) 3 \N +(forward, head, E) 4 \N +(forward, head, E) 5 \N +(forward, head, E) 6 \N +(backward, tail, A) 1 \N +(backward, tail, A) 2 \N +(backward, tail, A) 3 \N +(backward, tail, A) 4 \N +(backward, tail, A) 5 \N +(backward, tail, A) 6 \N +(backward, tail, B) 1 \N +(backward, tail, B) 2 \N +(backward, tail, B) 3 A +(backward, tail, B) 4 \N +(backward, tail, B) 5 \N +(backward, tail, B) 6 \N +(backward, tail, C) 1 \N +(backward, tail, C) 2 D +(backward, tail, C) 3 \N +(backward, tail, C) 4 B +(backward, tail, C) 5 A +(backward, tail, C) 6 B +(backward, tail, D) 1 C +(backward, tail, D) 2 \N +(backward, tail, D) 3 \N +(backward, tail, D) 4 \N +(backward, tail, D) 5 \N +(backward, tail, D) 6 \N +(backward, tail, E) 1 \N +(backward, tail, E) 2 \N +(backward, tail, E) 3 \N +(backward, tail, E) 4 \N +(backward, tail, E) 5 \N +(backward, tail, E) 6 \N +(forward, head, A->B) 1 C +(forward, head, A->B) 2 \N +(forward, head, A->B) 3 \N +(forward, head, A->B) 4 \N +(forward, head, A->B) 5 A +(forward, head, A->B) 6 A +(forward, head, A->C) 1 \N +(forward, head, A->C) 2 \N +(forward, head, A->C) 3 \N +(forward, head, A->C) 4 \N +(forward, head, A->C) 5 \N +(forward, head, A->C) 6 \N +(forward, head, B->A) 1 \N +(forward, head, B->A) 2 \N +(forward, head, B->A) 3 \N +(forward, head, B->A) 4 \N +(forward, head, B->A) 5 \N +(forward, head, B->A) 6 \N +(backward, tail, A->B) 1 \N +(backward, tail, A->B) 2 \N +(backward, tail, A->B) 3 \N +(backward, tail, A->B) 4 \N +(backward, tail, A->B) 5 \N +(backward, tail, A->B) 6 \N +(backward, tail, A->C) 1 \N +(backward, tail, A->C) 2 \N +(backward, tail, A->C) 3 \N +(backward, tail, A->C) 4 \N +(backward, tail, A->C) 5 \N +(backward, tail, A->C) 6 \N +(backward, tail, B->A) 1 \N +(backward, tail, B->A) 2 \N +(backward, tail, B->A) 3 \N +(backward, tail, B->A) 4 \N +(backward, tail, B->A) 5 \N +(backward, tail, B->A) 6 \N +(forward, head, A->A->B) 1 \N +(forward, head, A->A->B) 2 \N +(forward, head, A->A->B) 3 \N +(forward, head, A->A->B) 4 \N +(forward, head, A->A->B) 5 \N +(forward, head, A->A->B) 6 \N +(forward, head, B->A->A) 1 \N +(forward, head, B->A->A) 2 \N +(forward, head, B->A->A) 3 \N +(forward, head, B->A->A) 4 \N +(forward, head, B->A->A) 5 \N +(forward, head, B->A->A) 6 \N +(backward, tail, A->A->B) 1 \N +(backward, tail, A->A->B) 2 \N +(backward, tail, A->A->B) 3 \N +(backward, tail, A->A->B) 4 \N +(backward, tail, A->A->B) 5 \N +(backward, tail, A->A->B) 6 \N +(backward, tail, B->A->A) 1 \N +(backward, tail, B->A->A) 2 \N +(backward, tail, B->A->A) 3 \N +(backward, tail, B->A->A) 4 \N +(backward, tail, B->A->A) 5 \N +(backward, tail, B->A->A) 6 \N +(forward, head, A) id >= 10 10 B +(forward, head, A) id >= 10 10 C +(forward, head, A) id >= 10 10 \N +(forward, head, A) id >= 10 10 \N +(backward, tail, A) id >= 10 10 B +(backward, tail, A) id >= 10 10 A (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 +(0, C) id = 11 0 +(0, B->C) id = 11 0 (0, A->B->C) id = 11 1 (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 -(0, B->C->D) id = 11 1 -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 1 +(forward, head) id < 10 1 A +(forward, head) id < 10 2 B +(forward, head) id < 10 3 A +(forward, head) id < 10 4 A +(forward, head) id < 10 5 A +(forward, head) id < 10 6 A +(backward, tail) id < 10 1 D +(backward, tail) id < 10 2 C +(backward, tail) id < 10 3 B +(backward, tail) id < 10 4 C +(backward, tail) id < 10 5 C +(backward, tail) id < 10 6 C +(forward, first_match, A) 1 B +(forward, first_match, A) 2 \N +(forward, first_match, A) 3 B +(forward, first_match, A) 4 A +(forward, first_match, A) 5 B +(forward, first_match, A) 6 B +(forward, first_match, A) 10 B +(forward, first_match, A) 11 B +(forward, first_match, A->B) 1 C +(forward, first_match, A->B) 2 \N +(forward, first_match, A->B) 3 \N +(forward, first_match, A->B) 4 \N +(forward, first_match, A->B) 5 A +(forward, first_match, A->B) 6 A +(forward, first_match, A->B) 10 C +(forward, first_match, A->B) 11 C +(forward, first_match, A->B->C) 1 D +(forward, first_match, A->B->C) 2 \N +(forward, first_match, A->B->C) 3 \N +(forward, first_match, A->B->C) 4 \N +(forward, first_match, A->B->C) 5 \N +(forward, first_match, A->B->C) 6 \N +(forward, first_match, A->B->C) 10 D +(forward, first_match, A->B->C) 11 D +(forward, first_match, B) 1 C +(forward, first_match, B) 2 B +(forward, first_match, B) 3 \N +(forward, first_match, B) 4 C +(forward, first_match, B) 5 A +(forward, first_match, B) 6 A +(forward, first_match, B) 10 C +(forward, first_match, B) 11 C +(forward, first_match, B->B) 1 \N +(forward, first_match, B->B) 2 D +(forward, first_match, B->B) 3 \N +(forward, first_match, B->B) 4 \N +(forward, first_match, B->B) 5 \N +(forward, first_match, B->B) 6 \N +(forward, first_match, B->B) 10 \N +(forward, first_match, B->B) 11 \N +(forward, first_match, B->A) 1 \N +(forward, first_match, B->A) 2 \N +(forward, first_match, B->A) 3 \N +(forward, first_match, B->A) 4 \N +(forward, first_match, B->A) 5 C +(forward, first_match, B->A) 6 B +(forward, first_match, B->A) 10 \N +(forward, first_match, B->A) 11 \N +(backward, first_match, A) 1 \N +(backward, first_match, A) 2 \N +(backward, first_match, A) 3 \N +(backward, first_match, A) 4 \N +(backward, first_match, A) 5 \N +(backward, first_match, A) 6 \N +(backward, first_match, A) 10 \N +(backward, first_match, A) 11 \N +(backward, first_match, B) 1 A +(backward, first_match, B) 2 \N +(backward, first_match, B) 3 A +(backward, first_match, B) 4 A +(backward, first_match, B) 5 A +(backward, first_match, B) 6 A +(backward, first_match, B) 10 A +(backward, first_match, B) 11 A +(backward, first_match, B->A) 1 \N +(backward, first_match, B->A) 2 \N +(backward, first_match, B->A) 3 \N +(backward, first_match, B->A) 4 A +(backward, first_match, B->A) 5 \N +(backward, first_match, B->A) 6 \N +(backward, first_match, B->A) 10 \N +(backward, first_match, B->A) 11 \N +(backward, first_match, B->B) 1 \N +(backward, first_match, B->B) 2 \N +(backward, first_match, B->B) 3 \N +(backward, first_match, B->B) 4 \N +(backward, first_match, B->B) 5 \N +(backward, first_match, B->B) 6 \N +(backward, first_match, B->B) 10 \N +(backward, first_match, B->B) 11 \N +(max_args) 1 \N +(max_args) 2 \N +(max_args) 3 \N +(max_args) 4 \N +(max_args) 5 \N +(max_args) 6 \N +(max_args) 10 \N +(max_args) 11 \N +(forward, head, A) id = 12 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 57b0d7b1d8e..63810265bf9 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -6,7 +6,7 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',1,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',1,'D'); -INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',2,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',2,'D'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',2,'C'); @@ -27,26 +27,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); -SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); @@ -54,31 +54,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; @@ -86,13 +96,13 @@ DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; DROP TABLE IF EXISTS test_sequenceNextNode; -CREATE TABLE iF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; +CREATE TABLE IF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',1,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',1,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',1,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',1,'D'); -INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',2,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',2,'D'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',2,'C'); @@ -113,56 +123,73 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(0, A) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, A->B) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, B->C) id = 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, D->C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, C->B) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); + +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; DROP TABLE IF EXISTS test_sequenceNextNode; From 34ccad75dcd0cb1180388a75174901da3c5d58c2 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Wed, 10 Mar 2021 21:14:30 +0900 Subject: [PATCH 057/652] Remove insertion_requires_nullable_column --- src/AggregateFunctions/AggregateFunctionNull.h | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 8e638742162..821398e715e 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -228,15 +228,15 @@ public: }; -template +template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> + AggregateFunctionNullVariadic> { public: AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), + AggregateFunctionNullVariadic>(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) @@ -283,15 +283,8 @@ public: ColumnNullable & to_concrete = assert_cast(to); if (this->getFlag(place)) { - if constexpr (insertion_requires_nullable_column) - { - this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete, arena); - } - else - { - this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); - } + this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); } else { From 1ec927e7666391f8df9e962633f11e19510fb26f Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 11 Mar 2021 12:48:24 +0900 Subject: [PATCH 058/652] Fix special-build-failure and add argument validation --- .../AggregateFunctionSequenceNextNode.cpp | 16 ++++++++++------ .../AggregateFunctionSequenceNextNode.h | 6 +++--- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 274b75b383e..b8f517cba20 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -50,24 +50,28 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c { assert(max_args <= MAX_EVENTS_SIZE); + if (parameters.size() < 2) + throw Exception("Aggregate function " + name + " requires 2 parameters (direction, head)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + String param_dir = parameters.at(0).safeGet(); SeqDirection direction; - if (param_dir.compare("forward") == 0) + if (param_dir == "forward") direction = FORWARD; - else if (param_dir.compare("backward") == 0) + else if (param_dir == "backward") direction = BACKWARD; else throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS}; String param_base = parameters.at(1).safeGet(); SeqBase base; - if (param_base.compare("head") == 0) + if (param_base == "head") base = HEAD; - else if (param_base.compare("tail") == 0) + else if (param_base == "tail") base = TAIL; - else if (param_base.compare("first_match") == 0) + else if (param_base == "first_match") base = FIRST_MATCH; - else if (param_base.compare("last_match") == 0) + else if (param_base == "last_match") base = LAST_MATCH; else throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 78d19beedc4..f22ca1d61a6 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -263,18 +263,18 @@ public: auto & value = data(place).value; - UInt64 size = std::max(static_cast(events_size + 1), value.size()); + size_t size = std::max(static_cast(events_size + 1), value.size()); switch (Base) { case HEAD: writeVarUInt(size, buf); - for (UInt64 i = 0; i < size; ++i) + for (size_t i = 0; i < size; ++i) value[i]->write(buf); break; case TAIL: writeVarUInt(size, buf); - for (UInt64 i = value.size() - 1; i >= size; --i) + for (size_t i = value.size() - 1; i >= size; --i) value[i]->write(buf); break; From bf01eeb8839b78239bc84891bf3a0a483d4bfd81 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 11 Mar 2021 16:44:02 +0900 Subject: [PATCH 059/652] Fix a bug in sequenceNextNode --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index f22ca1d61a6..f58469f27bb 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -263,7 +263,7 @@ public: auto & value = data(place).value; - size_t size = std::max(static_cast(events_size + 1), value.size()); + size_t size = std::min(static_cast(events_size + 1), value.size()); switch (Base) { case HEAD: From 06bb4e2c7171f775cb1443cfee6441b989161bdd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Mar 2021 13:00:49 +0300 Subject: [PATCH 060/652] wip --- tests/integration/helpers/cluster.py | 15 +++++----- tests/integration/helpers/hdfs_api.py | 12 ++++---- tests/integration/pytest.ini | 2 +- tests/integration/runner | 2 +- tests/integration/test_MemoryTracking/test.py | 3 -- .../test_access_control_on_cluster/test.py | 2 +- .../test_adaptive_granularity/test.py | 13 +++++---- .../test.py | 4 +-- .../test_alter_on_mixed_type_cluster/test.py | 2 +- tests/integration/test_check_table/test.py | 8 +++--- .../test_compression_codec_read/test.py | 2 ++ .../integration/test_dictionaries_ddl/test.py | 4 +-- .../test_dictionaries_mysql/test.py | 14 +++++----- .../configs/dictionaries/.gitignore | 2 ++ .../test_default_reading.py | 2 +- .../test_default_string.py | 3 +- .../test_dict_get_or_default.py | 2 +- .../test_disabled_mysql_server/test.py | 8 +++--- .../test.py | 2 +- .../test_format_avro_confluent/test.py | 4 --- .../test_insert_into_distributed/test.py | 2 +- .../test_join_set_family_s3/test.py | 3 -- tests/integration/test_log_family_s3/test.py | 3 -- tests/integration/test_merge_tree_s3/test.py | 3 -- .../test_merge_tree_s3_failover/test.py | 3 -- .../test_merge_tree_s3_restore/test.py | 3 -- .../test_merge_tree_s3_with_cache/test.py | 3 -- .../test_postgresql_protocol/test.py | 2 +- .../test_profile_events_s3/test.py | 3 -- tests/integration/test_quota/test.py | 2 +- .../test_replicated_database/test.py | 28 +++++++++---------- .../test_replicated_merge_tree_s3/test.py | 3 -- tests/integration/test_s3_with_https/test.py | 3 -- tests/integration/test_s3_with_proxy/test.py | 4 --- tests/integration/test_storage_mysql/test.py | 2 +- .../test_storage_postgresql/test.py | 1 + tests/integration/test_storage_s3/test.py | 3 -- .../test_storage_s3/test_redirect.py | 3 -- 38 files changed, 72 insertions(+), 108 deletions(-) create mode 100644 tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index eb6ecc8a39f..124f77061cb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -579,7 +579,7 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - start_deadline = time.time() + 60.0 # seconds + start_deadline = time.time() + 120.0 # seconds node.wait_for_start(start_deadline) return node @@ -916,13 +916,15 @@ class ClickHouseCluster: if self.with_hdfs and self.base_hdfs_cmd: logging.debug('Setup HDFS') os.makedirs(self.hdfs_logs_dir) + os.chmod(self.hdfs_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api() - self.wait_hdfs_to_start(hdfs_api, 120) + self.wait_hdfs_to_start(hdfs_api, 300) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') os.makedirs(self.hdfs_kerberized_logs_dir) + os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(hdfs_api, timeout=300) @@ -964,7 +966,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 60.0 # seconds + start_deadline = time.time() + 120.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -998,17 +1000,16 @@ class ClickHouseCluster: sanitizer_assert_instance = line.split('|')[0].strip() break - for instance in list(self.instances.values()): + for name, instance in self.instances.items(): try: if not instance.is_up: continue if instance.contains_in_log(SANITIZER_SIGN): sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) - logging.ERROR(f"Sanitizer in instance {instance.name} log {sanitizer_assert_instance}") + logging.ERROR(f"Sanitizer in instance {name} log {sanitizer_assert_instance}") if instance.contains_in_log("Fatal"): fatal_log = instance.grep_in_log("Fatal") - name = instance.name logging.ERROR(f"Crash in instance {name} fatal log {fatal_log}") except Exception as e: logging.error(f"Failed to check fails in logs: {e}") @@ -1324,7 +1325,7 @@ class ClickHouseInstance: from helpers.test_tools import assert_eq_with_retry assert_eq_with_retry(self, "select 1", "1", retry_count=int(stop_wait_sec / 0.5), sleep_time=0.5) - def restart_clickhouse(self, stop_start_wait_sec=5, kill=False): + def restart_clickhouse(self, stop_start_wait_sec=15, kill=False): self.stop_clickhouse(stop_start_wait_sec, kill) self.start_clickhouse(stop_start_wait_sec) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 8a69f6baffd..5b1b4c402da 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -47,12 +47,12 @@ class HDFSApi(object): # logging.basicConfig(level=logging.DEBUG) # logging.getLogger().setLevel(logging.DEBUG) - requests_log = logging.getLogger("requests.packages.urllib3") - requests_log.setLevel(logging.DEBUG) - requests_log.propagate = True - kerb_log = logging.getLogger("requests_kerberos") - kerb_log.setLevel(logging.DEBUG) - kerb_log.propagate = True + # requests_log = logging.getLogger("requests.packages.urllib3") + # requests_log.setLevel(logging.INFO) + # requests_log.propagate = True + # kerb_log = logging.getLogger("requests_kerberos") + # kerb_log.setLevel(logging.DEBUG) + # kerb_log.propagate = True if kerberized: self._run_kinit() diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index 7d4a3ad1c29..47086d45b3d 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -6,7 +6,7 @@ junit_duration_report = call junit_suite_name = integration log_cli = 1 log_cli_level = CRITICAL -log_cli_format = %(message)s +log_cli_format = %%(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) log_file = pytest.log log_file_level = DEBUG log_file_format = %(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) diff --git a/tests/integration/runner b/tests/integration/runner index f3309f983a3..3aa3aa5ef64 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -199,7 +199,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=180 \ + --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=300 \ {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, diff --git a/tests/integration/test_MemoryTracking/test.py b/tests/integration/test_MemoryTracking/test.py index a0ad8dc519d..1ee528eaa8d 100644 --- a/tests/integration/test_MemoryTracking/test.py +++ b/tests/integration/test_MemoryTracking/test.py @@ -13,9 +13,6 @@ node = cluster.add_instance('node', main_configs=[ 'configs/asynchronous_metrics_update_period_s.xml', ]) -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope='module', autouse=True) def start_cluster(): try: diff --git a/tests/integration/test_access_control_on_cluster/test.py b/tests/integration/test_access_control_on_cluster/test.py index bc740402161..6bcf67779ef 100644 --- a/tests/integration/test_access_control_on_cluster/test.py +++ b/tests/integration/test_access_control_on_cluster/test.py @@ -18,7 +18,7 @@ def started_cluster(): def test_access_control_on_cluster(): - ch1.query_with_retry("CREATE USER Alex ON CLUSTER 'cluster'", retry_count=3) + ch1.query_with_retry("CREATE USER IF NOT EXISTS Alex ON CLUSTER 'cluster'", retry_count=5) assert ch1.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n" assert ch2.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n" assert ch3.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n" diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 47209908e5c..2ea2f29212f 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -1,5 +1,6 @@ import time +import logging import pytest from helpers.client import QueryRuntimeException, QueryTimeoutExceedException from helpers.cluster import ClickHouseCluster @@ -268,16 +269,18 @@ def test_version_single_node_update(start_dynamic_cluster, n, tables): ] ) def test_mixed_granularity_single_node(start_dynamic_cluster, node): - assert node.name == "node9" or node.name == "node10" - assert_eq_with_retry(node, - "SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'", - '0') # check that enable_mixed_granularity_parts is off by default - node.query( "INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)") node.query( "INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)") + path_to_part = node.query( + "SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition DESC LIMIT 1").strip() + + result = node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk*'".format( + p=path_to_part)]) # check that we have non adaptive files + logging.info(f"path {path_to_part} result\n {result}") + def callback(n): new_config = """ diff --git a/tests/integration/test_adaptive_granularity_different_settings/test.py b/tests/integration/test_adaptive_granularity_different_settings/test.py index ce8f32b6ec6..f78a0ef2da6 100644 --- a/tests/integration/test_adaptive_granularity_different_settings/test.py +++ b/tests/integration/test_adaptive_granularity_different_settings/test.py @@ -37,8 +37,8 @@ def test_attach_detach(start_cluster): node2.query("INSERT INTO test VALUES (3), (4)") - node1.query("SYSTEM SYNC REPLICA test", timeout=10) - node2.query("SYSTEM SYNC REPLICA test", timeout=10) + node1.query_with_retry("SYSTEM SYNC REPLICA test", timeout=10) + node2.query_with_retry("SYSTEM SYNC REPLICA test", timeout=10) assert node1.query("SELECT COUNT() FROM test") == "4\n" assert node2.query("SELECT COUNT() FROM test") == "4\n" diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 5e516176c0b..852554f009d 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -78,7 +78,7 @@ def test_alter_replicated_on_cluster(started_cluster): assert node3.query("SELECT date FROM test_table_replicated") == '2019-10-01 00:00:00\n' assert node4.query("SELECT date FROM test_table_replicated") == '2019-10-01 00:00:00\n' - node3.query("ALTER TABLE test_table_replicated ON CLUSTER 'test_cluster_mixed' MODIFY COLUMN value String", settings={"replication_alter_partitions_sync": "2"}) + node3.query_with_retry("ALTER TABLE test_table_replicated ON CLUSTER 'test_cluster_mixed' MODIFY COLUMN value String", settings={"replication_alter_partitions_sync": "2"}) for node in [node2, node4]: node.query("INSERT INTO test_table_replicated VALUES(toDateTime('2019-10-02 00:00:00'), 2, 'Hello')") diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index d204f6c5810..b184813d24f 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -122,9 +122,9 @@ def test_check_replicated_table_simple(started_cluster): def test_check_replicated_table_corruption(started_cluster): for node in [node1, node2]: - node.query("DROP TABLE IF EXISTS replicated_mt_1") + node.query_with_retry("DROP TABLE IF EXISTS replicated_mt_1") - node.query(''' + node.query_with_retry(''' CREATE TABLE replicated_mt_1(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt_1', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; '''.format(replica=node.name)) @@ -136,7 +136,7 @@ def test_check_replicated_table_corruption(started_cluster): assert node1.query("SELECT count() from replicated_mt_1") == "4\n" assert node2.query("SELECT count() from replicated_mt_1") == "4\n" - part_name = node1.query( + part_name = node1.query_with_retry( "SELECT name from system.parts where table = 'replicated_mt_1' and partition_id = '201901' and active = 1").strip() corrupt_data_part_on_disk(node1, "replicated_mt_1", part_name) @@ -144,7 +144,7 @@ def test_check_replicated_table_corruption(started_cluster): "check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format( p=part_name) - node1.query("SYSTEM SYNC REPLICA replicated_mt_1") + node1.query_with_retry("SYSTEM SYNC REPLICA replicated_mt_1") assert node1.query("CHECK TABLE replicated_mt_1 PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" diff --git a/tests/integration/test_compression_codec_read/test.py b/tests/integration/test_compression_codec_read/test.py index 0eb1f5aa867..35ae60f05ea 100644 --- a/tests/integration/test_compression_codec_read/test.py +++ b/tests/integration/test_compression_codec_read/test.py @@ -17,6 +17,8 @@ def start_cluster(): cluster.shutdown() def test_default_codec_read(start_cluster): + node1.query("DROP TABLE IF EXISTS test_18340") + node1.query(""" CREATE TABLE test_18340 ( diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 59ccc59d0af..3aa76b4af88 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -66,7 +66,7 @@ def started_cluster(): (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) + mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE DATABASE clickhouse") execute_mysql_query(mysql_conn, @@ -245,7 +245,7 @@ def test_file_dictionary_restrictions(started_cluster): def test_dictionary_with_where(started_cluster): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) + mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))") diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 90cfe53dd68..c18f3d6cf70 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -31,7 +31,7 @@ def started_cluster(): cluster.start() # Create a MySQL database - mysql_connection = get_mysql_conn() + mysql_connection = get_mysql_conn(cluster) create_mysql_db(mysql_connection, 'test') mysql_connection.close() @@ -54,7 +54,7 @@ def test_load_mysql_dictionaries(started_cluster): for n in range(0, 5): # Create MySQL tables, fill them and create CH dict tables - prepare_mysql_table('test', str(n)) + prepare_mysql_table(started_cluster, 'test', str(n)) # Check dictionaries are loaded and have correct number of elements for n in range(0, 100): @@ -72,8 +72,8 @@ def create_mysql_db(mysql_connection, name): cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) -def prepare_mysql_table(table_name, index): - mysql_connection = get_mysql_conn() +def prepare_mysql_table(started_cluster, table_name, index): + mysql_connection = get_mysql_conn(started_cluster) # Create table create_mysql_table(mysql_connection, table_name + str(index)) @@ -89,16 +89,16 @@ def prepare_mysql_table(table_name, index): # Create CH Dictionary tables based on MySQL tables query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index))) -def get_mysql_conn(): +def get_mysql_conn(started_cluster): errors = [] conn = None for _ in range(5): try: if conn is None: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=started_cluster.mysql_ip, port=started_cluster.mysql_port) else: conn.ping(reconnect=True) - logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + logging.debug(f"MySQL Connection establised: {started_cluster.mysql_ip}:{started_cluster.mysql_port}") return conn except Exception as e: errors += [str(e)] diff --git a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore new file mode 100644 index 00000000000..d6b7ef32c84 --- /dev/null +++ b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 0c801ce3f12..46820fb4a89 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -7,7 +7,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="reading") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index caabdf12c66..12cde30d8ed 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -9,8 +9,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="string") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index e794ffa5a37..789d4aeb8b3 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -7,7 +7,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="default") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_disabled_mysql_server/test.py b/tests/integration/test_disabled_mysql_server/test.py index 2fc84ee74a5..d7977404c73 100644 --- a/tests/integration/test_disabled_mysql_server/test.py +++ b/tests/integration/test_disabled_mysql_server/test.py @@ -22,10 +22,10 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): + def __init__(self, started_cluster, user='root', password='clickhouse'): self.user = user - self.port = port - self.hostname = hostname + self.port = cluster.mysql_port + self.hostname = cluster.mysql_ip self.password = password self.mysql_connection = None # lazy init @@ -45,7 +45,7 @@ class MySQLNodeInstance: def test_disabled_mysql_server(started_cluster): - with contextlib.closing(MySQLNodeInstance()) as mysql_node: + with contextlib.closing(MySQLNodeInstance(started_cluster)) as mysql_node: mysql_node.query("DROP DATABASE IF EXISTS test_db_disabled;") mysql_node.query("CREATE DATABASE test_db_disabled;") mysql_node.query("CREATE TABLE test_db_disabled.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index 976fc5211a7..94beb7b57ca 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -17,7 +17,7 @@ node = cluster.add_instance('node', def start_cluster(): try: cluster.start() - node.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic + node.query('CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary') # Different paths with Atomic yield cluster finally: cluster.shutdown() diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 49bc7baeb39..23e2d8d8c47 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -7,10 +7,6 @@ from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegis from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - @pytest.fixture(scope="module") def started_cluster(): try: diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index d71d1075c70..704620634cb 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -100,7 +100,7 @@ def test_reconnect(started_cluster): pm.heal_all() time.sleep(1) instance.query("INSERT INTO distributed VALUES (3)") - time.sleep(1) + time.sleep(5) assert remote.query("SELECT count(*) FROM local1").strip() == '3' diff --git a/tests/integration/test_join_set_family_s3/test.py b/tests/integration/test_join_set_family_s3/test.py index 2a20c8eade9..625cac48dee 100644 --- a/tests/integration/test_join_set_family_s3/test.py +++ b/tests/integration/test_join_set_family_s3/test.py @@ -4,9 +4,6 @@ import sys import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index c23e7545b27..115ec47871b 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -4,9 +4,6 @@ import sys import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 45b3c3c65f0..0779a57ccb6 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -6,9 +6,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 11a1e464cb6..7d9496f41d7 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -7,9 +7,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - # Runs custom python-based S3 endpoint. def run_endpoint(cluster): diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 346d9aced3f..934fa1e5ecd 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -6,9 +6,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_merge_tree_s3_with_cache/test.py b/tests/integration/test_merge_tree_s3_with_cache/test.py index d5d6db2fb77..7022f90a2b9 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/test.py +++ b/tests/integration/test_merge_tree_s3_with_cache/test.py @@ -3,9 +3,6 @@ import logging import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 1404cf409f3..60cf8a9239b 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=os.environ.get("DOCKER_API_VERSION"), timeout=180).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 3d65a489610..c70c29d1bf7 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -5,9 +5,6 @@ import pytest import requests from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 353d776c0f3..941c728a4de 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -61,7 +61,7 @@ def copy_quota_xml(local_file_name, reload_immediately=True): def started_cluster(): try: cluster.start() - + instance.query("DROP TABLE IF EXISTS test_table") instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()") instance.query("INSERT INTO test_table SELECT number FROM numbers(50)") diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 99e7d6077f8..9b169b87fe2 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -211,21 +211,21 @@ def test_recover_staled_replica(started_cluster): with PartitionManager() as pm: pm.drop_instance_zk_connections(dummy_node) dummy_node.query_and_get_error("RENAME TABLE recover.t1 TO recover.m1") - main_node.query("RENAME TABLE recover.t1 TO recover.m1", settings=settings) - main_node.query("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings) - main_node.query("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings) - main_node.query("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings) - main_node.query("DROP TABLE recover.rmt5", settings=settings) - main_node.query("DROP DICTIONARY recover.d2", settings=settings) - main_node.query("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", settings=settings) + main_node.query_with_retry("RENAME TABLE recover.t1 TO recover.m1", settings=settings) + main_node.query_with_retry("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings) + main_node.query_with_retry("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings) + main_node.query_with_retry("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings) + main_node.query_with_retry("DROP TABLE recover.rmt5", settings=settings) + main_node.query_with_retry("DROP DICTIONARY recover.d2", settings=settings) + main_node.query_with_retry("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) assert main_node.query("SELECT name FROM system.tables WHERE database='recover' ORDER BY name") == "d1\nd2\nm1\nmt1\nmt2\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" query = "SELECT name, uuid, create_table_query FROM system.tables WHERE database='recover' ORDER BY name" diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 3b3540ef1b8..d04bdae36e2 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -5,9 +5,6 @@ import string import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 1a5c6e2ce12..6c7b47ea0b1 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -3,9 +3,6 @@ import logging import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - def check_proxy_logs(cluster, proxy_instance): logs = cluster.get_container_logs(proxy_instance) diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 586895fffe5..7a1a2292eef 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,10 +5,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id('resolver') diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index a285b263688..aea6e0816d0 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -169,7 +169,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_host, port=cluster.mysql_port) return conn diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 13b10e0ca68..0bb7047d875 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -21,6 +21,7 @@ def get_postgres_conn(database=False): def create_postgres_db(conn, name): cursor = conn.cursor() + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) cursor.execute("CREATE DATABASE {}".format(name)) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5a81196a644..56cc7cf6e81 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -11,9 +11,6 @@ import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - MINIO_INTERNAL_PORT = 9001 # Creates S3 bucket for tests and allows anonymous read-write access to it. diff --git a/tests/integration/test_storage_s3/test_redirect.py b/tests/integration/test_storage_s3/test_redirect.py index ab677f46d9a..10a20184722 100644 --- a/tests/integration/test_storage_s3/test_redirect.py +++ b/tests/integration/test_storage_s3/test_redirect.py @@ -11,9 +11,6 @@ import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(cluster): # Allows read-write access for bucket without authorization. From e8a435d2fb41b5bf6dddbefdeb2e898fdd26dd12 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 15 Mar 2021 18:31:50 +0900 Subject: [PATCH 061/652] Add base condition --- .../parametric-functions.md | 174 +++++++++----- .../AggregateFunctionSequenceNextNode.cpp | 46 ++-- .../AggregateFunctionSequenceNextNode.h | 44 ++-- .../01656_sequence_next_node.reference | 30 +++ .../0_stateless/01656_sequence_next_node.sql | 221 ++++++++++-------- 5 files changed, 331 insertions(+), 184 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index dcc63ebd960..3bff7fa795c 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -513,7 +513,7 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode(direction, base)(timestamp, event_column, event1, event2, event3, ...) +sequenceNextNode(direction, base)(timestamp, base_condition, event_column, event1, event2, event3, ...) ``` **Parameters** @@ -529,11 +529,11 @@ sequenceNextNode(direction, base)(timestamp, event_column, event1, event2, event **Arguments** - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. -- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `base_condition` — Condition that the base point must fulfill. +- `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` - `cond` — Conditions describing the chain of events. `UInt8` **Returned value** - - `event_column[next_index]` - if the pattern is matched and next value exists. - `NULL` - if the pattern isn’t matched or next value doesn't exist. @@ -556,7 +556,7 @@ ORDER BY id; INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); -SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'A', page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; ``` Result: @@ -578,123 +578,181 @@ INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, ``` ```SQL -SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'Home', page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Base point, Matched with Home 1970-01-01 09:00:02 1 Gift // Matched with Gift 1970-01-01 09:00:03 1 Exit // The result - 1970-01-01 09:00:01 3 Home // Base point, Matched with Home - 1970-01-01 09:00:02 3 Home // Unmatched with Gift - 1970-01-01 09:00:03 3 Gift - 1970-01-01 09:00:04 3 Basket + 1970-01-01 09:00:01 2 Home // Base point, Matched with Home + 1970-01-01 09:00:02 2 Home // Unmatched with Gift + 1970-01-01 09:00:03 2 Gift + 1970-01-01 09:00:04 2 Basket - 1970-01-01 09:00:01 4 Gift // Base point, Unmatched with Home - 1970-01-01 09:00:02 4 Home - 1970-01-01 09:00:03 4 Gift - 1970-01-01 09:00:04 4 Basket + 1970-01-01 09:00:01 3 Gift // Base point, Unmatched with Home + 1970-01-01 09:00:02 3 Home + 1970-01-01 09:00:03 3 Gift + 1970-01-01 09:00:04 3 Basket ``` **Behavior for `backward` and `tail`** ```SQL -SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'tail')(dt, page = 'Basket', page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home 1970-01-01 09:00:02 1 Gift 1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home // The result -1970-01-01 09:00:03 3 Gift // Matched with Gift -1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home // The result +1970-01-01 09:00:03 2 Gift // Matched with Gift +1970-01-01 09:00:04 2 Basket // Base point, Matched with Basket -1970-01-01 09:00:01 4 Gift -1970-01-01 09:00:02 4 Home // The result -1970-01-01 09:00:03 4 Gift // Base point, Matched with Gift -1970-01-01 09:00:04 4 Basket // Base point, Matched with Basket +1970-01-01 09:00:01 3 Gift +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point, Matched with Gift +1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket ``` **Behavior for `forward` and `first_match`** ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit // The result -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket The result +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket The result -1970-01-01 09:00:01 4 Gift // Base point -1970-01-01 09:00:02 4 Home // Thre result -1970-01-01 09:00:03 4 Gift -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift // Base point +1970-01-01 09:00:02 3 Home // Thre result +1970-01-01 09:00:03 3 Gift +1970-01-01 09:00:04 3 Basket ``` ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit // Unmatched with Home -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket // Unmatched with Home +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket // Unmatched with Home -1970-01-01 09:00:01 4 Gift // Base point -1970-01-01 09:00:02 4 Home // Matched with Home -1970-01-01 09:00:03 4 Gift // The result -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift // Base point +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // The result +1970-01-01 09:00:04 3 Basket ``` **Behavior for `backward` and `last_match`** ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // The result 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home // The result -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home // The result +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket -1970-01-01 09:00:01 4 Gift -1970-01-01 09:00:02 4 Home // The result -1970-01-01 09:00:03 4 Gift // Base point -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket ``` ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Matched with Home, the result is null 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit -1970-01-01 09:00:01 3 Home // The result -1970-01-01 09:00:02 3 Home // Matched with Home -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket +1970-01-01 09:00:01 2 Home // The result +1970-01-01 09:00:02 2 Home // Matched with Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket -1970-01-01 09:00:01 4 Gift // The result -1970-01-01 09:00:02 4 Home // Matched with Home -1970-01-01 09:00:03 4 Gift // Base point -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift // The result +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket +``` + + +**Behavior for `base_condition`** + +```SQL +CREATE TABLE test_flow_basecond +( + `dt` DateTime, + `id` int, + `page` String, + `ref` String +) +ENGINE = MergeTree +PARTITION BY toYYYYMMDD(dt) +ORDER BY id + +INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3, 1, 'B', 'ref2') (4, 1, 'B', 'ref1'); +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'head')(dt, ref = 'ref1', page, page = 'A') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'. + 1970-01-01 09:00:02 1 A ref3 + 1970-01-01 09:00:03 1 B ref2 + 1970-01-01 09:00:04 1 B ref1 + ``` + +```SQL +SELECT id, sequenceNextNode('backward', 'tail')(dt, ref = 'ref4', page, page = 'B') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 + 1970-01-01 09:00:02 1 A ref3 + 1970-01-01 09:00:03 1 B ref2 + 1970-01-01 09:00:04 1 B ref1 // The tail can't be base point becasue the ref column of the tail unmatched with 'ref4'. +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, ref = 'ref3', page, page = 'A') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'. + 1970-01-01 09:00:02 1 A ref3 // Base point + 1970-01-01 09:00:03 1 B ref2 // The result + 1970-01-01 09:00:04 1 B ref1 +``` + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, ref = 'ref2', page, page = 'B') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 + 1970-01-01 09:00:02 1 A ref3 // The result + 1970-01-01 09:00:03 1 B ref2 // Base point + 1970-01-01 09:00:04 1 B ref1 // This row can't be base point becasue the ref column unmatched with 'ref2'. ``` diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index b8f517cba20..c31337e25c6 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -12,6 +12,7 @@ namespace DB { constexpr size_t MAX_EVENTS_SIZE = 64; +constexpr size_t MIN_REQUIRED_ARGS = 3; namespace ErrorCodes { @@ -27,13 +28,13 @@ template inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl2(const DataTypePtr data_type, const DataTypes & argument_types, SeqBase base) { if (base == HEAD) - return std::make_shared, Direction, HEAD>>(data_type, argument_types); + return std::make_shared, Direction, HEAD, MIN_REQUIRED_ARGS>>(data_type, argument_types); else if (base == TAIL) - return std::make_shared, Direction, TAIL>>(data_type, argument_types); + return std::make_shared, Direction, TAIL, MIN_REQUIRED_ARGS>>(data_type, argument_types); else if (base == FIRST_MATCH) - return std::make_shared, Direction, FIRST_MATCH>>(data_type, argument_types); + return std::make_shared, Direction, FIRST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); else - return std::make_shared, Direction, LAST_MATCH>>(data_type, argument_types); + return std::make_shared, Direction, LAST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); } template @@ -46,9 +47,9 @@ inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl1(const DataT } AggregateFunctionPtr -createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, const DataTypes & argument_types, const Array & parameters) +createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, const DataTypes & argument_types, const Array & parameters) { - assert(max_args <= MAX_EVENTS_SIZE); + assert(max_events <= MAX_EVENTS_SIZE); if (parameters.size() < 2) throw Exception("Aggregate function " + name + " requires 2 parameters (direction, head)", @@ -76,19 +77,26 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c else throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; - if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < 3) - throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() < 2) + if (argument_types.size() < MIN_REQUIRED_ARGS) throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > max_args + 2) + else if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < MIN_REQUIRED_ARGS + 1) + throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > max_events + MIN_REQUIRED_ARGS) throw Exception("Aggregate function " + name + " requires at most " + - std::to_string(max_args + 2) + - " (timestamp, value_column, " + std::to_string(max_args) + " events) arguments.", + std::to_string(max_events + MIN_REQUIRED_ARGS) + + " (timestamp, value_column, " + std::to_string(max_events) + " events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (const auto i : ext::range(2, argument_types.size())) + if (const auto * cond_arg = argument_types[1].get()) + { + if (!isUInt8(cond_arg)) + throw Exception{"Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " + + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + for (const auto i : ext::range(MIN_REQUIRED_ARGS, argument_types.size())) { const auto * cond_arg = argument_types[i].get(); if (!isUInt8(cond_arg)) @@ -96,12 +104,12 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) + if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String) throw Exception{"Illegal type " + argument_types[1].get()->getName() + " of second argument of aggregate function " + name + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - DataTypePtr data_type = makeNullable(argument_types[1]); + DataTypePtr data_type = makeNullable(argument_types[2]); WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) @@ -122,11 +130,11 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } -auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_args) +auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) { - return [max_args](const std::string & name, const DataTypes & argument_types, const Array & parameters) + return [max_events](const std::string & name, const DataTypes & argument_types, const Array & parameters) { - return createAggregateFunctionSequenceNode(name, max_args, argument_types, parameters); + return createAggregateFunctionSequenceNode(name, max_events, argument_types, parameters); }; } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index f58469f27bb..a37816cfa4a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -49,6 +49,7 @@ struct NodeBase DataTypeDateTime::FieldType event_time; std::bitset events_bitset; + bool can_be_base; char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -68,6 +69,7 @@ struct NodeBase writeBinary(event_time, buf); UInt64 ulong_bitset = events_bitset.to_ulong(); writeBinary(ulong_bitset, buf); + writeBinary(can_be_base, buf); } static Node * read(ReadBuffer & buf, Arena * arena) @@ -83,6 +85,7 @@ struct NodeBase UInt64 ulong_bitset; readBinary(ulong_bitset, buf); node->events_bitset = ulong_bitset; + readBinary(node->can_be_base, buf); return node; } @@ -146,13 +149,15 @@ struct SequenceNextNodeGeneralData }; /// Implementation of sequenceFirstNode -template +template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } + static constexpr size_t EventColumn = 2; + static constexpr size_t BaseCondition = 1; DataTypePtr & data_type; UInt8 events_size; @@ -160,10 +165,10 @@ class SequenceNextNodeImpl final public: SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) - , events_size(arguments.size() - 2) + , events_size(arguments.size() - MinRequiredArgs) , max_elems(max_elems_) { } @@ -196,7 +201,7 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - Node * node = Node::allocate(*columns[1], row_num, arena); + Node * node = Node::allocate(*columns[EventColumn], row_num, arena); const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; @@ -208,10 +213,12 @@ public: /// + 4 (bit of event3) node->events_bitset.reset(); for (UInt8 i = 0; i < events_size; ++i) - if (assert_cast *>(columns[2 + i])->getData()[row_num]) + if (assert_cast *>(columns[MinRequiredArgs + i])->getData()[row_num]) node->events_bitset.set(i); node->event_time = timestamp; + node->can_be_base = assert_cast *>(columns[BaseCondition])->getData()[row_num]; + data(place).value.push_back(node, arena); } @@ -306,19 +313,31 @@ public: inline UInt32 getBaseIndex(Data & data, bool & exist) const { + exist = false; + if (data.value.size() == 0) + return 0; + switch (Base) { case HEAD: - exist = true; - return 0; + if (data.value[0]->can_be_base) + { + exist = true; + return 0; + } + break; case TAIL: - exist = true; - return data.value.size() - 1; + if (data.value[data.value.size() - 1]->can_be_base) + { + exist = true; + return data.value.size() - 1; + } + break; case FIRST_MATCH: for (UInt64 i = 0; i < data.value.size(); ++i) - if (data.value[i]->events_bitset.test(0)) + if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base) { exist = true; return i; @@ -329,7 +348,7 @@ public: for (UInt64 i = 0; i < data.value.size(); ++i) { auto reversed_i = data.value.size() - i - 1; - if (data.value[reversed_i]->events_bitset.test(0)) + if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base) { exist = true; return reversed_i; @@ -338,7 +357,6 @@ public: break; } - exist = false; return 0; } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 9309641f3bf..739f19818a6 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -459,3 +459,33 @@ (max_args) 10 \N (max_args) 11 \N (forward, head, A) id = 12 A +(forward, head, 1) 1 A +(forward, head, 1) 2 \N +(forward, head, 1) 3 \N +(forward, head, 1, A) 1 B +(forward, head, 1, A) 2 \N +(forward, head, 1, A) 3 \N +(forward, head, 1, A->B) 1 C +(forward, head, 1, A->B) 2 \N +(forward, head, 1, A->B) 3 \N +(backward, tail, 1) 1 \N +(backward, tail, 1) 2 A +(backward, tail, 1) 3 \N +(backward, tail, 1, A) 1 \N +(backward, tail, 1, A) 2 B +(backward, tail, 1, A) 3 \N +(backward, tail, 1, A->B) 1 \N +(backward, tail, 1, A->B) 2 C +(backward, tail, 1, A->B) 3 \N +(forward, first_match, 1, B) 1 C +(forward, first_match, 1, B) 2 A +(forward, first_match, 1, B) 3 D +(forward, first_match, 1, B->C) 1 D +(forward, first_match, 1, B->C) 2 \N +(forward, first_match, 1, B->C) 3 \N +(backward, first_match, 1, B) 1 A +(backward, first_match, 1, B) 2 C +(backward, first_match, 1, B) 3 B +(backward, first_match, 1, B->C) 1 \N +(backward, first_match, 1, B->C) 2 \N +(backward, first_match, 1, B->C) 3 \N diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 63810265bf9..c03abed8a54 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -27,26 +27,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); @@ -54,41 +54,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; @@ -123,73 +123,106 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); -SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; DROP TABLE IF EXISTS test_sequenceNextNode; + +DROP TABLE IF EXISTS test_base_condition; + +CREATE TABLE IF NOT EXISTS test_base_condition (dt DateTime, id int, action String, referrer String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',1,'A','1'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',1,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',1,'C','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',1,'D','4'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',2,'D','4'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',2,'C','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',2,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',2,'A','1'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',3,'B','10'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4'); + +SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_base_condition; From a4e9cf26b0c5f95621148e7d60f3c8ea268324c4 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 19 Mar 2021 17:43:20 +0900 Subject: [PATCH 062/652] Fix reverse indexing of TAIL in serialize() --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index a37816cfa4a..c0ecdadf8b1 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -281,8 +281,8 @@ public: case TAIL: writeVarUInt(size, buf); - for (size_t i = value.size() - 1; i >= size; --i) - value[i]->write(buf); + for (size_t i = 0; i < size; ++i) + value[value.size() - size + i]->write(buf); break; case FIRST_MATCH: From a67f066f0a9bb25bbdd63f4a455a99debe6c5e81 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:44:03 +0300 Subject: [PATCH 063/652] fixes --- tests/integration/helpers/cluster.py | 3 +- .../materialize_with_ddl.py | 376 +++++++++--------- .../test_materialize_mysql_database/test.py | 54 ++- 3 files changed, 213 insertions(+), 220 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 124f77061cb..fbf5e425426 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -656,10 +656,11 @@ class ClickHouseCluster: raise Exception("Cannot wait MySQL container") def wait_mysql8_to_start(self, timeout=60): + self.mysql8_ip = self.get_instance_ip('mysql80') start = time.time() while time.time() - start < timeout: try: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql8_port) + conn = pymysql.connect(user='root', password='clickhouse', host=self.mysql8_ip, port=self.mysql8_port) conn.close() logging.debug("Mysql 8 Started") return diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 7e711d62576..c2f46a4b986 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -3,7 +3,7 @@ import time import pymysql.cursors import pytest from helpers.network import PartitionManager -import pytest +import logging from helpers.client import QueryRuntimeException from helpers.cluster import get_docker_compose_path, run_and_check import random @@ -12,7 +12,7 @@ import threading from multiprocessing.dummy import Pool from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=60, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=1, interval_seconds=3): lastest_result = '' for i in range(retry_count): @@ -21,10 +21,10 @@ def check_query(clickhouse_node, query, result_set, retry_count=60, interval_sec if result_set == lastest_result: return - print(lastest_result) + logging.debug(f"latest_result{lastest_result}") time.sleep(interval_seconds) except Exception as e: - print(("check_query retry {} exception {}".format(i + 1, e))) + logging.debug(f"check_query retry {i+1} exception {e}") time.sleep(interval_seconds) else: assert clickhouse_node.query(query) == result_set @@ -118,275 +118,275 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + mysql_node.query("DROP DATABASE IF EXISTS test_database_dt") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_dt") + mysql_node.query("CREATE DATABASE test_database_dt DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_dt.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") + clickhouse_node.query("CREATE DATABASE test_database_dt ENGINE = MaterializeMySQL('{}:3306', 'test_database_dt', 'root', 'clickhouse')".format(service_name)) + assert "test_database_dt" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dt.test_table_1 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + mysql_node.query("CREATE TABLE test_database_dt.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dt.test_table_2 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") - + clickhouse_node.query("DROP DATABASE test_database_dt") + mysql_node.query("DROP DATABASE test_database_dt") def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_drop") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_drop") + mysql_node.query("CREATE DATABASE test_database_drop DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_drop.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("DROP TABLE test_database.test_table_1;") + mysql_node.query("DROP TABLE test_database_drop.test_table_1;") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database_drop.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + mysql_node.query("TRUNCATE TABLE test_database_drop.test_table_2;") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_drop ENGINE = MaterializeMySQL('{}:3306', 'test_database_drop', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + assert "test_database_drop" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_drop.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") + mysql_node.query("CREATE TABLE test_database_drop.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_drop FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") - mysql_node.query("DROP TABLE test_database.test_table_1;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + mysql_node.query("DROP TABLE test_database_drop.test_table_1;") + mysql_node.query("TRUNCATE TABLE test_database_drop.test_table_2;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_drop FORMAT TSV", "test_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "") + + clickhouse_node.query("DROP DATABASE test_database_drop") + mysql_node.query("DROP DATABASE test_database_drop") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_create") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_create") + mysql_node.query("CREATE DATABASE test_database_create DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database_create.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") # it already has some data - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") + mysql_node.query("INSERT INTO test_database_create.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_create ENGINE = MaterializeMySQL('{}:3306', 'test_database_create', 'root', 'clickhouse')".format( service_name)) # Check for pre-existing status - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", + assert "test_database_create" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SELECT * FROM test_database_create.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("CREATE TABLE test_database_create.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_create.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") + check_query(clickhouse_node, "SELECT * FROM test_database_create.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_create") + mysql_node.query("DROP DATABASE test_database_create") def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_rename") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_rename") + mysql_node.query("CREATE DATABASE test_database_rename DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_rename.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") + mysql_node.query("RENAME TABLE test_database_rename.test_table_1 TO test_database_rename.test_table_2") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_rename ENGINE = MaterializeMySQL('{}:3306', 'test_database_rename', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") - mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + assert "test_database_rename" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename FORMAT TSV", "test_table_2\n") + mysql_node.query("RENAME TABLE test_database_rename.test_table_2 TO test_database_rename.test_table_1") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename FORMAT TSV", "test_table_1\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_rename") + mysql_node.query("DROP DATABASE test_database_rename") def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_add") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_add") + mysql_node.query("CREATE DATABASE test_database_add DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_add.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( "0" if service_name == "mysql57" else "(id)")) # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_add ENGINE = MaterializeMySQL('{}:3306', 'test_database_add', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_add" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "DESC test_database_add.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("CREATE TABLE test_database_add.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_add FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_add.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") + "ALTER TABLE test_database_add.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + "ALTER TABLE test_database_add.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( "0" if service_name == "mysql57" else "(id)")) default_expression = "DEFAULT\t0" if service_name == "mysql57" else "DEFAULT\tid" - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + check_query(clickhouse_node, "DESC test_database_add.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_add.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database_add.test_table_2 ORDER BY id FORMAT TSV", "1\t2\t3\t4\t5\n6\t7\t8\t9\t10\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_add") + mysql_node.query("DROP DATABASE test_database_add") def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_drop") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_drop") + mysql_node.query("CREATE DATABASE test_database_alter_drop DEFAULT CHARACTER SET 'utf8'") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_drop.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") + mysql_node.query("ALTER TABLE test_database_alter_drop.test_table_1 DROP COLUMN drop_column") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_drop ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_drop', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_drop" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_drop FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_drop.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_drop FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_drop.test_table_2 DROP COLUMN drop_column") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("INSERT INTO test_database_alter_drop.test_table_2 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_drop.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_drop") + mysql_node.query("DROP DATABASE test_database_alter_drop") def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_rename") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_rename") + mysql_node.query("CREATE DATABASE test_database_alter_rename DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_rename.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") + mysql_node.query("ALTER TABLE test_database_alter_rename.test_table_1 RENAME COLUMN rename_column TO new_column_name") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_rename ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_rename', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_rename" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_rename.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nrename_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_rename.test_table_2 RENAME COLUMN rename_column TO new_column_name") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_alter_rename.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_rename.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t4\n5\t6\n7\t8\n9\t10\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_rename") + mysql_node.query("DROP DATABASE test_database_alter_rename") def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_modify") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_modify") + mysql_node.query("CREATE DATABASE test_database_alter_modify DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_modify.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_1 MODIFY COLUMN modify_column INT") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_modify ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_modify', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_modify" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_modify FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_modify.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_modify FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT FIRST") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT FIRST") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "modify_column\tNullable(Int32)\t\t\t\t\t\nid\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT AFTER id") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT AFTER id") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, NULL)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") + mysql_node.query("INSERT INTO test_database_alter_modify.test_table_2 VALUES(1, 2), (3, NULL)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_modify.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_modify") + mysql_node.query("DROP DATABASE test_database_alter_modify") # TODO: need ClickHouse support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; @@ -394,53 +394,53 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n # pass def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_rename_table") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_rename_table") + mysql_node.query("CREATE DATABASE test_database_rename_table DEFAULT CHARACTER SET 'utf8'") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + "CREATE TABLE test_database_rename_table.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") mysql_node.query( - "ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_3") + "ALTER TABLE test_database_rename_table.test_table_1 DROP COLUMN drop_column, RENAME TO test_database_rename_table.test_table_2, RENAME TO test_database_rename_table.test_table_3") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_rename_table ENGINE = MaterializeMySQL('{}:3306', 'test_database_rename_table', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_3 FORMAT TSV", + assert "test_database_rename_table" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_3\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_3 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + "CREATE TABLE test_database_rename_table.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_1\ntest_table_3\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_4") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\ntest_table_4\n") - check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", + "ALTER TABLE test_database_rename_table.test_table_1 DROP COLUMN drop_column, RENAME TO test_database_rename_table.test_table_2, RENAME TO test_database_rename_table.test_table_4") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_3\ntest_table_4\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_4 VALUES(1), (2), (3), (4), (5)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("INSERT INTO test_database_rename_table.test_table_4 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database_rename_table.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_rename_table") + mysql_node.query("DROP DATABASE test_database_rename_table") def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database") + mysql_node.query("DROP DATABASE IF EXISTS test_database_event") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_event") + mysql_node.query("CREATE DATABASE test_database_event") mysql_node.query("RESET MASTER") - mysql_node.query("CREATE TABLE test_database.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(1)") + mysql_node.query("CREATE TABLE test_database_event.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") + mysql_node.query("INSERT INTO test_database_event.t1(a) VALUES(1)") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_event ENGINE = MaterializeMySQL('{}:3306', 'test_database_event', 'root', 'clickhouse')".format( service_name)) # Reject one empty GTID QUERY event with 'BEGIN' and 'COMMIT' @@ -456,13 +456,13 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name # Reject one 'BEGIN' QUERY event and 'COMMIT' XID event. mysql_node.query("/* start */ begin /* end */") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(2)") + mysql_node.query("INSERT INTO test_database_event.t1(a) VALUES(2)") mysql_node.query("/* start */ commit /* end */") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "t1\n") - check_query(clickhouse_node, "SELECT * FROM test_database.t1 ORDER BY a FORMAT TSV", "1\tBEGIN\n2\tBEGIN\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_event FORMAT TSV", "t1\n") + check_query(clickhouse_node, "SELECT * FROM test_database_event.t1 ORDER BY a FORMAT TSV", "1\tBEGIN\n2\tBEGIN\n") + clickhouse_node.query("DROP DATABASE test_database_event") + mysql_node.query("DROP DATABASE test_database_event") def select_without_columns(clickhouse_node, mysql_node, service_name): @@ -592,23 +592,23 @@ def drop_instance_mysql_connections(clickhouse_node, pm, action='REJECT'): time.sleep(5) def network_partition_test(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_network") clickhouse_node.query("DROP DATABASE IF EXISTS test") - mysql_node.query("DROP DATABASE IF EXISTS test_database") + mysql_node.query("DROP DATABASE IF EXISTS test_database_network") mysql_node.query("DROP DATABASE IF EXISTS test") - mysql_node.query("CREATE DATABASE test_database;") - mysql_node.query("CREATE TABLE test_database.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + mysql_node.query("CREATE DATABASE test_database_network;") + mysql_node.query("CREATE TABLE test_database_network.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") mysql_node.query("CREATE DATABASE test;") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table", '') + "CREATE DATABASE test_database_network ENGINE = MaterializeMySQL('{}:3306', 'test_database_network', 'root', 'clickhouse')".format(service_name)) + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table", '') with PartitionManager() as pm: drop_instance_mysql_connections(clickhouse_node, pm) - mysql_node.query('INSERT INTO test_database.test_table VALUES(1)') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table", '') + mysql_node.query('INSERT INTO test_database_network.test_table VALUES(1)') + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table", '') with pytest.raises(QueryRuntimeException) as exception: clickhouse_node.query( @@ -618,18 +618,18 @@ def network_partition_test(clickhouse_node, mysql_node, service_name): restore_instance_mysql_connections(clickhouse_node, pm) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table FORMAT TSV", '1\n') + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table FORMAT TSV", '1\n') clickhouse_node.query( "CREATE DATABASE test ENGINE = MaterializeMySQL('{}:3306', 'test', 'root', 'clickhouse')".format(service_name)) - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table\n") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_network FORMAT TSV", "test_table\n") mysql_node.query("CREATE TABLE test.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test FORMAT TSV", "test\n") - clickhouse_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_network") clickhouse_node.query("DROP DATABASE test") - mysql_node.query("DROP DATABASE test_database") + mysql_node.query("DROP DATABASE test_database_network") mysql_node.query("DROP DATABASE test") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index f617d5f958f..3f256fd5d67 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -31,7 +31,7 @@ def started_cluster(): class MySQLConnection: - def __init__(self, port, user='root', password='clickhouse', ip_address='127.0.0.1', docker_compose=None, project_name=cluster.project_name): + def __init__(self, port, user='root', password='clickhouse', ip_address=None, docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address @@ -77,39 +77,31 @@ class MySQLConnection: cursor.execute(executio_query) return cursor.fetchall() - def start_and_wait(self): - run_and_check(['docker-compose', - '-p', cluster.project_name, - '-f', self.docker_compose, - 'up', '--no-recreate', '-d', - ]) - self.wait_mysql_to_start(120) - def close(self): if self.mysql_connection is not None: self.mysql_connection.close() @pytest.fixture(scope="module") def started_mysql_5_7(): - mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', '127.0.0.1') + mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', cluster.mysql_ip) yield mysql_node @pytest.fixture(scope="module") def started_mysql_8_0(): - mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', '127.0.0.1') + mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', cluster.mysql8_ip) yield mysql8_node -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @@ -121,7 +113,7 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @@ -132,73 +124,73 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") From c3c409556eef70f9391c699eda7dcc338e2139c1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:44:28 +0300 Subject: [PATCH 064/652] support tests list in runner --- tests/integration/runner | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 3aa3aa5ef64..1c1aed2f6db 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -148,6 +148,14 @@ if __name__ == "__main__": dest="parallel", help="Parallelism") + parser.add_argument( + "-t", "--tests_list", + action="store", + nargs='+', + default=[], + dest="tests_list", + help="List of tests to run") + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() @@ -200,7 +208,7 @@ if __name__ == "__main__": --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=300 \ - {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( + {env_tags} -e PYTEST_OPTS='{parallel} {opts} {tests_list}' {img} {command}".format( net=net, tty=tty, bin=args.binary, @@ -211,6 +219,7 @@ if __name__ == "__main__": env_tags=env_tags, parallel=parallel_args, opts=' '.join(args.pytest_args), + tests_list=' '.join(args.tests_list), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, command=args.command From 8f53d1344cbf5260bbc0884f330aa966c0d8fffb Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:48:28 +0300 Subject: [PATCH 065/652] better test cases names --- .../integration/test_dictionaries_ddl/test.py | 8 +- .../test_distributed_over_distributed/test.py | 11 +- tests/integration/test_multiple_disks/test.py | 50 +++--- .../test_mysql_database_engine/test.py | 84 ++++----- .../test.py | 12 +- tests/integration/test_ttl_move/test.py | 168 +++++++++--------- tests/integration/test_ttl_replicated/test.py | 4 +- 7 files changed, 170 insertions(+), 167 deletions(-) diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 3aa76b4af88..ea0de044e6a 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -60,10 +60,10 @@ def started_cluster(): @pytest.mark.parametrize("clickhouse,name,layout", [ - (node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), - (node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), - (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), - (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), + pytest.param(node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())', id="complex_node1_hashed"), + pytest.param(node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))', id="complex_node1_cache"), + pytest.param(node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())', id="complex_node2_hashed"), + pytest.param(node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))', id="complex_node2_cache"), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) diff --git a/tests/integration/test_distributed_over_distributed/test.py b/tests/integration/test_distributed_over_distributed/test.py index 410a03a6af1..ae86a70f31b 100644 --- a/tests/integration/test_distributed_over_distributed/test.py +++ b/tests/integration/test_distributed_over_distributed/test.py @@ -2,7 +2,6 @@ # (just in case, with real separate instances). - import pytest from helpers.cluster import ClickHouseCluster @@ -51,9 +50,13 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("node", list(NODES.values())) -@pytest.mark.parametrize("source", - ["distributed_over_distributed_table", "cluster('test_cluster', default, distributed_table)"]) +@pytest.mark.parametrize("node,source", [ + pytest.param(NODES["node1"], "distributed_over_distributed_table", id="dod_node1"), + pytest.param(NODES["node1"], "cluster('test_cluster', default, distributed_table)", id="cluster_node1"), + pytest.param(NODES["node2"], "distributed_over_distributed_table", id="dod_node2"), + pytest.param(NODES["node2"], "cluster('test_cluster', default, distributed_table)", id="cluster_node2"), +] +) class TestDistributedOverDistributedSuite: def test_select_with_order_by_node(self, started_cluster, node, source): assert node.query("SELECT * FROM {source} ORDER BY node, key".format(source=source)) \ diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 24ee6c0493b..1d672f8f7ce 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -290,8 +290,8 @@ def test_query_parser(start_cluster): @pytest.mark.parametrize("name,engine", [ - ("test_alter_policy", "MergeTree()"), - ("replicated_test_alter_policy", "ReplicatedMergeTree('/clickhouse/test_alter_policy', '1')",), + pytest.param("test_alter_policy", "MergeTree()", id="mt"), + pytest.param("replicated_test_alter_policy", "ReplicatedMergeTree('/clickhouse/test_alter_policy', '1')", id="replicated"), ]) def test_alter_policy(start_cluster, name, engine): try: @@ -364,8 +364,8 @@ def test_no_warning_about_zero_max_data_part_size(start_cluster): @pytest.mark.parametrize("name,engine", [ - ("mt_on_jbod", "MergeTree()"), - ("replicated_mt_on_jbod", "ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')",), + pytest.param("mt_on_jbod", "MergeTree()", id="mt"), + pytest.param("replicated_mt_on_jbod", "ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')", id="replicated"), ]) def test_round_robin(start_cluster, name, engine): try: @@ -400,8 +400,8 @@ def test_round_robin(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("mt_with_huge_part", "MergeTree()"), - ("replicated_mt_with_huge_part", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')",), + pytest.param("mt_with_huge_part", "MergeTree()", id="mt"), + pytest.param("replicated_mt_with_huge_part", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')", id="replicated"), ]) def test_max_data_part_size(start_cluster, name, engine): try: @@ -427,8 +427,8 @@ def test_max_data_part_size(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("mt_with_overflow", "MergeTree()"), - ("replicated_mt_with_overflow", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')",), + pytest.param("mt_with_overflow", "MergeTree()", id="mt"), + pytest.param("replicated_mt_with_overflow", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')", id="replicated"), ]) def test_jbod_overflow(start_cluster, name, engine): try: @@ -480,8 +480,8 @@ def test_jbod_overflow(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("moving_mt", "MergeTree()"), - ("moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')",), + pytest.param("moving_mt", "MergeTree()", id="mt"), + pytest.param("moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')", id="replicated"), ]) def test_background_move(start_cluster, name, engine): try: @@ -530,8 +530,8 @@ def test_background_move(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("stopped_moving_mt", "MergeTree()"), - ("stopped_moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')",), + pytest.param("stopped_moving_mt", "MergeTree()", id="mt"), + pytest.param("stopped_moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')", id="replicated"), ]) def test_start_stop_moves(start_cluster, name, engine): try: @@ -633,7 +633,7 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): @pytest.mark.parametrize("name,engine", [ - ("altering_mt", "MergeTree()"), + pytest.param("altering_mt", "MergeTree()", id="mt"), # ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",), # SYSTEM STOP MERGES doesn't disable merges assignments ]) @@ -827,9 +827,9 @@ def produce_alter_move(node, name): @pytest.mark.parametrize("name,engine", [ - ("concurrently_altering_mt", "MergeTree()"), - ("concurrently_altering_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')",), + pytest.param("concurrently_altering_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_altering_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", id="replicated"), ]) def test_concurrent_alter_move(start_cluster, name, engine): try: @@ -882,9 +882,9 @@ def test_concurrent_alter_move(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("concurrently_dropping_mt", "MergeTree()"), - ("concurrently_dropping_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')",), + pytest.param("concurrently_dropping_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_dropping_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')", id="replicated"), ]) def test_concurrent_alter_move_and_drop(start_cluster, name, engine): try: @@ -935,8 +935,8 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("detach_attach_mt", "MergeTree()"), - ("replicated_detach_attach_mt", "ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')",), + pytest.param("detach_attach_mt", "MergeTree()", id="mt"), + pytest.param("replicated_detach_attach_mt", "ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')", id="replicated"), ]) def test_detach_attach(start_cluster, name, engine): try: @@ -966,8 +966,8 @@ def test_detach_attach(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("mutating_mt", "MergeTree()"), - ("replicated_mutating_mt", "ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",), + pytest.param("mutating_mt", "MergeTree()", id="mt"), + pytest.param("replicated_mutating_mt", "ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')", id="replicated"), ]) def test_mutate_to_another_disk(start_cluster, name, engine): try: @@ -1012,8 +1012,8 @@ def test_mutate_to_another_disk(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("alter_modifying_mt", "MergeTree()"), - ("replicated_alter_modifying_mt", "ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')",), + pytest.param("alter_modifying_mt", "MergeTree()", id="mt"), + pytest.param("replicated_alter_modifying_mt", "ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')", id="replicated"), ]) def test_concurrent_alter_modify(start_cluster, name, engine): try: diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 07a4a987b59..5e6486e31c6 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -220,60 +220,60 @@ timestamp_values_no_subsecond = ["'2015-05-18 07:40:01'", "'2019-09-16 19:20:11' # ("common_types", "FLOAT", "Nullable(Float32)", float_values, ""), # ("common_types", "FLOAT UNSIGNED", "Nullable(Float32)", float_values, ""), - ("common_types", "INT", "Nullable(Int32)", int32_values, ""), - ("common_types", "INT NOT NULL", "Int32", int32_values, ""), - ("common_types", "INT UNSIGNED NOT NULL", "UInt32", uint32_values, ""), - ("common_types", "INT UNSIGNED", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10)", "Nullable(Int32)", int32_values, ""), - ("common_types", "INT(10) NOT NULL", "Int32", int32_values, ""), - ("common_types", "INT(10) UNSIGNED NOT NULL", "UInt32", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INTEGER", "Nullable(Int32)", int32_values, ""), - ("common_types", "INTEGER UNSIGNED", "Nullable(UInt32)", uint32_values, ""), + pytest.param("common_types", "INT", "Nullable(Int32)", int32_values, "", id="common_types_1"), + pytest.param("common_types", "INT NOT NULL", "Int32", int32_values, "", id="common_types_2"), + pytest.param("common_types", "INT UNSIGNED NOT NULL", "UInt32", uint32_values, "", id="common_types_3"), + pytest.param("common_types", "INT UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_4"), + pytest.param("common_types", "INT UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, "", id="common_types_5"), + pytest.param("common_types", "INT UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, "", id="common_types_6"), + pytest.param("common_types", "INT(10)", "Nullable(Int32)", int32_values, "", id="common_types_7"), + pytest.param("common_types", "INT(10) NOT NULL", "Int32", int32_values, "", id="common_types_8"), + pytest.param("common_types", "INT(10) UNSIGNED NOT NULL", "UInt32", uint32_values, "", id="common_types_8"), + pytest.param("common_types", "INT(10) UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_9"), + pytest.param("common_types", "INT(10) UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, "", id="common_types_10"), + pytest.param("common_types", "INT(10) UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, "", id="common_types_11"), + pytest.param("common_types", "INTEGER", "Nullable(Int32)", int32_values, "", id="common_types_12"), + pytest.param("common_types", "INTEGER UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_13"), - ("common_types", "MEDIUMINT", "Nullable(Int32)", mint_values, ""), - ("common_types", "MEDIUMINT UNSIGNED", "Nullable(UInt32)", umint_values, ""), + pytest.param("common_types", "MEDIUMINT", "Nullable(Int32)", mint_values, "", id="common_types_14"), + pytest.param("common_types", "MEDIUMINT UNSIGNED", "Nullable(UInt32)", umint_values, "", id="common_types_15"), - ("common_types", "SMALLINT", "Nullable(Int16)", int16_values, ""), - ("common_types", "SMALLINT UNSIGNED", "Nullable(UInt16)", uint16_values, ""), + pytest.param("common_types", "SMALLINT", "Nullable(Int16)", int16_values, "", id="common_types_16"), + pytest.param("common_types", "SMALLINT UNSIGNED", "Nullable(UInt16)", uint16_values, "", id="common_types_17"), - ("common_types", "TINYINT", "Nullable(Int8)", int8_values, ""), - ("common_types", "TINYINT UNSIGNED", "Nullable(UInt8)", uint8_values, ""), + pytest.param("common_types", "TINYINT", "Nullable(Int8)", int8_values, "", id="common_types_18"), + pytest.param("common_types", "TINYINT UNSIGNED", "Nullable(UInt8)", uint8_values, "", id="common_types_19"), - ("common_types", "VARCHAR(10)", "Nullable(String)", string_values, ""), + pytest.param("common_types", "VARCHAR(10)", "Nullable(String)", string_values, "", id="common_types_20"), - ("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values, - "decimal,datetime64"), - ("decimal_default_nullable", "decimal", "Nullable(Decimal(10, 0))", decimal_values, - "decimal,datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "Decimal(18, 6)", decimal_values, - "decimal,datetime64"), - ("decimal_38_6", "decimal(38, 6) NOT NULL", "Decimal(38, 6)", decimal_values, - "decimal,datetime64"), + pytest.param("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values, + "decimal,datetime64", id="decimal_1"), + pytest.param("decimal_default_nullable", "decimal", "Nullable(Decimal(10, 0))", decimal_values, + "decimal,datetime64", id="decimal_2"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "Decimal(18, 6)", decimal_values, + "decimal,datetime64", id="decimal_3"), + pytest.param("decimal_38_6", "decimal(38, 6) NOT NULL", "Decimal(38, 6)", decimal_values, + "decimal,datetime64", id="decimal_4"), # Due to python DB driver roundtrip MySQL timestamp and datetime values # are printed with 6 digits after decimal point, so to simplify tests a bit, # we only validate precision of 0 and 6. - ("timestamp_default", "timestamp", "DateTime", timestamp_values, "decimal,datetime64"), - ("timestamp_6", "timestamp(6)", "DateTime64(6)", timestamp_values, "decimal,datetime64"), - ("datetime_default", "DATETIME NOT NULL", "DateTime64(0)", timestamp_values, - "decimal,datetime64"), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime64(6)", timestamp_values, - "decimal,datetime64"), + pytest.param("timestamp_default", "timestamp", "DateTime", timestamp_values, "decimal,datetime64", id="timestamp_default"), + pytest.param("timestamp_6", "timestamp(6)", "DateTime64(6)", timestamp_values, "decimal,datetime64", id="timestamp_6"), + pytest.param("datetime_default", "DATETIME NOT NULL", "DateTime64(0)", timestamp_values, + "decimal,datetime64", id="datetime_default"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime64(6)", timestamp_values, + "decimal,datetime64", id="datetime_6_1"), # right now precision bigger than 39 is not supported by ClickHouse's Decimal, hence fall back to String - ("decimal_40_6", "decimal(40, 6) NOT NULL", "String", decimal_values, - "decimal,datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, ""), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, - "decimal"), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, ""), + pytest.param("decimal_40_6", "decimal(40, 6) NOT NULL", "String", decimal_values, + "decimal,datetime64", id="decimal_40_6"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "datetime64", id="decimal_18_6_1"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "", id="decimal_18_6_2"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, + "decimal", id="datetime_6_2"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, "", id="datetime_6_3"), ]) def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, mysql_values, setting_mysql_datatypes_support_level): diff --git a/tests/integration/test_send_request_to_leader_replica/test.py b/tests/integration/test_send_request_to_leader_replica/test.py index 8c58d02d104..721e446ff82 100644 --- a/tests/integration/test_send_request_to_leader_replica/test.py +++ b/tests/integration/test_send_request_to_leader_replica/test.py @@ -39,12 +39,12 @@ def started_cluster(): @pytest.mark.parametrize("table,query,expected,n1,n2", [ - ("sometable", "ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2), - ("sometable", "TRUNCATE TABLE sometable", '0', node1, node2), - ("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2), - ("someothertable", "ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4), - ("someothertable", "TRUNCATE TABLE someothertable", '0', node3, node4), - ("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4), + pytest.param("sometable", "ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2, id="case1"), + pytest.param("sometable", "TRUNCATE TABLE sometable", '0', node1, node2, id="case2"), + pytest.param("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2, id="case3"), + pytest.param("someothertable", "ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4, id="case4"), + pytest.param("someothertable", "TRUNCATE TABLE someothertable", '0', node3, node4, id="case5"), + pytest.param("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4, id="case6"), ]) def test_alter_table_drop_partition(started_cluster, table, query, expected, n1, n2): to_insert = '''\ diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 7e01b919589..254447478f9 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -73,12 +73,12 @@ def wait_parts_mover(node, table, *args, **kwargs): @pytest.mark.parametrize("name,engine,alter", [ - ("mt_test_rule_with_invalid_destination", "MergeTree()", 0), - ("replicated_mt_test_rule_with_invalid_destination", - "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 0), - ("mt_test_rule_with_invalid_destination", "MergeTree()", 1), - ("replicated_mt_test_rule_with_invalid_destination", - "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1), + pytest.param("mt_test_rule_with_invalid_destination", "MergeTree()", 0, id="case0"), + pytest.param("replicated_mt_test_rule_with_invalid_destination", + "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 0, id="case1"), + pytest.param("mt_test_rule_with_invalid_destination", "MergeTree()", 1, id="case2"), + pytest.param("replicated_mt_test_rule_with_invalid_destination", + "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1, id="case3"), ]) def test_rule_with_invalid_destination(started_cluster, name, engine, alter): name = unique_table_name(name) @@ -136,12 +136,12 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_inserts_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_inserts_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_do_not_work', '1')", 0), - ("mt_test_inserts_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_inserts_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1), + pytest.param("mt_test_inserts_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_inserts_to_disk_do_not_work"), + pytest.param("replicated_mt_test_inserts_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_do_not_work', '1')", 0, id="replicated_mt_test_inserts_to_disk_do_not_work"), + pytest.param("mt_test_inserts_to_disk_work", "MergeTree()", 1, id="mt_test_inserts_to_disk_work_1"), + pytest.param("replicated_mt_test_inserts_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1, id="replicated_mt_test_inserts_to_disk_work_1"), ]) def test_inserts_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -176,9 +176,9 @@ def test_inserts_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_work_after_storage_policy_change", "MergeTree()"), - ("replicated_mt_test_moves_work_after_storage_policy_change", - "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')"), + pytest.param("mt_test_moves_work_after_storage_policy_change", "MergeTree()", id="mt_test_moves_work_after_storage_policy_change"), + pytest.param("replicated_mt_test_moves_work_after_storage_policy_change", + "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')", id="replicated_mt_test_moves_work_after_storage_policy_change"), ]) def test_moves_work_after_storage_policy_change(started_cluster, name, engine): name = unique_table_name(name) @@ -224,12 +224,12 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_moves_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_moves_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_do_not_work', '1')", 0), - ("mt_test_moves_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_moves_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1), + pytest.param("mt_test_moves_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_moves_to_disk_do_not_work"), + pytest.param("replicated_mt_test_moves_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_do_not_work', '1')", 0, id="replicated_mt_test_moves_to_disk_do_not_work"), + pytest.param("mt_test_moves_to_disk_work", "MergeTree()", 1, id="mt_test_moves_to_disk_work"), + pytest.param("replicated_mt_test_moves_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1, id="replicated_mt_test_moves_to_disk_work"), ]) def test_moves_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -275,9 +275,9 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_to_volume_work", "MergeTree()"), - ("replicated_mt_test_moves_to_volume_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')"), + pytest.param("mt_test_moves_to_volume_work", "MergeTree()", id="mt_test_moves_to_volume_work"), + pytest.param("replicated_mt_test_moves_to_volume_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')", id="replicated_mt_test_moves_to_volume_work"), ]) def test_moves_to_volume_work(started_cluster, name, engine): name = unique_table_name(name) @@ -322,12 +322,12 @@ def test_moves_to_volume_work(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_inserts_to_volume_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_inserts_to_volume_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_do_not_work', '1')", 0), - ("mt_test_inserts_to_volume_work", "MergeTree()", 1), - ("replicated_mt_test_inserts_to_volume_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1), + pytest.param("mt_test_inserts_to_volume_do_not_work", "MergeTree()", 0, id="mt_test_inserts_to_volume_do_not_work"), + pytest.param("replicated_mt_test_inserts_to_volume_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_do_not_work', '1')", 0, id="replicated_mt_test_inserts_to_volume_do_not_work"), + pytest.param("mt_test_inserts_to_volume_work", "MergeTree()", 1, id="mt_test_inserts_to_volume_work"), + pytest.param("replicated_mt_test_inserts_to_volume_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1, id="replicated_mt_test_inserts_to_volume_work"), ]) def test_inserts_to_volume_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -366,9 +366,9 @@ def test_inserts_to_volume_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_to_disk_eventually_work", "MergeTree()"), - ("replicated_mt_test_moves_to_disk_eventually_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')"), + pytest.param("mt_test_moves_to_disk_eventually_work", "MergeTree()", id="mt_test_moves_to_disk_eventually_work"), + pytest.param("replicated_mt_test_moves_to_disk_eventually_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')", id="replicated_mt_test_moves_to_disk_eventually_work"), ]) def test_moves_to_disk_eventually_work(started_cluster, name, engine): name = unique_table_name(name) @@ -460,12 +460,12 @@ def test_replicated_download_ttl_info(started_cluster): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_merges_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_merges_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_do_not_work', '1')", 0), - ("mt_test_merges_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_merges_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1), + pytest.param("mt_test_merges_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_merges_to_disk_do_not_work"), + pytest.param("replicated_mt_test_merges_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_do_not_work', '1')", 0, id="mt_test_merges_to_disk_do_not_work"), + pytest.param("mt_test_merges_to_disk_work", "MergeTree()", 1, id="mt_test_merges_to_disk_work"), + pytest.param("replicated_mt_test_merges_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1, id="replicated_mt_test_merges_to_disk_work"), ]) def test_merges_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -524,9 +524,9 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_merges_with_full_disk_work", "MergeTree()"), - ("replicated_mt_test_merges_with_full_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')"), + pytest.param("mt_test_merges_with_full_disk_work", "MergeTree()", id="mt_test_merges_with_full_disk_work"), + pytest.param("replicated_mt_test_merges_with_full_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')", id="replicated_mt_test_merges_with_full_disk_work"), ]) def test_merges_with_full_disk_work(started_cluster, name, engine): name = unique_table_name(name) @@ -596,12 +596,12 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_moves_after_merges_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_moves_after_merges_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')", 0), - ("mt_test_moves_after_merges_work", "MergeTree()", 1), - ("replicated_mt_test_moves_after_merges_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1), + pytest.param("mt_test_moves_after_merges_do_not_work", "MergeTree()", 0, id="mt_test_moves_after_merges_do_not_work"), + pytest.param("replicated_mt_test_moves_after_merges_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')", 0, id="replicated_mt_test_moves_after_merges_do_not_work"), + pytest.param("mt_test_moves_after_merges_work", "MergeTree()", 1, id="mt_test_moves_after_merges_work"), + pytest.param("replicated_mt_test_moves_after_merges_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1, id="replicated_mt_test_moves_after_merges_work"), ]) def test_moves_after_merges_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -654,18 +654,18 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine,positive,bar", [ - ("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "DELETE"), - ("replicated_mt_test_moves_after_alter_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "DELETE"), - ("mt_test_moves_after_alter_work", "MergeTree()", 1, "DELETE"), - ("replicated_mt_test_moves_after_alter_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "DELETE"), - ("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "TO DISK 'external'"), - ("replicated_mt_test_moves_after_alter_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "TO DISK 'external'"), - ("mt_test_moves_after_alter_work", "MergeTree()", 1, "TO DISK 'external'"), - ("replicated_mt_test_moves_after_alter_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'"), + pytest.param("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "DELETE", id="mt_negative"), + pytest.param("replicated_mt_test_moves_after_alter_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "DELETE", id="repicated_negative"), + pytest.param("mt_test_moves_after_alter_work", "MergeTree()", 1, "DELETE", id="mt_positive"), + pytest.param("replicated_mt_test_moves_after_alter_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "DELETE", id="repicated_positive"), + pytest.param("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "TO DISK 'external'", id="mt_external_negative"), + pytest.param("replicated_mt_test_moves_after_alter_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "TO DISK 'external'", id="replicated_external_negative"), + pytest.param("mt_test_moves_after_alter_work", "MergeTree()", 1, "TO DISK 'external'", id="mt_external_positive"), + pytest.param("replicated_mt_test_moves_after_alter_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'", id="replicated_external_positive"), ]) def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, bar): name = unique_table_name(name) @@ -704,9 +704,9 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, b @pytest.mark.parametrize("name,engine", [ - ("mt_test_materialize_ttl_in_partition", "MergeTree()"), - ("replicated_mt_test_materialize_ttl_in_partition", - "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')"), + pytest.param("mt_test_materialize_ttl_in_partition", "MergeTree()", id="mt"), + pytest.param("replicated_mt_test_materialize_ttl_in_partition", + "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')", id="replicated"), ]) def test_materialize_ttl_in_partition(started_cluster, name, engine): name = unique_table_name(name) @@ -769,12 +769,12 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_alter_multiple_ttls_positive", "MergeTree()", True), - ("mt_replicated_test_alter_multiple_ttls_positive", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_positive', '1')", True), - ("mt_test_alter_multiple_ttls_negative", "MergeTree()", False), - ("mt_replicated_test_alter_multiple_ttls_negative", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False), + pytest.param("mt_test_alter_multiple_ttls_positive", "MergeTree()", True, id="positive"), + pytest.param("mt_replicated_test_alter_multiple_ttls_positive", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_positive', '1')", True, id="replicated_positive"), + pytest.param("mt_test_alter_multiple_ttls_negative", "MergeTree()", False, id="negative"), + pytest.param("mt_replicated_test_alter_multiple_ttls_negative", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False, id="replicated_negative"), ]) def test_alter_multiple_ttls(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -868,9 +868,9 @@ limitations under the License.""" @pytest.mark.parametrize("name,engine", [ - ("concurrently_altering_ttl_mt", "MergeTree()"), - ("concurrently_altering_ttl_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')",), + pytest.param("concurrently_altering_ttl_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_altering_ttl_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')", id="replicated_mt"), ]) def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): name = unique_table_name(name) @@ -977,8 +977,8 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): @pytest.mark.skip(reason="Flacky test") @pytest.mark.parametrize("name,positive", [ - ("test_double_move_while_select_negative", 0), - ("test_double_move_while_select_positive", 1), + pytest.param("test_double_move_while_select_negative", 0, id="negative"), + pytest.param("test_double_move_while_select_positive", 1, id="positive"), ]) def test_double_move_while_select(started_cluster, name, positive): name = unique_table_name(name) @@ -1038,12 +1038,12 @@ def test_double_move_while_select(started_cluster, name, positive): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_alter_with_merge_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_alter_with_merge_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_do_not_work', '1')", 0), - ("mt_test_alter_with_merge_work", "MergeTree()", 1), - ("replicated_mt_test_alter_with_merge_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1), + pytest.param("mt_test_alter_with_merge_do_not_work", "MergeTree()", 0, id="mt"), + pytest.param("replicated_mt_test_alter_with_merge_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_do_not_work', '1')", 0, id="replicated"), + pytest.param("mt_test_alter_with_merge_work", "MergeTree()", 1, id="mt_work"), + pytest.param("replicated_mt_test_alter_with_merge_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1, id="replicated_work"), ]) def test_alter_with_merge_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -1131,10 +1131,10 @@ limitations under the License.""" @pytest.mark.parametrize("name,dest_type,engine", [ - ("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()"), - ("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()"), - ("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), - ("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), + pytest.param("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()", id="disk"), + pytest.param("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()", id="volume"), + pytest.param("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')", id="replicated_disk"), + pytest.param("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')", id="replicated_volume"), ]) def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): name = unique_table_name(name) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 389e249790f..a4936aee624 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -227,8 +227,8 @@ def optimize_with_retry(node, table_name, retry=20): time.sleep(0.5) @pytest.mark.parametrize("name,engine", [ - ("test_ttl_alter_delete", "MergeTree()"), - ("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')"), + pytest.param("test_ttl_alter_delete", "MergeTree()", id="test_ttl_alter_delete"), + pytest.param("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')", id="test_ttl_alter_delete_replicated"), ]) def test_ttl_alter_delete(started_cluster, name, engine): """Copyright 2019, Altinity LTD From dbf04c2a815927a1e98a378903b45e1770b78d1a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:50:17 +0300 Subject: [PATCH 066/652] fixes --- tests/integration/test_mysql_protocol/test.py | 17 ++++++++--------- tests/integration/test_odbc_interaction/test.py | 4 ++-- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 1e0d3652a91..5b9c180b25e 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -20,8 +20,7 @@ node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "confi "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) -server_port = 9001 - +server_port = cluster.mysql_port @pytest.fixture(scope="module") def server_address(): @@ -64,31 +63,31 @@ def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') + ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') + ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') + ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') def test_mysql_client(mysql_client, server_address): diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 3b860ff7cee..cbd8dd827bf 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -40,10 +40,10 @@ def get_mysql_conn(): for _ in range(15): try: if conn is None: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_ip, port=cluster.mysql_port) else: conn.ping(reconnect=True) - logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + logging.debug(f"MySQL Connection establised: {cluster.mysql_ip}:{cluster.mysql_port}") return conn except Exception as e: errors += [str(e)] From 0b9bdfc5c599798a5ffd4bd0acda3d48c78c9504 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 18:21:39 +0300 Subject: [PATCH 067/652] fix --- .../materialize_with_ddl.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 4740f14bcbf..c9cf93554c8 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -12,7 +12,7 @@ import threading from multiprocessing.dummy import Pool from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=1, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=10, interval_seconds=3): lastest_result = '' for i in range(retry_count): @@ -694,9 +694,10 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") + clickhouse_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") mysql_node.query("CREATE DATABASE kill_mysql_while_insert") mysql_node.query("CREATE TABLE kill_mysql_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") - clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse')".format(service_name)) + clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse') SETTINGS max_wait_time_when_mysql_unavailable=-1".format(service_name)) check_query(clickhouse_node, "SHOW TABLES FROM kill_mysql_while_insert FORMAT TSV", 'test\n') try: @@ -711,7 +712,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): clickhouse_node.cluster.restart_service(service_name) finally: with pytest.raises(QueryRuntimeException) as exception: - time.sleep(5) + time.sleep(2) clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test") mysql_node.alloc_connection() From f14d4be88efa37a235c08940f0c1a4d5ac7d5be6 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 19:44:08 +0300 Subject: [PATCH 068/652] change postgres --- tests/integration/helpers/cluster.py | 13 ++++++++++--- .../test_dictionaries_postgresql/test.py | 18 +++++++++--------- .../integration/test_odbc_interaction/test.py | 14 +++++++------- .../test_postgresql_database_engine/test.py | 18 +++++++++--------- .../test_postgresql_protocol/test.py | 4 ++-- .../test_storage_postgresql/test.py | 16 ++++++++-------- 6 files changed, 45 insertions(+), 38 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3a4476696ed..d0e999049ea 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -302,8 +302,8 @@ class ClickHouseCluster: self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) env_variables['POSTGRES_HOST'] = self.postgres_host env_variables['POSTGRES_PORT'] = str(self.postgres_port) - env_variables['POSTGRES_LOGS'] = self.postgres_logs_dir - env_variables['POSTGRES2_LOGS'] = self.postgres2_logs_dir + env_variables['POSTGRES_DIR'] = self.postgres_logs_dir + env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir env_variables['POSTGRES_LOGS_FS'] = "bind" self.with_postgres = True @@ -684,7 +684,7 @@ class ClickHouseCluster: self.postgres_ip = self.get_instance_ip(self.postgres_host) self.postgres2_ip = self.get_instance_ip(self.postgres2_host) start = time.time() - for up in [self.postgres_ip, self.postgres2_ip]: + for ip in [self.postgres_ip, self.postgres2_ip]: while time.time() - start < timeout: try: conn = psycopg2.connect(host=ip, port=self.postgres_port, user='postgres', password='mysecretpassword') @@ -908,6 +908,13 @@ class ClickHouseCluster: if self.with_postgres and self.base_postgres_cmd: logging.debug('Setup Postgres') + if os.path.exists(self.postgres_dir): + shutil.rmtree(self.postgres_dir) + os.makedirs(self.postgres_logs_dir) + os.chmod(self.postgres_logs_dir, stat.S_IRWXO) + os.makedirs(self.postgres2_logs_dir) + os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) + subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(30) self.wait_postgres_to_start(30) diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index e5a4af1d91a..86763b52a6f 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -21,11 +21,11 @@ click_dict_table_template = """ ) ENGINE = Dictionary({}) """ -def get_postgres_conn(port, database=False): +def get_postgres_conn(ip, port, database=False): if database == True: - conn_string = "host='localhost' port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(port) + conn_string = "host={} port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(ip, port) else: - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(port) + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) @@ -73,10 +73,10 @@ def started_cluster(): def test_load_dictionaries(started_cluster): - conn = get_postgres_conn(database=True, port=started_cluster.postgres_port) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(cursor, table_name) + create_and_fill_postgres_table(cursor, table_name, port=started_cluster.postgres_port, host=started_cluster.postgres_ip) create_dict(table_name) dict_name = 'dict0' @@ -91,10 +91,10 @@ def test_load_dictionaries(started_cluster): def test_invalidate_query(started_cluster): - conn = get_postgres_conn(database=True, port=started_cluster.postgres_port) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(cursor, table_name) + create_and_fill_postgres_table(cursor, table_name, port=started_cluster.postgres_port, host=started_cluster.postgres_ip) # invalidate query: SELECT value FROM test0 WHERE id = 0 dict_name = 'dict0' @@ -128,9 +128,9 @@ def test_invalidate_query(started_cluster): def test_dictionary_with_replicas(started_cluster): - conn1 = get_postgres_conn(port=5432, database=True) + conn1 = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor1 = conn1.cursor() - conn2 = get_postgres_conn(port=5441, database=True) + conn2 = get_postgres_conn(ip=started_cluster.postgres2_ip, port=started_cluster.postgres_port, database=True) cursor2 = conn2.cursor() create_postgres_table(cursor1, 'test1') diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index cbd8dd827bf..da6abcc2751 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -64,8 +64,8 @@ def create_mysql_table(conn, table_name): cursor.execute(create_table_sql_template.format(table_name)) -def get_postgres_conn(): - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) +def get_postgres_conn(started_cluster): + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(started_cluster.postgres_ip, started_cluster.postgres_port) errors = [] for _ in range(15): try: @@ -112,7 +112,7 @@ def started_cluster(): create_mysql_db(mysql_conn, 'clickhouse') print("mysql database created") - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(cluster) print("postgres connection received") create_postgres_db(postgres_conn, 'clickhouse') @@ -294,7 +294,7 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(1, 'hello'),(2, 'world')") @@ -304,7 +304,7 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(3, 'xxx')") @@ -318,7 +318,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): def test_postgres_insert(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) conn.cursor().execute("truncate table clickhouse.test_table") # Also test with Servername containing '.' and '-' symbols (defined in @@ -377,7 +377,7 @@ def test_bridge_dies_with_parent(started_cluster): def test_odbc_postgres_date_data_type(started_cluster): - conn = get_postgres_conn(); + conn = get_postgres_conn(started_cluster); cursor = conn.cursor() cursor.execute("CREATE TABLE IF NOT EXISTS clickhouse.test_date (column1 integer, column2 date)") diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index 2cb148db6ef..e89f1109c3a 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -14,11 +14,11 @@ postgres_table_template = """ id Integer NOT NULL, value Integer, PRIMARY KEY (id)) """ -def get_postgres_conn(database=False): +def get_postgres_conn(cluster, database=False): if database == True: - conn_string = "host='localhost' port={} dbname='test_database' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port={cluster.postgres_port} dbname='test_database' user='postgres' password='mysecretpassword'" else: - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port={cluster.postgres_port} user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -36,7 +36,7 @@ def create_postgres_table(cursor, table_name): def started_cluster(): try: cluster.start() - conn = get_postgres_conn() + conn = get_postgres_conn(cluster) cursor = conn.cursor() create_postgres_db(cursor, 'test_database') yield cluster @@ -47,7 +47,7 @@ def started_cluster(): def test_postgres_database_engine_with_postgres_ddl(started_cluster): # connect to database as well - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -68,7 +68,7 @@ def test_postgres_database_engine_with_postgres_ddl(started_cluster): def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -94,7 +94,7 @@ def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): def test_postgresql_database_engine_queries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -114,7 +114,7 @@ def test_postgresql_database_engine_queries(started_cluster): def test_get_create_table_query_with_multidim_arrays(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -147,7 +147,7 @@ def test_get_create_table_query_with_multidim_arrays(started_cluster): def test_postgresql_database_engine_table_cache(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 60cf8a9239b..39ccb273299 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=os.environ.get("DOCKER_API_VERSION"), timeout=180).containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') @@ -65,7 +65,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') def test_psql_is_ready(psql_server): diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index d92a32062fc..5215d8f55ba 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -9,11 +9,11 @@ from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=[], with_postgres=True) -def get_postgres_conn(database=False): +def get_postgres_conn(cluster, database=False): if database == True: - conn_string = "host='localhost' port='{}' dbname='clickhouse' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port='{cluster.postgres_port}' dbname='clickhouse' user='postgres' password='mysecretpassword'" else: - conn_string = "host='localhost' port='{}' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port='{cluster.postgres_port}' user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -29,7 +29,7 @@ def create_postgres_db(conn, name): def started_cluster(): try: cluster.start() - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(cluster) print("postgres connected") create_postgres_db(postgres_conn, 'clickhouse') yield cluster @@ -39,10 +39,10 @@ def started_cluster(): def test_postgres_select_insert(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() table_name = 'test_many' - table = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) + table = f'''postgresql('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'clickhouse', '{table_name}', 'postgres', 'mysecretpassword')''' cursor.execute('CREATE TABLE IF NOT EXISTS {} (a integer, b text, c integer)'.format(table_name)) result = node1.query(''' @@ -59,7 +59,7 @@ def test_postgres_select_insert(started_cluster): def test_postgres_conversions(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( @@ -134,7 +134,7 @@ def test_postgres_conversions(started_cluster): def test_non_default_scema(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') cursor.execute('CREATE TABLE test_schema.test_table (a integer)') From fd32609f380ef422a54c686ff9bbd468a11d74bf Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 19:47:59 +0300 Subject: [PATCH 069/652] run parallel tests --- tests/integration/ci-runner.py | 27 +++++-- tests/integration/parallel.txt | 125 +++++++++++++++++++++++++++++++++ 2 files changed, 148 insertions(+), 4 deletions(-) create mode 100644 tests/integration/parallel.txt diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index f758457ada0..fb1d85a2a3f 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -257,6 +257,17 @@ class ClickhouseIntegrationTestsRunner: all_tests.append(line.strip()) return list(sorted(all_tests)) + def _get_parallel_tests(self, repo_path): + parallel_tests_file_path = "{}/tests/integration/all_tests.txt".format(repo_path) + if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: + raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) + + parallel_tests = [] + with open(parallel_tests_file_path, "r") as parallel_tests_file: + for line in parallel_tests_file: + all_tests.append(line.strip()) + return list(sorted(parallel_tests)) + def group_test_by_file(self, tests): result = {} for test in tests: @@ -326,7 +337,7 @@ class ClickhouseIntegrationTestsRunner: test_names.add(test_name) test_cmd = ' '.join([test for test in sorted(test_names)]) - cmd = "cd {}/tests/integration && ./runner {} '-ss {} -rfEp --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && ./runner {} -t {} --parallel 10 '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: @@ -425,7 +436,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Dump iptables before run %s", subprocess.check_output("iptables -L", shell=True)) all_tests = self._get_all_tests(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) - grouped_tests = self.group_test_by_file(all_tests) + filtered_parallel_tests = filter(lambda test: test in all_tests, parallel_tests) + filtered_unparallel_tests = filter(lambda test: test not in parallel_tests, all_tests) + not_found_tests = filter(lambda test: test not in all_tests, parallel_tests) + logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_unparallel_tests)) + logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) + + grouped_tests = self.group_test_by_file(filtered_unparallel_tests) logging.info("Found %s tests groups", len(grouped_tests)) counters = { @@ -436,7 +453,9 @@ class ClickhouseIntegrationTestsRunner: tests_times = defaultdict(float) logs = [] - items_to_run = list(grouped_tests.items()) + items_to_run = list() + items_to_run += list(("parallel", filtered_parallel_tests)) + items_to_run += list(grouped_tests.items()) logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): @@ -444,7 +463,7 @@ class ClickhouseIntegrationTestsRunner: random.shuffle(items_to_run) for group, tests in items_to_run: - logging.info("Running test group %s countaining %s tests", group, len(tests)) + logging.info("Running test group %s containing %s tests", group, len(tests)) group_counters, group_test_times, log_name, log_path = self.run_test_group(repo_path, group, tests, MAX_RETRY) total_tests = 0 for counter, value in group_counters.items(): diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt new file mode 100644 index 00000000000..fac2f589358 --- /dev/null +++ b/tests/integration/parallel.txt @@ -0,0 +1,125 @@ +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache] +test_dictionaries_mysql/test.py::test_load_mysql_dictionaries +test_disabled_mysql_server/test.py::test_disabled_mysql_server +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic] +test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary] +test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic] +test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary] +test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic] +test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary] +test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic] +test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary] +test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic] +test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0] +test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1] +test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0] +test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1] +test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0] +test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1] +test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0] +test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1] +test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database +test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database +test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database +test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database +test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine +test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine +test_mysql_database_engine/test.py::test_mysql_types[common_types_1] +test_mysql_database_engine/test.py::test_mysql_types[common_types_2] +test_mysql_database_engine/test.py::test_mysql_types[common_types_3] +test_mysql_database_engine/test.py::test_mysql_types[common_types_4] +test_mysql_database_engine/test.py::test_mysql_types[common_types_5] +test_mysql_database_engine/test.py::test_mysql_types[common_types_6] +test_mysql_database_engine/test.py::test_mysql_types[common_types_7] +test_mysql_database_engine/test.py::test_mysql_types[common_types_80] +test_mysql_database_engine/test.py::test_mysql_types[common_types_81] +test_mysql_database_engine/test.py::test_mysql_types[common_types_9] +test_mysql_database_engine/test.py::test_mysql_types[common_types_10] +test_mysql_database_engine/test.py::test_mysql_types[common_types_11] +test_mysql_database_engine/test.py::test_mysql_types[common_types_12] +test_mysql_database_engine/test.py::test_mysql_types[common_types_13] +test_mysql_database_engine/test.py::test_mysql_types[common_types_14] +test_mysql_database_engine/test.py::test_mysql_types[common_types_15] +test_mysql_database_engine/test.py::test_mysql_types[common_types_16] +test_mysql_database_engine/test.py::test_mysql_types[common_types_17] +test_mysql_database_engine/test.py::test_mysql_types[common_types_18] +test_mysql_database_engine/test.py::test_mysql_types[common_types_19] +test_mysql_database_engine/test.py::test_mysql_types[common_types_20] +test_mysql_database_engine/test.py::test_mysql_types[decimal_1] +test_mysql_database_engine/test.py::test_mysql_types[decimal_2] +test_mysql_database_engine/test.py::test_mysql_types[decimal_3] +test_mysql_database_engine/test.py::test_mysql_types[decimal_4] +test_mysql_database_engine/test.py::test_mysql_types[timestamp_default] +test_mysql_database_engine/test.py::test_mysql_types[timestamp_6] +test_mysql_database_engine/test.py::test_mysql_types[datetime_default] +test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1] +test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6] +test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1] +test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2] +test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2] +test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3] +test_mysql_protocol/test.py::test_mysql_client +test_mysql_protocol/test.py::test_mysql_client_exception +test_mysql_protocol/test.py::test_mysql_affected_rows +test_mysql_protocol/test.py::test_mysql_replacement_query +test_mysql_protocol/test.py::test_mysql_explain +test_mysql_protocol/test.py::test_mysql_federated +test_mysql_protocol/test.py::test_mysql_set_variables +test_mysql_protocol/test.py::test_python_client +test_mysql_protocol/test.py::test_golang_client +test_mysql_protocol/test.py::test_php_client +test_mysql_protocol/test.py::test_mysqljs_client +test_mysql_protocol/test.py::test_java_client +test_mysql_protocol/test.py::test_types +test_odbc_interaction/test.py::test_mysql_simple_select_works +test_odbc_interaction/test.py::test_mysql_insert +test_storage_mysql/test.py::test_many_connections +test_storage_mysql/test.py::test_insert_select +test_storage_mysql/test.py::test_replace_select +test_storage_mysql/test.py::test_insert_on_duplicate_select +test_storage_mysql/test.py::test_where +test_storage_mysql/test.py::test_table_function +test_storage_mysql/test.py::test_binary_type +test_storage_mysql/test.py::test_enum_type From 880a03a9dfd3494f68b69eac835d6060cd8e50e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sat, 20 Mar 2021 22:44:58 +0300 Subject: [PATCH 070/652] fix --- tests/integration/ci-runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index fb1d85a2a3f..139d73a9d34 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -435,6 +435,7 @@ class ClickhouseIntegrationTestsRunner: self._install_clickhouse(build_path) logging.info("Dump iptables before run %s", subprocess.check_output("iptables -L", shell=True)) all_tests = self._get_all_tests(repo_path) + parallel_tests = self._get_parallel_tests(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) filtered_parallel_tests = filter(lambda test: test in all_tests, parallel_tests) filtered_unparallel_tests = filter(lambda test: test not in parallel_tests, all_tests) From 360887893b48caaff1e208d235c9f25cf1fa5b8c Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Mar 2021 14:51:47 +0300 Subject: [PATCH 071/652] fix --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index c2152cc6759..9633d4294ef 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -271,7 +271,7 @@ class ClickhouseIntegrationTestsRunner: parallel_tests = [] with open(parallel_tests_file_path, "r") as parallel_tests_file: for line in parallel_tests_file: - all_tests.append(line.strip()) + parallel_tests.append(line.strip()) return list(sorted(parallel_tests)) def group_test_by_file(self, tests): From ee84ab2a4805cb7ca1a97041db5ff8e578647ed9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Mar 2021 18:33:46 +0300 Subject: [PATCH 072/652] fixes --- tests/integration/ci-runner.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 9633d4294ef..93ca0435720 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -468,9 +468,9 @@ class ClickhouseIntegrationTestsRunner: all_tests = self._get_all_tests(repo_path) parallel_tests = self._get_parallel_tests(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) - filtered_parallel_tests = filter(lambda test: test in all_tests, parallel_tests) - filtered_unparallel_tests = filter(lambda test: test not in parallel_tests, all_tests) - not_found_tests = filter(lambda test: test not in all_tests, parallel_tests) + filtered_parallel_tests = list(filter(lambda test: test in all_tests, parallel_tests)) + filtered_unparallel_tests = list(filter(lambda test: test not in parallel_tests, all_tests)) + not_found_tests = list(filter(lambda test: test not in all_tests, parallel_tests)) logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_unparallel_tests)) logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) From ecc58a76d768d45cb84b30ad5c4aacca18567108 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Mar 2021 06:35:08 +0300 Subject: [PATCH 073/652] fix --- tests/integration/ci-runner.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 93ca0435720..4196d97d2c6 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -264,7 +264,7 @@ class ClickhouseIntegrationTestsRunner: return list(sorted(all_tests)) def _get_parallel_tests(self, repo_path): - parallel_tests_file_path = "{}/tests/integration/all_tests.txt".format(repo_path) + parallel_tests_file_path = "{}/tests/integration/parallel.txt".format(repo_path) if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) @@ -475,6 +475,7 @@ class ClickhouseIntegrationTestsRunner: logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) grouped_tests = self.group_test_by_file(filtered_unparallel_tests) + grouped_tests["parallel"] = filtered_parallel_tests logging.info("Found %s tests groups", len(grouped_tests)) counters = { @@ -487,9 +488,7 @@ class ClickhouseIntegrationTestsRunner: tests_times = defaultdict(float) logs = [] - items_to_run = list() - items_to_run += list(("parallel", filtered_parallel_tests)) - items_to_run += list(grouped_tests.items()) + items_to_run = list(grouped_tests.items()) logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): From e79b5b57fdc2a936045fb0a1d448415435d5cba7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Mar 2021 14:35:45 +0300 Subject: [PATCH 074/652] try export dockerd.log --- docker/test/integration/runner/dockerd-entrypoint.sh | 2 +- tests/integration/ci-runner.py | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 9f04dde720d..a066e914241 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,7 +1,7 @@ #!/bin/bash set -e -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.0.0.0/8,size=24 &>/var/log/somefile & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.10.0.0/16,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4196d97d2c6..9a82516b756 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -15,6 +15,7 @@ MAX_RETRY = 2 SLEEP_BETWEEN_RETRIES = 5 CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge" +DOCKERD_LOGS_PATH = "/ClickHouse/tests/integration/dockerd.log" TRIES_COUNT = 10 MAX_TIME_SECONDS = 3600 @@ -446,6 +447,10 @@ class ClickhouseIntegrationTestsRunner: self._compress_logs("{}/tests/integration".format(repo_path), test_logs) logging.info("Compression finished") + result_path_dockerd_logs = os.path.join(str(self.path()), "dockerd.log") + if os.path.exists(result_path_dockerd_logs): + shutil.copy(DOCKERD_LOGS_PATH, result_path_dockerd_logs) + test_result = [] for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"): if state == "PASSED": @@ -518,6 +523,10 @@ class ClickhouseIntegrationTestsRunner: self._compress_logs("{}/tests/integration".format(repo_path), test_logs) logging.info("Compression finished") + result_path_dockerd_logs = os.path.join(str(self.path()), "dockerd.log") + if os.path.exists(result_path_dockerd_logs): + shutil.copy(DOCKERD_LOGS_PATH, result_path_dockerd_logs) + if counters["FAILED"] or counters["ERROR"]: logging.info("Overall status failure, because we have tests in FAILED or ERROR state") result_state = "failure" From af1c14b4ee6b25df2d73d4be446fae9cc1717e7d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Mar 2021 18:42:36 +0300 Subject: [PATCH 075/652] try other network --- docker/test/integration/runner/dockerd-entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index a066e914241..037feb7ff59 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,7 +1,7 @@ #!/bin/bash set -e -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.10.0.0/16,size=24 &>/ClickHouse/tests/integration/dockerd.log & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 From 662d494d2051ae5caab134151ad31fd72fa214f3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Mar 2021 16:23:23 +0300 Subject: [PATCH 076/652] fixes --- tests/integration/test_mysql_protocol/test.py | 8 +++---- tests/integration/test_storage_mysql/test.py | 22 +++++++++---------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 5b9c180b25e..4a33dcc7e3a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -20,7 +20,7 @@ node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "confi "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) -server_port = cluster.mysql_port +server_port = 9001 @pytest.fixture(scope="module") def server_address(): @@ -70,7 +70,7 @@ def golang_container(): def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( - ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') @@ -78,7 +78,7 @@ def php_container(): def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( - ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') @@ -86,7 +86,7 @@ def nodejs_container(): def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( - ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index aea6e0816d0..16d1b6048e6 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -24,7 +24,7 @@ def started_cluster(): try: cluster.start() - conn = get_mysql_conn() + conn = get_mysql_conn(cluster) ## create mysql db and table create_mysql_db(conn, 'clickhouse') yield cluster @@ -35,7 +35,7 @@ def started_cluster(): def test_many_connections(started_cluster): table_name = 'test_many_connections' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -54,7 +54,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_select(started_cluster): table_name = 'test_insert_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -70,7 +70,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_replace_select(started_cluster): table_name = 'test_replace_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -89,7 +89,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_on_duplicate_select(started_cluster): table_name = 'test_insert_on_duplicate_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -108,7 +108,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_where(started_cluster): table_name = 'test_where' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); @@ -128,7 +128,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, 'table_function') table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' @@ -149,7 +149,7 @@ def test_table_function(started_cluster): conn.close() def test_binary_type(started_cluster): - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') @@ -158,7 +158,7 @@ def test_binary_type(started_cluster): def test_enum_type(started_cluster): table_name = 'test_enum_type' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); @@ -168,8 +168,8 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' conn.close() -def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_host, port=cluster.mysql_port) +def get_mysql_conn(started_cluster): + conn = pymysql.connect(user='root', password='clickhouse', host=started_cluster.mysql_ip, port=started_cluster.mysql_port) return conn From 815a2ad7f2c2caa4f30b4d11c63e32dbb16fa6e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Mar 2021 16:50:32 +0300 Subject: [PATCH 077/652] fix --- tests/integration/test_mysql_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 4a33dcc7e3a..26f44fd9491 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -217,7 +217,7 @@ def test_mysql_replacement_query(mysql_client, server_address): --password=123 -e "select database();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout.decode() == 'DATABASE()\ndefault\n' + assert stdout.decode() == 'database()\ndefault\n' code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default From 8b070495397d549e0a65da9480c24c9e002573b8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 25 Mar 2021 07:38:24 +0300 Subject: [PATCH 078/652] undo --- tests/integration/test_mysql_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 26f44fd9491..4a33dcc7e3a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -217,7 +217,7 @@ def test_mysql_replacement_query(mysql_client, server_address): --password=123 -e "select database();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout.decode() == 'database()\ndefault\n' + assert stdout.decode() == 'DATABASE()\ndefault\n' code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default From 3f9b81b0a7dc6c30786df3542465b52c74dd6278 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 26 Mar 2021 13:08:24 +0300 Subject: [PATCH 079/652] tune --- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/test_multiple_disks/test.py | 2 +- tests/integration/test_storage_postgresql/test.py | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c3c7710b366..0a64f8fdc7b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -895,7 +895,7 @@ class ClickHouseCluster: os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(60) + self.wait_mysql_to_start(120) if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') @@ -904,7 +904,7 @@ class ClickHouseCluster: os.makedirs(self.mysql8_logs_dir) os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql8_cmd + common_opts) - self.wait_mysql8_to_start(120) + self.wait_mysql8_to_start(180) if self.with_postgres and self.base_postgres_cmd: logging.debug('Setup Postgres') diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 1d672f8f7ce..7ad7fe92b0f 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -926,7 +926,7 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): tasks.append(p.apply_async(alter_drop, (100,))) for task in tasks: - task.get(timeout=60) + task.get(timeout=120) assert node1.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 8e3ec6c6e76..ceb2842697f 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -168,7 +168,7 @@ def test_non_default_scema(started_cluster): def test_concurrent_queries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query(''' From 9e02588505046431c8a0cbf72bbe389db0ba26a3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 26 Mar 2021 15:53:53 +0300 Subject: [PATCH 080/652] more parallel tests --- tests/integration/parallel.txt | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt index fac2f589358..67daeced0f1 100644 --- a/tests/integration/parallel.txt +++ b/tests/integration/parallel.txt @@ -11,6 +11,9 @@ test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache] test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed] test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache] test_dictionaries_mysql/test.py::test_load_mysql_dictionaries +test_dictionaries_postgresql/test.py::test_load_dictionaries +test_dictionaries_postgresql/test.py::test_invalidate_query +test_dictionaries_postgresql/test.py::test_dictionary_with_replicas test_disabled_mysql_server/test.py::test_disabled_mysql_server test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary] test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic] @@ -115,6 +118,19 @@ test_mysql_protocol/test.py::test_java_client test_mysql_protocol/test.py::test_types test_odbc_interaction/test.py::test_mysql_simple_select_works test_odbc_interaction/test.py::test_mysql_insert +test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema +test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow +test_odbc_interaction/test.py::test_postgres_insert +test_odbc_interaction/test.py::test_odbc_postgres_date_data_type +test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl +test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl +test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries +test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays +test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache +test_postgresql_protocol/test.py::test_psql_is_ready +test_postgresql_protocol/test.py::test_psql_client +test_postgresql_protocol/test.py::test_python_client +test_postgresql_protocol/test.py::test_java_client test_storage_mysql/test.py::test_many_connections test_storage_mysql/test.py::test_insert_select test_storage_mysql/test.py::test_replace_select @@ -123,3 +139,7 @@ test_storage_mysql/test.py::test_where test_storage_mysql/test.py::test_table_function test_storage_mysql/test.py::test_binary_type test_storage_mysql/test.py::test_enum_type +test_storage_postgresql/test.py::test_postgres_select_insert +test_storage_postgresql/test.py::test_postgres_conversions +test_storage_postgresql/test.py::test_non_default_scema +test_storage_postgresql/test.py::test_concurrent_queries From df8013c22d34317a373f36b305ecfda5119dfc08 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 26 Mar 2021 21:46:42 +0300 Subject: [PATCH 081/652] fix --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 60b6ef6bf0c..0d92bdf8493 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -424,7 +424,7 @@ def test_custom_auth_headers(started_cluster): instance.query( "CREATE TABLE test ({table_format}) ENGINE = S3('http://resolver:8080/{bucket}/{file}', 'CSV')".format( - bucket=cluster.minio_restricted_bucket, + bucket=started_cluster.minio_restricted_bucket, file=filename, table_format=table_format )) From 7b7580204c000890eaa16baf301040d1e07b6250 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sun, 28 Mar 2021 00:26:53 +0300 Subject: [PATCH 082/652] fix --- tests/integration/test_storage_kafka/test.py | 34 +++++++++----------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 60ca3f5a790..5e13cdfde68 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2557,26 +2557,23 @@ def test_premature_flush_on_eof(kafka_cluster): ''') -@pytest.mark.timeout(120) +@pytest.mark.timeout(180) def test_kafka_unavailable(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce(kafka_cluster, 'test_kafka_unavailable', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) kafka_cluster.pause_container('kafka1') instance.query(''' - DROP TABLE IF EXISTS test.destination_kafka_unavailable; - DROP TABLE IF EXISTS test.test_kafka_unavailable; - CREATE TABLE test.test_kafka_unavailable (key UInt64, value UInt64) + CREATE TABLE test.test_bad_reschedule (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_kafka_unavailable', - kafka_group_name = 'test_kafka_unavailable', + kafka_topic_list = 'test_bad_reschedule', + kafka_group_name = 'test_bad_reschedule', kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + kafka_max_block_size = 1000; - CREATE MATERIALIZED VIEW test.destination_kafka_unavailable Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS SELECT key, now() as consume_ts, @@ -2586,19 +2583,20 @@ def test_kafka_unavailable(kafka_cluster): _offset, _partition, _timestamp - FROM test.test_kafka_unavailable; + FROM test.test_bad_reschedule; ''') - instance.query("SELECT * FROM test.test_kafka_unavailable") - - instance.wait_for_log_line('brokers are down') - instance.wait_for_log_line('stalled. Reschedule', repetitions=2) + instance.query("SELECT * FROM test.test_bad_reschedule") + instance.query("SELECT count() FROM test.destination_unavailable") + # enough to trigger issue + time.sleep(30) kafka_cluster.unpause_container('kafka1') - instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 5000 - time.sleep(5) # needed to give time for kafka client in python test to recovery + while int(instance.query("SELECT count() FROM test.destination_unavailable")) < 20000: + print("Waiting for consume") + time.sleep(1) + @pytest.mark.timeout(180) def test_kafka_issue14202(kafka_cluster): From 808da339136a75bf5599b235e7b18066fb5d6ab4 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 29 Mar 2021 17:52:51 +0900 Subject: [PATCH 083/652] Include bitset and rename test --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 1 + ...xt_node.reference => 01656_sequence_next_node_long.reference} | 0 ..._sequence_next_node.sql => 01656_sequence_next_node_long.sql} | 0 3 files changed, 1 insertion(+) rename tests/queries/0_stateless/{01656_sequence_next_node.reference => 01656_sequence_next_node_long.reference} (100%) rename tests/queries/0_stateless/{01656_sequence_next_node.sql => 01656_sequence_next_node_long.sql} (100%) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c0ecdadf8b1..2ace80de0a5 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -22,6 +22,7 @@ #include #include +#include namespace DB diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node_long.reference similarity index 100% rename from tests/queries/0_stateless/01656_sequence_next_node.reference rename to tests/queries/0_stateless/01656_sequence_next_node_long.reference diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node_long.sql similarity index 100% rename from tests/queries/0_stateless/01656_sequence_next_node.sql rename to tests/queries/0_stateless/01656_sequence_next_node_long.sql From 24c52414335b9cff55db5c0c9d5dd47ea0b0271c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Apr 2021 18:19:36 +0300 Subject: [PATCH 084/652] Minor style fixes in AggregateFunctionSequenceNextNode --- .../AggregateFunctionSequenceNextNode.cpp | 119 ++++++------- .../AggregateFunctionSequenceNextNode.h | 158 +++++++++--------- 2 files changed, 134 insertions(+), 143 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index c31337e25c6..c3a42703e3d 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -11,8 +11,9 @@ namespace DB { -constexpr size_t MAX_EVENTS_SIZE = 64; -constexpr size_t MIN_REQUIRED_ARGS = 3; +constexpr size_t max_events_size = 64; + +constexpr size_t min_required_args = 3; namespace ErrorCodes { @@ -24,84 +25,72 @@ namespace ErrorCodes namespace { -template -inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl2(const DataTypePtr data_type, const DataTypes & argument_types, SeqBase base) -{ - if (base == HEAD) - return std::make_shared, Direction, HEAD, MIN_REQUIRED_ARGS>>(data_type, argument_types); - else if (base == TAIL) - return std::make_shared, Direction, TAIL, MIN_REQUIRED_ARGS>>(data_type, argument_types); - else if (base == FIRST_MATCH) - return std::make_shared, Direction, FIRST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); - else - return std::make_shared, Direction, LAST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); -} - template -inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl1(const DataTypePtr data_type, const DataTypes & argument_types, SeqDirection direction, SeqBase base) +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl( + const DataTypePtr data_type, const DataTypes & argument_types, SequenceDirection direction, SequenceBase base) { - if (direction == FORWARD) - return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); - else - return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); + return std::make_shared>>( + data_type, argument_types, base, direction, min_required_args); } AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, const DataTypes & argument_types, const Array & parameters) { - assert(max_events <= MAX_EVENTS_SIZE); + assert(max_events <= max_events_size); if (parameters.size() < 2) - throw Exception("Aggregate function " + name + " requires 2 parameters (direction, head)", + throw Exception("Aggregate function '" + name + "' requires 2 parameters (direction, head)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + auto expected_param_type = Field::Types::Which::String; + if (parameters.at(0).getType() != expected_param_type || parameters.at(1).getType() != expected_param_type) + throw Exception("Aggregate function '" + name + "' requires 'String' parameters", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String param_dir = parameters.at(0).safeGet(); - SeqDirection direction; - if (param_dir == "forward") - direction = FORWARD; - else if (param_dir == "backward") - direction = BACKWARD; - else + std::unordered_map seq_dir_mapping{ + {"forward", SequenceDirection::Forward}, + {"backward", SequenceDirection::Backward}, + }; + if (!seq_dir_mapping.contains(param_dir)) throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS}; + SequenceDirection direction = seq_dir_mapping[param_dir]; String param_base = parameters.at(1).safeGet(); - SeqBase base; - if (param_base == "head") - base = HEAD; - else if (param_base == "tail") - base = TAIL; - else if (param_base == "first_match") - base = FIRST_MATCH; - else if (param_base == "last_match") - base = LAST_MATCH; - else + std::unordered_map seq_base_mapping{ + {"head", SequenceBase::Head}, + {"tail", SequenceBase::Tail}, + {"first_match", SequenceBase::FirstMatch}, + {"last_match", SequenceBase::LastMatch}, + }; + if (!seq_base_mapping.contains(param_base)) throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; + SequenceBase base = seq_base_mapping[param_base]; - if (argument_types.size() < MIN_REQUIRED_ARGS) - throw Exception("Aggregate function " + name + " requires at least two arguments.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < MIN_REQUIRED_ARGS + 1) - throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > max_events + MIN_REQUIRED_ARGS) - throw Exception("Aggregate function " + name + " requires at most " + - std::to_string(max_events + MIN_REQUIRED_ARGS) + - " (timestamp, value_column, " + std::to_string(max_events) + " events) arguments.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (argument_types.size() < min_required_args) + throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (const auto * cond_arg = argument_types[1].get()) - { - if (!isUInt8(cond_arg)) - throw Exception{"Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " - + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } + bool is_base_match_type = base == SequenceBase::FirstMatch || base == SequenceBase::LastMatch; + if (is_base_match_type && argument_types.size() < min_required_args + 1) + throw Exception( + "Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (const auto i : ext::range(MIN_REQUIRED_ARGS, argument_types.size())) + if (argument_types.size() > max_events + min_required_args) + throw Exception(fmt::format( + "Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.", + name, max_events + min_required_args, max_events), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (const auto * cond_arg = argument_types[1].get(); cond_arg && !isUInt8(cond_arg)) + throw Exception("Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " + + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + for (const auto i : ext::range(min_required_args, argument_types.size())) { const auto * cond_arg = argument_types[i].get(); if (!isUInt8(cond_arg)) - throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + " of aggregate function " - + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(fmt::format( + "Illegal type '{}' of {} argument of aggregate function '{}', must be UInt8", cond_arg->getName(), i + 1, name), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String) @@ -113,17 +102,17 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt16) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt32) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt64) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.isDate()) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.isDateTime()) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); throw Exception{"Illegal type " + argument_types.front().get()->getName() + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", @@ -143,7 +132,7 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(MAX_EVENTS_SIZE), properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(max_events_size), properties }); factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 2ace80de0a5..c3907a5adee 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -28,18 +28,18 @@ namespace DB { -enum SeqDirection +enum class SequenceDirection { - FORWARD = 0, - BACKWARD = 1 + Forward, + Backward, }; -enum SeqBase +enum SequenceBase { - HEAD = 0, - TAIL = 1, - FIRST_MATCH = 2, - LAST_MATCH = 3 + Head, + Tail, + FirstMatch, + LastMatch, }; /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl @@ -121,7 +121,7 @@ struct NodeString : public NodeBase, MaxEventsSize> } }; -/// TODO : Expends SequenceNextNodeGeneralData to support other types +/// TODO : Support other types than string template struct SequenceNextNodeGeneralData { @@ -150,26 +150,40 @@ struct SequenceNextNodeGeneralData }; /// Implementation of sequenceFirstNode -template +template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { + using Self = SequenceNextNodeImpl; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - static constexpr size_t EventColumn = 2; - static constexpr size_t BaseCondition = 1; + + static constexpr size_t base_cond_column_idx = 1; + static constexpr size_t event_column_idx = 2; + + SequenceBase seq_base_kind; + SequenceDirection seq_direction; + const size_t min_required_args; DataTypePtr & data_type; UInt8 events_size; UInt64 max_elems; - public: - SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( - {data_type_}, {}) + SequenceNextNodeImpl( + const DataTypePtr & data_type_, + const DataTypes & arguments, + SequenceBase seq_base_kind_, + SequenceDirection seq_direction_, + size_t min_required_args_, + UInt64 max_elems_ = std::numeric_limits::max()) + : IAggregateFunctionDataHelper, Self>({data_type_}, {}) + , seq_base_kind(seq_base_kind_) + , seq_direction(seq_direction_) + , min_required_args(min_required_args_) , data_type(this->argument_types[0]) - , events_size(arguments.size() - MinRequiredArgs) + , events_size(arguments.size() - min_required_args) , max_elems(max_elems_) { } @@ -202,7 +216,7 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - Node * node = Node::allocate(*columns[EventColumn], row_num, arena); + Node * node = Node::allocate(*columns[event_column_idx], row_num, arena); const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; @@ -214,11 +228,11 @@ public: /// + 4 (bit of event3) node->events_bitset.reset(); for (UInt8 i = 0; i < events_size; ++i) - if (assert_cast *>(columns[MinRequiredArgs + i])->getData()[row_num]) + if (assert_cast *>(columns[min_required_args + i])->getData()[row_num]) node->events_bitset.set(i); node->event_time = timestamp; - node->can_be_base = assert_cast *>(columns[BaseCondition])->getData()[row_num]; + node->can_be_base = assert_cast *>(columns[base_cond_column_idx])->getData()[row_num]; data(place).value.push_back(node, arena); } @@ -264,7 +278,7 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { - // Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node. + /// Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node. this->data(const_cast(place)).sort(); writeBinary(data(place).sorted, buf); @@ -272,22 +286,22 @@ public: auto & value = data(place).value; size_t size = std::min(static_cast(events_size + 1), value.size()); - switch (Base) + switch (seq_base_kind) { - case HEAD: + case SequenceBase::Head: writeVarUInt(size, buf); for (size_t i = 0; i < size; ++i) value[i]->write(buf); break; - case TAIL: + case SequenceBase::Tail: writeVarUInt(size, buf); for (size_t i = 0; i < size; ++i) value[value.size() - size + i]->write(buf); break; - case FIRST_MATCH: - case LAST_MATCH: + case SequenceBase::FirstMatch: + case SequenceBase::LastMatch: writeVarUInt(value.size(), buf); for (auto & node : value) node->write(buf); @@ -312,53 +326,42 @@ public: value[i] = Node::read(buf, arena); } - inline UInt32 getBaseIndex(Data & data, bool & exist) const + inline std::optional getBaseIndex(Data & data) const { - exist = false; if (data.value.size() == 0) - return 0; + return {}; - switch (Base) + switch (seq_base_kind) { - case HEAD: + case SequenceBase::Head: if (data.value[0]->can_be_base) - { - exist = true; return 0; - } break; - case TAIL: + case SequenceBase::Tail: if (data.value[data.value.size() - 1]->can_be_base) - { - exist = true; return data.value.size() - 1; + break; + + case SequenceBase::FirstMatch: + for (size_t i = 0; i < data.value.size(); ++i) + { + if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base) + return i; } break; - case FIRST_MATCH: - for (UInt64 i = 0; i < data.value.size(); ++i) - if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base) - { - exist = true; - return i; - } - break; - - case LAST_MATCH: - for (UInt64 i = 0; i < data.value.size(); ++i) + case SequenceBase::LastMatch: + for (size_t i = 0; i < data.value.size(); ++i) { auto reversed_i = data.value.size() - i - 1; if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base) - { - exist = true; return reversed_i; - } } break; } - return 0; + return {}; } /// This method returns an index of next node that matched the events. @@ -366,40 +369,37 @@ public: /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { - const UInt32 unmatched = data.value.size(); + const UInt32 unmatched_idx = data.value.size(); if (data.value.size() <= events_size) - return unmatched; + return unmatched_idx; data.sort(); - bool base_existence; - UInt32 base = getBaseIndex(data, base_existence); - if (!base_existence) - return unmatched; + std::optional base_opt = getBaseIndex(data); + if (!base_opt.has_value()) + return unmatched_idx; + UInt32 base = static_cast(base_opt.value()); if (events_size == 0) - { - return data.value.size() > 0 ? base : unmatched; - } - else - { - UInt32 i = 0; - switch (Direction) - { - case FORWARD: - for (i = 0; i < events_size && base + i < data.value.size(); ++i) - if (data.value[base + i]->events_bitset.test(i) == false) - break; - return (i == events_size) ? base + i : unmatched; + return data.value.size() > 0 ? base : unmatched_idx; - case BACKWARD: - for (i = 0; i < events_size && i < base; ++i) - if (data.value[base - i]->events_bitset.test(i) == false) - break; - return (i == events_size) ? base - i : unmatched; - } + UInt32 i = 0; + switch (seq_direction) + { + case SequenceDirection::Forward: + for (i = 0; i < events_size && base + i < data.value.size(); ++i) + if (!data.value[base + i]->events_bitset.test(i)) + break; + return (i == events_size) ? base + i : unmatched_idx; + + case SequenceDirection::Backward: + for (i = 0; i < events_size && i < base; ++i) + if (!data.value[base - i]->events_bitset.test(i)) + break; + return (i == events_size) ? base - i : unmatched_idx; } + __builtin_unreachable(); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override @@ -414,7 +414,9 @@ public: to_concrete.getNullMapData().push_back(0); } else + { to.insertDefault(); + } } bool allocatesMemoryInArena() const override { return true; } From 63dc35780ed4b28bcf4a2dcdac16930279c4fe7d Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 2 Apr 2021 11:58:58 +0900 Subject: [PATCH 085/652] More parameter checking and remove a useless function --- .../AggregateFunctionNull.h | 22 ------------------- .../AggregateFunctionSequenceNextNode.cpp | 9 ++++++-- 2 files changed, 7 insertions(+), 24 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 821398e715e..e1238182ab5 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -276,28 +276,6 @@ public: this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); } - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override - { - if constexpr (result_is_nullable) - { - ColumnNullable & to_concrete = assert_cast(to); - if (this->getFlag(place)) - { - this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); - } - else - { - to_concrete.insertDefault(); - } - } - else - { - this->nested_function->insertResultInto(this->nestedPlace(place), to, arena); - } - } - - private: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index c3a42703e3d..6f2e2a5c3d7 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -66,13 +66,18 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; SequenceBase base = seq_base_mapping[param_base]; + if ((base == SequenceBase::Head && direction == SequenceDirection::Backward) || + (base == SequenceBase::Tail && direction == SequenceDirection::Forward)) + throw Exception(fmt::format( + "Invalid argument combination of '{}' with '{}'", param_base, param_dir), ErrorCodes::BAD_ARGUMENTS); + if (argument_types.size() < min_required_args) - throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Aggregate function " + name + " requires at least " + toString(min_required_args) + " arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); bool is_base_match_type = base == SequenceBase::FirstMatch || base == SequenceBase::LastMatch; if (is_base_match_type && argument_types.size() < min_required_args + 1) throw Exception( - "Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + "Aggregate function " + name + " requires at least " + toString(min_required_args + 1) + " arguments when base is first_match or last_match.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (argument_types.size() > max_events + min_required_args) From 9115fd8cecd2ce95e59eae1495b50fd252913e9b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 7 Apr 2021 15:22:53 +0300 Subject: [PATCH 086/652] wip --- .../compose/docker_compose_cassandra.yml | 4 +- .../compose/docker_compose_postgres.yml | 4 +- tests/integration/helpers/cluster.py | 53 ++++++++++++++----- tests/integration/helpers/external_sources.py | 3 ++ tests/integration/parallel.txt | 42 +++++++++++++++ .../test_alter_on_mixed_type_cluster/test.py | 12 ++--- .../test_concurrent_ttl_merges/test.py | 4 +- .../test_cross_replication/test.py | 2 +- .../test_ddl_worker_non_leader/test.py | 6 +-- .../test_cassandra.py | 2 +- 10 files changed, 101 insertions(+), 31 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_cassandra.yml b/docker/test/integration/runner/compose/docker_compose_cassandra.yml index e8aed06c08a..b6190a11d73 100644 --- a/docker/test/integration/runner/compose/docker_compose_cassandra.yml +++ b/docker/test/integration/runner/compose/docker_compose_cassandra.yml @@ -1,7 +1,5 @@ version: '2.3' services: cassandra1: - image: cassandra + image: cassandra:4.0 restart: always - ports: - - ${CASSANDRA_EXTERNAL_PORT}:${CASSANDRA_INTERNAL_PORT} diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 3b760e2f84b..a2f205c7afd 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -2,6 +2,7 @@ version: '2.3' services: postgres1: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always expose: - ${POSTGRES_PORT} @@ -24,6 +25,7 @@ services: target: /postgres/ postgres2: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always environment: POSTGRES_HOST_AUTH_METHOD: "trust" @@ -34,4 +36,4 @@ services: volumes: - type: ${POSTGRES_LOGS_FS:-tmpfs} source: ${POSTGRES2_DIR:-} - target: /postgres/ \ No newline at end of file + target: /postgres/ diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0a64f8fdc7b..3f26135dccd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -16,6 +16,7 @@ import traceback import urllib.parse import shlex +from cassandra.policies import RoundRobinPolicy import cassandra.cluster import psycopg2 import pymongo @@ -99,7 +100,6 @@ def get_docker_compose_path(): logging.debug(f"Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {LOCAL_DOCKER_COMPOSE_DIR}") return LOCAL_DOCKER_COMPOSE_DIR - def check_kafka_is_available(kafka_id, kafka_port): p = subprocess.Popen(('docker', 'exec', @@ -225,7 +225,9 @@ class ClickHouseCluster: # available when with_cassandra == True self.cassandra_host = "cassandra1" - self.cassandra_port = get_open_port() + self.cassandra_port = 9042 + self.cassandra_ip = None + self.cassandra_id = self.get_instance_docker_id(self.cassandra_host) # available when with_rabbitmq == True self.rabbitmq_host = "rabbitmq1" @@ -551,8 +553,7 @@ class ClickHouseCluster: if with_cassandra and not self.with_cassandra: self.with_cassandra = True - env_variables['CASSANDRA_EXTERNAL_PORT'] = str(self.cassandra_port) - env_variables['CASSANDRA_INTERNAL_PORT'] = "9042" + env_variables['CASSANDRA_PORT'] = str(self.cassandra_port) self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] @@ -778,7 +779,7 @@ class ClickHouseCluster: logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) - def wait_minio_to_start(self, timeout=30, secure=False): + def wait_minio_to_start(self, timeout=120, secure=False): os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', @@ -819,18 +820,27 @@ class ClickHouseCluster: logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) - def wait_cassandra_to_start(self, timeout=30): - cass_client = cassandra.cluster.Cluster(["localhost"], self.cassandra_port) + raise Exception("Can't wait Schema Registry to start") + + + def wait_cassandra_to_start(self, timeout=120): + self.cassandra_ip = self.get_instance_ip(self.cassandra_host) + cass_client = cassandra.cluster.Cluster([self.cassandra_ip], port=self.cassandra_port, load_balancing_policy=RoundRobinPolicy()) start = time.time() while time.time() - start < timeout: try: + logging.info(f"Check Cassandra Online {self.cassandra_id} {self.cassandra_ip} {self.cassandra_port}") + check = self.exec_in_container(self.cassandra_id, ["bash", "-c", f"/opt/cassandra/bin/cqlsh -u cassandra -p cassandra -e 'describe keyspaces' {self.cassandra_ip} {self.cassandra_port}"], user='root') + logging.info("Cassandra Online") cass_client.connect() - logging.info("Connected to Cassandra") + logging.info("Connected Clients to Cassandra") return except Exception as ex: logging.warning("Can't connect to Cassandra: %s", str(ex)) time.sleep(1) + raise Exception("Can't wait Cassandra to start") + def start(self, destroy_dirs=True): logging.debug("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) if self.is_up: @@ -844,6 +854,8 @@ class ClickHouseCluster: if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) logging.debug("Unstopped containers killed") + subprocess_call(['docker-compose', 'ps', '--services', '--all']) + except: pass @@ -895,7 +907,7 @@ class ClickHouseCluster: os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(120) + self.wait_mysql_to_start(180) if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') @@ -916,8 +928,7 @@ class ClickHouseCluster: os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) - self.wait_postgres_to_start(30) - self.wait_postgres_to_start(30) + self.wait_postgres_to_start(120) if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') @@ -987,7 +998,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 120.0 # seconds + start_deadline = time.time() + 180.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -1334,8 +1345,20 @@ class ClickHouseInstance: if not self.stay_alive: raise Exception("clickhouse can be stopped only with stay_alive=True instance") - self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root') - time.sleep(start_wait_sec) + try: + ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], user='root') + if ps_clickhouse == " PID TTY STAT TIME COMMAND" : + logging.warning("ClickHouse process already stopped") + return + + self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root') + time.sleep(start_wait_sec) + ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], user='root') + if ps_clickhouse != " PID TTY STAT TIME COMMAND" : + logging.warning(f"Force kill clickhouse in stop_clickhouse. ps:{ps_clickhouse}") + self.stop_clickhouse(kill=True) + except Exception as e: + logging.warning(f"Stop ClickHouse raised an error {e}") def start_clickhouse(self, stop_wait_sec=5): if not self.stay_alive: @@ -1360,8 +1383,10 @@ class ClickHouseInstance: return len(result) > 0 def grep_in_log(self, substring): + logging.debug(f"grep in log called {substring}") result = self.exec_in_container( ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) + logging.debug(f"grep result {result}") return result def count_in_log(self, substring): diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index 66cd45583a7..32ebdfa58c6 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -462,6 +462,9 @@ class SourceCassandra(ExternalSource): ) def prepare(self, structure, table_name, cluster): + if self.internal_hostname is None: + self.internal_hostname = cluster.cassandra_ip + self.client = cassandra.cluster.Cluster([self.internal_hostname], port=self.internal_port) self.session = self.client.connect() self.session.execute( diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt index 67daeced0f1..8fd6c6999cc 100644 --- a/tests/integration/parallel.txt +++ b/tests/integration/parallel.txt @@ -131,6 +131,48 @@ test_postgresql_protocol/test.py::test_psql_is_ready test_postgresql_protocol/test.py::test_psql_client test_postgresql_protocol/test.py::test_python_client test_postgresql_protocol/test.py::test_java_client +test_storage_kafka/test.py::test_kafka_json_as_string +test_storage_kafka/test.py::test_kafka_formats +test_storage_kafka/test.py::test_kafka_settings_old_syntax +test_storage_kafka/test.py::test_kafka_settings_new_syntax +test_storage_kafka/test.py::test_kafka_issue11308 +test_storage_kafka/test.py::test_kafka_issue4116 +test_storage_kafka/test.py::test_kafka_consumer_hang +test_storage_kafka/test.py::test_kafka_consumer_hang2 +test_storage_kafka/test.py::test_kafka_csv_with_delimiter +test_storage_kafka/test.py::test_kafka_tsv_with_delimiter +test_storage_kafka/test.py::test_kafka_select_empty +test_storage_kafka/test.py::test_kafka_json_without_delimiter +test_storage_kafka/test.py::test_kafka_protobuf +test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf +test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter +test_storage_kafka/test.py::test_kafka_materialized_view +test_storage_kafka/test.py::test_librdkafka_compression +test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery +test_storage_kafka/test.py::test_kafka_many_materialized_views +test_storage_kafka/test.py::test_kafka_flush_on_big_message +test_storage_kafka/test.py::test_kafka_virtual_columns +test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view +test_storage_kafka/test.py::test_kafka_insert +test_storage_kafka/test.py::test_kafka_produce_consume +test_storage_kafka/test.py::test_kafka_commit_on_block_write +test_storage_kafka/test.py::test_kafka_virtual_columns2 +test_storage_kafka/test.py::test_kafka_produce_key_timestamp +test_storage_kafka/test.py::test_kafka_flush_by_time +test_storage_kafka/test.py::test_kafka_flush_by_block_size +test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk +test_storage_kafka/test.py::test_kafka_rebalance +test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed +test_storage_kafka/test.py::test_exception_from_destructor +test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop +test_storage_kafka/test.py::test_bad_reschedule +test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed +test_storage_kafka/test.py::test_premature_flush_on_eof +test_storage_kafka/test.py::test_kafka_unavailable +test_storage_kafka/test.py::test_kafka_issue14202 +test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer +test_storage_kerberized_kafka/test.py::test_kafka_json_as_string +test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc test_storage_mysql/test.py::test_many_connections test_storage_mysql/test.py::test_insert_select test_storage_mysql/test.py::test_replace_select diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 852554f009d..c22626cb379 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -17,19 +17,19 @@ def started_cluster(): cluster.start() for node in [node1, node2]: - node.query(''' - CREATE TABLE test_table_replicated(date Date, id UInt32, value Int32) + node.query_with_retry(''' + CREATE TABLE IF NOT EXISTS test_table_replicated(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}') ORDER BY id; '''.format(replica=node.name)) - node.query('''CREATE TABLE test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') + node.query_with_retry('''CREATE TABLE IF NOT EXISTS test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') for node in [node3, node4]: - node.query(''' - CREATE TABLE test_table_replicated(date Date, id UInt32, value Int32) + node.query_with_retry(''' + CREATE TABLE IF NOT EXISTS test_table_replicated(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/1/someotable', '{replica}') ORDER BY id; '''.format(replica=node.name)) - node.query('''CREATE TABLE test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') + node.query_with_retry('''CREATE TABLE IF NOT EXISTS test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') yield cluster diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index ba5ed9f0758..18206a4dfb9 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -116,7 +116,7 @@ def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster): node1.query("SYSTEM STOP TTL MERGES") for i in range(100): - node1.query("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) + node1.query_with_retry("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) assert node1.query("SELECT COUNT() FROM replicated_ttl") == "100\n" @@ -147,7 +147,7 @@ def test_limited_ttl_merges_two_replicas(started_cluster): node2.query("SYSTEM STOP TTL MERGES") for i in range(100): - node1.query( + node1.query_with_retry( "INSERT INTO replicated_ttl_2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(10000)".format(i)) node2.query("SYSTEM SYNC REPLICA replicated_ttl_2", timeout=10) diff --git a/tests/integration/test_cross_replication/test.py b/tests/integration/test_cross_replication/test.py index 8a118934c93..cc5618e04e6 100644 --- a/tests/integration/test_cross_replication/test.py +++ b/tests/integration/test_cross_replication/test.py @@ -45,7 +45,7 @@ CREATE TABLE distributed(date Date, id UInt32, shard_id UInt32) 2017-06-16 333 2 ''' node1.query("INSERT INTO distributed FORMAT TSV", stdin=to_insert) - time.sleep(0.5) + time.sleep(5) yield cluster diff --git a/tests/integration/test_ddl_worker_non_leader/test.py b/tests/integration/test_ddl_worker_non_leader/test.py index b64f99d5345..172fc03c005 100644 --- a/tests/integration/test_ddl_worker_non_leader/test.py +++ b/tests/integration/test_ddl_worker_non_leader/test.py @@ -20,14 +20,14 @@ def started_cluster(): def test_non_leader_replica(started_cluster): - node1.query('''CREATE TABLE sometable(id UInt32, value String) + node1.query_with_retry('''CREATE TABLE IF NOT EXISTS sometable(id UInt32, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '1') ORDER BY tuple()''') - node2.query('''CREATE TABLE sometable(id UInt32, value String) + node2.query_with_retry('''CREATE TABLE IF NOT EXISTS sometable(id UInt32, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '2') ORDER BY tuple() SETTINGS replicated_can_become_leader = 0''') node1.query("INSERT INTO sometable SELECT number, toString(number) FROM numbers(100)") - node2.query("SYSTEM SYNC REPLICA sometable", timeout=10) + node2.query_with_retry("SYSTEM SYNC REPLICA sometable", timeout=10) assert node1.query("SELECT COUNT() FROM sometable") == "100\n" assert node2.query("SELECT COUNT() FROM sometable") == "100\n" diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 1271619f1f7..65080ab40dd 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -25,7 +25,7 @@ def setup_module(module): cluster = ClickHouseCluster(__file__, name=test_name) - SOURCE = SourceCassandra("Cassandra", "localhost", cluster.cassandra_port, cluster.cassandra_host, "9042", "", "") + SOURCE = SourceCassandra("Cassandra", None, cluster.cassandra_port, cluster.cassandra_host, cluster.cassandra_port, "", "") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() From d13d69e246ec5d2fe26f80cb67fd8afdd27c77d2 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 8 Apr 2021 00:26:51 +0900 Subject: [PATCH 087/652] Add allow_experimental_funnel_functions and fix argument order of sequenceNextNode --- .../AggregateFunctionSequenceNextNode.cpp | 19 +- .../AggregateFunctionSequenceNextNode.h | 4 +- src/Core/Settings.h | 1 + .../01656_sequence_next_node_long.sql | 210 +++++++++--------- 4 files changed, 124 insertions(+), 110 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 6f2e2a5c3d7..2870f233b65 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include @@ -20,6 +22,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; + extern const int FUNCTION_NOT_ALLOWED; } namespace @@ -85,8 +88,8 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, "Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.", name, max_events + min_required_args, max_events), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (const auto * cond_arg = argument_types[1].get(); cond_arg && !isUInt8(cond_arg)) - throw Exception("Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " + if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg)) + throw Exception("Illegal type " + cond_arg->getName() + " of third argument of aggregate function " + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); for (const auto i : ext::range(min_required_args, argument_types.size())) @@ -98,12 +101,12 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String) + if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) throw Exception{"Illegal type " + argument_types[1].get()->getName() + " of second argument of aggregate function " + name + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - DataTypePtr data_type = makeNullable(argument_types[2]); + DataTypePtr data_type = makeNullable(argument_types[1]); WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) @@ -128,6 +131,14 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) { return [max_events](const std::string & name, const DataTypes & argument_types, const Array & parameters) { + if (CurrentThread::isInitialized()) + { + const Context * query_context = CurrentThread::get().getQueryContext(); + if (query_context && !query_context->getSettingsRef().allow_experimental_funnel_functions) + throw Exception{"Cannot call 'sequenceNextNode' aggregate function because experimental_funnel_functions is not allowed. " + "Set 'allow_experimental_funnel_functions = 1' setting to enable", ErrorCodes::FUNCTION_NOT_ALLOWED}; + } + return createAggregateFunctionSequenceNode(name, max_events, argument_types, parameters); }; } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c3907a5adee..629a130a0e3 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -160,8 +160,8 @@ class SequenceNextNodeImpl final static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - static constexpr size_t base_cond_column_idx = 1; - static constexpr size_t event_column_idx = 2; + static constexpr size_t base_cond_column_idx = 2; + static constexpr size_t event_column_idx = 1; SequenceBase seq_base_kind; SequenceDirection seq_direction; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2986564b398..cf51508c82c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -428,6 +428,7 @@ class IColumn; M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \ M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \ + M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate funcion", 0) \ M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated experimental parser", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ \ diff --git a/tests/queries/0_stateless/01656_sequence_next_node_long.sql b/tests/queries/0_stateless/01656_sequence_next_node_long.sql index c03abed8a54..492f8b58862 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node_long.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node_long.sql @@ -1,3 +1,5 @@ +SET allow_experimental_funnel_functions = 1; + DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; @@ -27,26 +29,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); @@ -54,41 +56,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; @@ -123,74 +125,74 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); -SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; DROP TABLE IF EXISTS test_sequenceNextNode; @@ -211,18 +213,18 @@ INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2'); INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3'); INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4'); -SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_base_condition; From e176b52c1383350f2c861df4aa9ef5c7f30f663d Mon Sep 17 00:00:00 2001 From: achimbab Date: Thu, 8 Apr 2021 13:02:41 +0900 Subject: [PATCH 088/652] Fix typo in Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cf51508c82c..ad495c6bee9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -428,7 +428,7 @@ class IColumn; M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \ M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \ - M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate funcion", 0) \ + M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate function", 0) \ M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated experimental parser", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ \ From 49e6b3e8c762d5678559f203156aa859164c1abe Mon Sep 17 00:00:00 2001 From: Vladimir Date: Thu, 8 Apr 2021 12:10:41 +0300 Subject: [PATCH 089/652] Apply suggestions from code review, doc for sequenceNextNode --- .../parametric-functions.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index a2531303c7a..2cc1ebd40bf 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -511,10 +511,12 @@ Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summ Returns a value of next event that matched an event chain. +_Experimental function, `SET allow_experimental_funnel_functions = 1` to enable it._ + **Syntax** ``` sql -sequenceNextNode(direction, base)(timestamp, base_condition, event_column, event1, event2, event3, ...) +sequenceNextNode(direction, base)(timestamp, event_column, base_condition, event1, event2, event3, ...) ``` **Parameters** @@ -530,8 +532,8 @@ sequenceNextNode(direction, base)(timestamp, base_condition, event_column, event **Arguments** - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. -- `base_condition` — Condition that the base point must fulfill. - `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `base_condition` — Condition that the base point must fulfill. - `cond` — Conditions describing the chain of events. `UInt8` **Returned value** @@ -557,7 +559,7 @@ ORDER BY id; INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); -SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'A', page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; ``` Result: @@ -579,7 +581,7 @@ INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, ``` ```SQL -SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'Home', page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Base point, Matched with Home @@ -600,7 +602,7 @@ SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'Home', page, page = ' **Behavior for `backward` and `tail`** ```SQL -SELECT id, sequenceNextNode('backward', 'tail')(dt, page = 'Basket', page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home @@ -622,7 +624,7 @@ SELECT id, sequenceNextNode('backward', 'tail')(dt, page = 'Basket', page, page **Behavior for `forward` and `first_match`** ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home @@ -641,7 +643,7 @@ SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, p ``` ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home @@ -663,7 +665,7 @@ SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, p **Behavior for `backward` and `last_match`** ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // The result From bd5b29d3a5b103fbd857a2a99d27cdcbd5b7bffd Mon Sep 17 00:00:00 2001 From: Vladimir Date: Thu, 8 Apr 2021 12:13:10 +0300 Subject: [PATCH 090/652] Apply suggestions from code review, doc for sequenceNextNode --- .../aggregate-functions/parametric-functions.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 2cc1ebd40bf..27f76ebc5b6 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -684,7 +684,7 @@ SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', p ``` ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Matched with Home, the result is null @@ -721,7 +721,7 @@ INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3 ``` ```SQL -SELECT id, sequenceNextNode('forward', 'head')(dt, ref = 'ref1', page, page = 'A') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, ref = 'ref1', page = 'A') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'. @@ -731,7 +731,7 @@ SELECT id, sequenceNextNode('forward', 'head')(dt, ref = 'ref1', page, page = 'A ``` ```SQL -SELECT id, sequenceNextNode('backward', 'tail')(dt, ref = 'ref4', page, page = 'B') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, ref = 'ref4', page = 'B') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 @@ -741,7 +741,7 @@ SELECT id, sequenceNextNode('backward', 'tail')(dt, ref = 'ref4', page, page = ' ``` ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, ref = 'ref3', page, page = 'A') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, ref = 'ref3', page = 'A') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'. @@ -751,7 +751,7 @@ SELECT id, sequenceNextNode('forward', 'first_match')(dt, ref = 'ref3', page, pa ``` ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, ref = 'ref2', page, page = 'B') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, ref = 'ref2', page = 'B') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 From 2cc9d314be53d0f8ca16c1050a71a88439d67563 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 8 Apr 2021 17:43:57 +0300 Subject: [PATCH 091/652] fix --- .../docker_compose_postgres_cluster.yml | 26 ++++----- tests/integration/helpers/cluster.py | 55 ++++++++++++------- .../test_storage_postgresql/test.py | 8 +-- 3 files changed, 52 insertions(+), 37 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml index aa9301b1060..94b941b74da 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml @@ -29,16 +29,16 @@ services: source: ${POSTGRES3_DIR:-} target: /postgres/ postgres4: - image: postgres - command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] - restart: always - environment: - POSTGRES_HOST_AUTH_METHOD: "trust" - POSTGRES_PASSWORD: mysecretpassword - PGDATA: /postgres/data - expose: - - ${POSTGRES_PORT} - volumes: - - type: ${POSTGRES_LOGS_FS:-tmpfs} - source: ${POSTGRES4_DIR:-} - target: /postgres/ \ No newline at end of file + image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] + restart: always + environment: + POSTGRES_HOST_AUTH_METHOD: "trust" + POSTGRES_PASSWORD: mysecretpassword + PGDATA: /postgres/data + expose: + - ${POSTGRES_PORT} + volumes: + - type: ${POSTGRES_LOGS_FS:-tmpfs} + source: ${POSTGRES4_DIR:-} + target: /postgres/ \ No newline at end of file diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 703882fcd49..7fb8d7dc25a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -344,12 +344,8 @@ class ClickHouseCluster: def setup_postgres_cmd(self, instance, env_variables, docker_compose_yml_dir): self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - env_variables['POSTGRES_HOST'] = self.postgres_host env_variables['POSTGRES_PORT'] = str(self.postgres_port) env_variables['POSTGRES_DIR'] = self.postgres_logs_dir - env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir - env_variables['POSTGRES3_DIR'] = self.postgres3_logs_dir - env_variables['POSTGRES4_DIR'] = self.postgres4_logs_dir env_variables['POSTGRES_LOGS_FS'] = "bind" self.with_postgres = True @@ -357,6 +353,17 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] return self.base_postgres_cmd + def setup_postgres_cluster_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_postgres_cluster = True + env_variables['POSTGRES_PORT'] = str(self.postgres_port) + env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir + env_variables['POSTGRES3_DIR'] = self.postgres3_logs_dir + env_variables['POSTGRES4_DIR'] = self.postgres4_logs_dir + env_variables['POSTGRES_LOGS_FS'] = "bind" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) + self.base_postgres_cluster_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')] + def setup_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_hdfs = True env_variables['HDFS_HOST'] = self.hdfs_host @@ -559,11 +566,7 @@ class ClickHouseCluster: cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_postgres_cluster and not self.with_postgres_cluster: - self.with_postgres_cluster = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cluster_cmd = ['docker-compose', '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')] - cmds.append(self.base_postgres_cluster_cmd) + cmds.append(self.setup_postgres_cluster_cmd(instance, env_variables, docker_compose_yml_dir)) if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True @@ -760,16 +763,30 @@ class ClickHouseCluster: def wait_postgres_to_start(self, timeout=60): self.postgres_ip = self.get_instance_ip(self.postgres_host) + start = time.time() + while time.time() - start < timeout: + try: + conn = psycopg2.connect(host=self.postgres_ip, port=self.postgres_port, user='postgres', password='mysecretpassword') + conn.close() + logging.debug("Postgres Started") + return + except Exception as ex: + logging.debug("Can't connect to Postgres " + str(ex)) + time.sleep(0.5) + + raise Exception("Cannot wait Postgres container") + + def wait_postgres_cluster_to_start(self, timeout=60): self.postgres2_ip = self.get_instance_ip(self.postgres2_host) self.postgres3_ip = self.get_instance_ip(self.postgres3_host) self.postgres4_ip = self.get_instance_ip(self.postgres4_host) start = time.time() - for ip in [self.postgres_ip, self.postgres2_ip, self.postgres3_ip, self.postgres4_ip]: + for ip in [self.postgres2_ip, self.postgres3_ip, self.postgres4_ip]: while time.time() - start < timeout: try: conn = psycopg2.connect(host=ip, port=self.postgres_port, user='postgres', password='mysecretpassword') conn.close() - logging.debug("Postgres Started") + logging.debug("Postgres Cluster Started") return except Exception as ex: logging.debug("Can't connect to Postgres " + str(ex)) @@ -1013,22 +1030,20 @@ class ClickHouseCluster: shutil.rmtree(self.postgres_dir) os.makedirs(self.postgres_logs_dir) os.chmod(self.postgres_logs_dir, stat.S_IRWXO) - os.makedirs(self.postgres2_logs_dir) - os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) - os.makedirs(self.postgres3_logs_dir) - os.chmod(self.postgres3_logs_dir, stat.S_IRWXO) - os.makedirs(self.postgres4_logs_dir) - os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(120) if self.with_postgres_cluster and self.base_postgres_cluster_cmd: print('Setup Postgres') + os.makedirs(self.postgres2_logs_dir) + os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) + os.makedirs(self.postgres3_logs_dir) + os.chmod(self.postgres3_logs_dir, stat.S_IRWXO) + os.makedirs(self.postgres4_logs_dir) + os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cluster_cmd + common_opts) - self.wait_postgres_to_start(120, port=5421) - self.wait_postgres_to_start(120, port=5441) - self.wait_postgres_to_start(120, port=5461) + self.wait_postgres_cluster_to_start(120) if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 1ab090cb89f..e5fffe389d2 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -57,7 +57,7 @@ def started_cluster(): def test_postgres_select_insert(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() table_name = 'test_many' table = f'''postgresql('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'clickhouse', '{table_name}', 'postgres', 'mysecretpassword')''' @@ -77,7 +77,7 @@ def test_postgres_select_insert(started_cluster): def test_postgres_conversions(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( @@ -158,7 +158,7 @@ def test_postgres_conversions(started_cluster): def test_non_default_scema(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') cursor.execute('CREATE TABLE test_schema.test_table (a integer)') @@ -190,7 +190,7 @@ def test_non_default_scema(started_cluster): def test_concurrent_queries(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() node1.query(''' From b4dded3fc29a437c889f10ffb10c968709e91c02 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 12 Apr 2021 10:03:12 +0300 Subject: [PATCH 092/652] wip --- .../compose/docker_compose_mysql_cluster.yml | 60 +++++++++---------- tests/integration/parallel.txt | 2 + tests/integration/test_mysql_protocol/test.py | 2 +- tests/integration/test_storage_mysql/test.py | 35 +++++------ .../test_storage_postgresql/test.py | 8 +-- tests/integration/test_storage_s3/test.py | 40 ++++++------- 6 files changed, 75 insertions(+), 72 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml index f3a5b9975c4..6ced7536812 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml @@ -5,10 +5,10 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} DATADIR: /mysql/ expose: - - ${MYSQL_PORT} + - ${MYSQL_CLUSTER_PORT} command: --server_id=100 --log-bin='mysql-bin-2.log' --default-time-zone='+3:00' @@ -19,40 +19,40 @@ services: --general-log=ON --general-log-file=/mysql/2_general.log volumes: - - type: ${MYSQL_LOGS_FS:-tmpfs} - source: ${MYSQL_LOGS:-} + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} target: /mysql/ mysql3: - image: mysql:5.7 - restart: always - environment: - MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} - DATADIR: /mysql/ - expose: - - ${MYSQL_PORT} - command: --server_id=100 - --log-bin='mysql-bin-3.log' - --default-time-zone='+3:00' - --gtid-mode="ON" - --enforce-gtid-consistency - --log-error-verbosity=3 - --log-error=/mysql/3_error.log - --general-log=ON - --general-log-file=/mysql/3_general.log - volumes: - - type: ${MYSQL_LOGS_FS:-tmpfs} - source: ${MYSQL_LOGS:-} - target: /mysql/ + image: mysql:5.7 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_CLUSTER_PORT} + command: --server_id=100 + --log-bin='mysql-bin-3.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/3_error.log + --general-log=ON + --general-log-file=/mysql/3_general.log + volumes: + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ mysql4: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} DATADIR: /mysql/ expose: - - ${MYSQL_PORT} + - ${MYSQL_CLUSTER_PORT} command: --server_id=100 --log-bin='mysql-bin-4.log' --default-time-zone='+3:00' @@ -63,6 +63,6 @@ services: --general-log=ON --general-log-file=/mysql/4_general.log volumes: - - type: ${MYSQL_LOGS_FS:-tmpfs} - source: ${MYSQL_LOGS:-} - target: /mysql/ \ No newline at end of file + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt index 8fd6c6999cc..869fa8a52ca 100644 --- a/tests/integration/parallel.txt +++ b/tests/integration/parallel.txt @@ -181,7 +181,9 @@ test_storage_mysql/test.py::test_where test_storage_mysql/test.py::test_table_function test_storage_mysql/test.py::test_binary_type test_storage_mysql/test.py::test_enum_type +test_storage_mysql/test.py::test_mysql_distributed test_storage_postgresql/test.py::test_postgres_select_insert test_storage_postgresql/test.py::test_postgres_conversions test_storage_postgresql/test.py::test_non_default_scema test_storage_postgresql/test.py::test_concurrent_queries +test_storage_postgresql/test.py::test_postgres_distributed diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 951a07c74cd..74ece5ad433 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -46,7 +46,7 @@ def mysql_server(mysql_client): :type mysql_client: Container :rtype: Container """ - retries = 30 + retries = 60 for i in range(retries): info = mysql_client.client.api.inspect_container(mysql_client.name) if info['State']['Health']['Status'] == 'healthy': diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index c4426817b1b..ca6ff6d71f1 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -36,7 +36,7 @@ def started_cluster(): try: cluster.start() - conn = get_mysql_conn(cluster, cluster.mysql57_ip) + conn = get_mysql_conn(cluster, cluster.mysql_ip) create_mysql_db(conn, 'clickhouse') ## create mysql db and table @@ -50,7 +50,7 @@ def started_cluster(): def test_many_connections(started_cluster): table_name = 'test_many_connections' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -70,7 +70,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_select(started_cluster): table_name = 'test_insert_select' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -86,7 +86,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_replace_select(started_cluster): table_name = 'test_replace_select' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -105,7 +105,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_on_duplicate_select(started_cluster): table_name = 'test_insert_on_duplicate_select' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -124,7 +124,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_where(started_cluster): table_name = 'test_where' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); @@ -144,7 +144,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, 'table_function') table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' @@ -166,7 +166,7 @@ def test_table_function(started_cluster): def test_binary_type(started_cluster): - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') @@ -176,7 +176,7 @@ def test_binary_type(started_cluster): def test_enum_type(started_cluster): table_name = 'test_enum_type' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); @@ -202,7 +202,7 @@ def create_mysql_table(conn, tableName): def test_mysql_distributed(started_cluster): table_name = 'test_replicas' - conn1 = get_mysql_conn(started_cluster, started_cluster.mysql57_ip) + conn1 = get_mysql_conn(started_cluster, started_cluster.mysql_ip) conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip) conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip) conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip) @@ -210,6 +210,7 @@ def test_mysql_distributed(started_cluster): create_mysql_db(conn1, 'clickhouse') create_mysql_db(conn2, 'clickhouse') create_mysql_db(conn3, 'clickhouse') + create_mysql_db(conn4, 'clickhouse') create_mysql_table(conn1, table_name) create_mysql_table(conn2, table_name) @@ -228,13 +229,13 @@ def test_mysql_distributed(started_cluster): nodes[i-1].query(''' CREATE TABLE test_replica{} (id UInt32, name String, age UInt32, money UInt32) - ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, i)) + ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, 57 if i==1 else i)) nodes[i-1].query("INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(i, i)) # test multiple ports parsing - result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql{1|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql{57|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') assert(result == 'host1\n' or result == 'host2\n' or result == 'host3\n') - result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql57:3306|mysql2:3306|mysql3:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') assert(result == 'host1\n' or result == 'host2\n' or result == 'host3\n') # check all replicas are traversed @@ -250,7 +251,7 @@ def test_mysql_distributed(started_cluster): node2.query(''' CREATE TABLE test_shards (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('MySQL', `mysql{1|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + ENGINE = ExternalDistributed('MySQL', `mysql{57|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') # Check only one replica in each shard is used result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") @@ -264,10 +265,10 @@ def test_mysql_distributed(started_cluster): result = node2.query(query) assert(result == 'host1\nhost2\nhost3\nhost4\n') - # disconnect mysql1 - started_cluster.pause_container('mysql1') + # disconnect mysql57 + started_cluster.pause_container('mysql57') result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - started_cluster.unpause_container('mysql1') + started_cluster.unpause_container('mysql57') assert(result == 'host2\nhost4\n' or result == 'host3\nhost4\n') if __name__ == '__main__': diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index e5fffe389d2..bb71001ef52 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -243,10 +243,10 @@ def test_concurrent_queries(started_cluster): def test_postgres_distributed(started_cluster): - conn0 = get_postgres_conn(started_cluster.postgres_ip, database=True) - conn1 = get_postgres_conn(started_cluster.postgres2_ip, database=True) - conn2 = get_postgres_conn(started_cluster.postgres3_ip, database=True) - conn3 = get_postgres_conn(started_cluster.postgres4_ip, database=True) + conn0 = get_postgres_conn(started_cluster, started_cluster.postgres_ip, database=True) + conn1 = get_postgres_conn(started_cluster, started_cluster.postgres2_ip, database=True) + conn2 = get_postgres_conn(started_cluster, started_cluster.postgres3_ip, database=True) + conn3 = get_postgres_conn(started_cluster, started_cluster.postgres4_ip, database=True) cursor0 = conn0.cursor() cursor1 = conn1.cursor() diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index f4a208543ce..1c859fa782e 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -114,14 +114,14 @@ def run_query(instance, query, stdin=None, settings=None): # Test simple put. Also checks that wrong credentials produce an error with every compression method. @pytest.mark.parametrize("maybe_auth,positive,compression", [ - ("", True, 'auto'), - ("'minio','minio123',", True, 'auto'), - ("'wrongid','wrongkey',", False, 'auto'), - ("'wrongid','wrongkey',", False, 'gzip'), - ("'wrongid','wrongkey',", False, 'deflate'), - ("'wrongid','wrongkey',", False, 'brotli'), - ("'wrongid','wrongkey',", False, 'xz'), - ("'wrongid','wrongkey',", False, 'zstd') + pytest.param("", True, 'auto', id="positive"), + pytest.param("'minio','minio123',", True, 'auto', id="auth_positive"), + pytest.param("'wrongid','wrongkey',", False, 'auto', id="auto-"), + pytest.param("'wrongid','wrongkey',", False, 'gzip', id=""), + pytest.param("'wrongid','wrongkey',", False, 'deflate', id=""), + pytest.param("'wrongid','wrongkey',", False, 'brotli', id=""), + pytest.param("'wrongid','wrongkey',", False, 'xz', id=""), + pytest.param("'wrongid','wrongkey',", False, 'zstd, id=""') ]) def test_put(started_cluster, maybe_auth, positive, compression): # type: (ClickHouseCluster) -> None @@ -147,7 +147,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): # Test put no data to S3. @pytest.mark.parametrize("auth", [ - "'minio','minio123'," + pytest.param("'minio','minio123'", id="minio") ]) def test_empty_put(started_cluster, auth): # type: (ClickHouseCluster) -> None @@ -181,9 +181,9 @@ def test_empty_put(started_cluster, auth): # Test put values in CSV format. @pytest.mark.parametrize("maybe_auth,positive", [ - ("", True), - ("'minio','minio123',", True), - ("'wrongid','wrongkey',", False) + pytest.param("", True, id="positive"), + pytest.param("'minio','minio123',", True, id="auth_positive"), + pytest.param("'wrongid','wrongkey',", False, id="negative"), ]) def test_put_csv(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None @@ -285,9 +285,9 @@ def test_put_get_with_globs(started_cluster): # Test multipart put. @pytest.mark.parametrize("maybe_auth,positive", [ - ("", True), + pytest.param("", True, id="positive"), + pytest.param("'wrongid','wrongkey'", False, id="negative"), # ("'minio','minio123',",True), Redirect with credentials not working with nginx. - ("'wrongid','wrongkey',", False), ]) def test_multipart_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None @@ -344,8 +344,8 @@ def test_remote_host_filter(started_cluster): @pytest.mark.parametrize("s3_storage_args", [ - "''", # 1 arguments - "'','','','','',''" # 6 arguments + pytest.param("''", id="1_argument"), + pytest.param("'','','','','',''", id="6_arguments"), ]) def test_wrong_s3_syntax(started_cluster, s3_storage_args): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance @@ -458,8 +458,8 @@ def test_custom_auth_headers_exclusion(started_cluster): assert '403 Forbidden' in ei.value.stderr @pytest.mark.parametrize("extension,method", [ - ("bin", "gzip"), - ("gz", "auto") + pytest.param("bin", "gzip", id="bin"), + pytest.param("gz", "auto", id="gz"), ]) def test_storage_s3_get_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket @@ -532,8 +532,8 @@ def test_storage_s3_put_uncompressed(started_cluster): @pytest.mark.parametrize("extension,method", [ - ("bin", "gzip"), - ("gz", "auto") + pytest.param("bin", "gzip", id="bin"), + pytest.param("gz", "auto", id="gz") ]) def test_storage_s3_put_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket From 7da322f95ab2856c7181769703160ad5885cb312 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 12 Apr 2021 10:04:07 +0300 Subject: [PATCH 093/652] better --- src/Server/MySQLHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 75c88a6ff93..743a883b68a 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -219,7 +219,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp int ret = socket().receiveBytes(buf + pos, packet_size - pos); if (ret == 0) { - throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3.", ErrorCodes::CANNOT_READ_ALL_DATA); + throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3", ErrorCodes::CANNOT_READ_ALL_DATA); } pos += ret; } From db657c03c89737cd76a042be5e11c9dd3fe9691a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 12 Apr 2021 18:08:09 +0300 Subject: [PATCH 094/652] fix --- tests/integration/test_cluster_copier/test.py | 2 +- tests/integration/test_storage_kafka/test.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 861529079fb..c6068e3a6e9 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -411,7 +411,7 @@ def test_non_partitioned_table(started_cluster): def test_self_copy(started_cluster): - execute_task(Task_self_copy(started_cluster), []) + execute_task(started_cluster, Task_self_copy(started_cluster), []) if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index f25760516ef..a2f526741a3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2702,7 +2702,7 @@ def test_kafka_engine_put_errors_to_stream(kafka_cluster): # Unexpected json content for table test.kafka. messages.append(json.dumps({'i': 'n_' + random_string(4), 's': random_string(8)})) - kafka_produce('json', messages) + kafka_produce(kafka_cluster, 'json', messages) while True: total_rows = instance.query('SELECT count() FROM test.kafka_data', ignore_error=True) @@ -2774,7 +2774,7 @@ def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_clus else: messages.append(gen_message_with_jsons(10, 0)) - kafka_produce('json', messages) + kafka_produce(kafka_cluster, 'json', messages) while True: total_rows = instance.query('SELECT count() FROM test.kafka_data', ignore_error=True) From e4d1ea846c0cd6dd1a14f83c45709abaece8803f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 13 Apr 2021 13:52:22 +0300 Subject: [PATCH 095/652] more --- tests/integration/ci-runner.py | 5 +- tests/integration/helpers/cluster.py | 42 ++-- tests/integration/parallel.txt | 189 ----------------- tests/integration/parralel.json | 191 ++++++++++++++++++ .../integration/test_odbc_interaction/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 6 files changed, 216 insertions(+), 215 deletions(-) delete mode 100644 tests/integration/parallel.txt create mode 100644 tests/integration/parralel.json diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index cba20652c71..06903fec891 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -269,14 +269,13 @@ class ClickhouseIntegrationTestsRunner: return list(sorted(all_tests)) def _get_parallel_tests(self, repo_path): - parallel_tests_file_path = "{}/tests/integration/parallel.txt".format(repo_path) + parallel_tests_file_path = "{}/tests/integration/parallel.json".format(repo_path) if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) parallel_tests = [] with open(parallel_tests_file_path, "r") as parallel_tests_file: - for line in parallel_tests_file: - parallel_tests.append(line.strip()) + parallel_tests = json.load(parallel_tests_file) return list(sorted(parallel_tests)) def group_test_by_file(self, tests): diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7fb8d7dc25a..cd5867f972a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -706,7 +706,7 @@ class ClickHouseCluster: ["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)], user='root') - def wait_mysql_to_start(self, timeout=60): + def wait_mysql_to_start(self, timeout=180): self.mysql_ip = self.get_instance_ip('mysql57') start = time.time() errors = [] @@ -740,7 +740,7 @@ class ClickHouseCluster: subprocess_call(['docker-compose', 'ps', '--services', '--all']) raise Exception("Cannot wait MySQL 8 container") - def wait_mysql_cluster_to_start(self, timeout=60): + def wait_mysql_cluster_to_start(self, timeout=180): self.mysql2_ip = self.get_instance_ip(self.mysql2_host) self.mysql3_ip = self.get_instance_ip(self.mysql3_host) self.mysql4_ip = self.get_instance_ip(self.mysql4_host) @@ -761,7 +761,7 @@ class ClickHouseCluster: logging.error("Can't connect to MySQL:{}".format(errors)) raise Exception("Cannot wait MySQL container") - def wait_postgres_to_start(self, timeout=60): + def wait_postgres_to_start(self, timeout=180): self.postgres_ip = self.get_instance_ip(self.postgres_host) start = time.time() while time.time() - start < timeout: @@ -776,7 +776,7 @@ class ClickHouseCluster: raise Exception("Cannot wait Postgres container") - def wait_postgres_cluster_to_start(self, timeout=60): + def wait_postgres_cluster_to_start(self, timeout=180): self.postgres2_ip = self.get_instance_ip(self.postgres2_host) self.postgres3_ip = self.get_instance_ip(self.postgres3_host) self.postgres4_ip = self.get_instance_ip(self.postgres4_host) @@ -794,7 +794,7 @@ class ClickHouseCluster: raise Exception("Cannot wait Postgres container") - def wait_zookeeper_to_start(self, timeout=60): + def wait_zookeeper_to_start(self, timeout=180): start = time.time() while time.time() - start < timeout: try: @@ -848,7 +848,7 @@ class ClickHouseCluster: time.sleep(1) - def wait_hdfs_to_start(self, hdfs_api, timeout=60): + def wait_hdfs_to_start(self, hdfs_api, timeout=300): start = time.time() while time.time() - start < timeout: try: @@ -861,7 +861,7 @@ class ClickHouseCluster: raise Exception("Can't wait HDFS to start") - def wait_mongo_to_start(self, timeout=30): + def wait_mongo_to_start(self, timeout=180): connection_str = 'mongodb://{user}:{password}@{host}:{port}'.format( host='localhost', port=self.mongo_port, user='root', password='clickhouse') connection = pymongo.MongoClient(connection_str) @@ -875,7 +875,7 @@ class ClickHouseCluster: logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) - def wait_minio_to_start(self, timeout=120, secure=False): + def wait_minio_to_start(self, timeout=180, secure=False): os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', @@ -904,7 +904,7 @@ class ClickHouseCluster: raise Exception("Can't wait Minio to start") - def wait_schema_registry_to_start(self, timeout=10): + def wait_schema_registry_to_start(self, timeout=180): sr_client = CachedSchemaRegistryClient({"url":'http://localhost:{}'.format(self.schema_registry_port)}) start = time.time() while time.time() - start < timeout: @@ -919,7 +919,7 @@ class ClickHouseCluster: raise Exception("Can't wait Schema Registry to start") - def wait_cassandra_to_start(self, timeout=120): + def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) cass_client = cassandra.cluster.Cluster([self.cassandra_ip], port=self.cassandra_port, load_balancing_policy=RoundRobinPolicy()) start = time.time() @@ -994,7 +994,7 @@ class ClickHouseCluster: run_and_check(self.base_zookeeper_cmd + common_opts, env=env) for command in self.pre_zookeeper_commands: self.run_kazoo_commands_with_retries(command, repeats=5) - self.wait_zookeeper_to_start(120) + self.wait_zookeeper_to_start() if self.with_mysql and self.base_mysql_cmd: logging.debug('Setup MySQL') @@ -1003,7 +1003,7 @@ class ClickHouseCluster: os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(180) + self.wait_mysql_to_start() if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') @@ -1012,7 +1012,7 @@ class ClickHouseCluster: os.makedirs(self.mysql8_logs_dir) os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql8_cmd + common_opts) - self.wait_mysql8_to_start(180) + self.wait_mysql8_to_start() if self.with_mysql_cluster and self.base_mysql_cluster_cmd: print('Setup MySQL') @@ -1022,7 +1022,7 @@ class ClickHouseCluster: os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cluster_cmd + common_opts) - self.wait_mysql_cluster_to_start(120) + self.wait_mysql_cluster_to_start() if self.with_postgres and self.base_postgres_cmd: logging.debug('Setup Postgres') @@ -1032,7 +1032,7 @@ class ClickHouseCluster: os.chmod(self.postgres_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) - self.wait_postgres_to_start(120) + self.wait_postgres_to_start() if self.with_postgres_cluster and self.base_postgres_cluster_cmd: print('Setup Postgres') @@ -1043,13 +1043,13 @@ class ClickHouseCluster: os.makedirs(self.postgres4_logs_dir) os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cluster_cmd + common_opts) - self.wait_postgres_cluster_to_start(120) + self.wait_postgres_cluster_to_start() if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - self.wait_schema_registry_to_start(30) + self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: logging.debug('Setup kerberized kafka') @@ -1066,7 +1066,7 @@ class ClickHouseCluster: os.chmod(self.hdfs_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api() - self.wait_hdfs_to_start(hdfs_api, 300) + self.wait_hdfs_to_start(hdfs_api) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') @@ -1074,7 +1074,7 @@ class ClickHouseCluster: os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(hdfs_api, timeout=300) + self.wait_hdfs_to_start(hdfs_api) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') @@ -1656,8 +1656,8 @@ class ClickHouseInstance: "Database": "postgres", "UserName": "postgres", "Password": "mysecretpassword", - "Port": "5432", - "Servername": self.cluster.postgres_host, + "Port": str(self.cluster.postgres_port), + "Servername": self.cluster.postgres_ip, "Protocol": "9.3", "ReadOnly": "No", "RowVersioning": "No", diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt deleted file mode 100644 index 869fa8a52ca..00000000000 --- a/tests/integration/parallel.txt +++ /dev/null @@ -1,189 +0,0 @@ -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed] -test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed] -test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed] -test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache] -test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed] -test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache] -test_dictionaries_mysql/test.py::test_load_mysql_dictionaries -test_dictionaries_postgresql/test.py::test_load_dictionaries -test_dictionaries_postgresql/test.py::test_invalidate_query -test_dictionaries_postgresql/test.py::test_dictionary_with_replicas -test_disabled_mysql_server/test.py::test_disabled_mysql_server -test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic] -test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary] -test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic] -test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary] -test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic] -test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary] -test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic] -test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary] -test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic] -test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary] -test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic] -test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary] -test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic] -test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary] -test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic] -test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary] -test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic] -test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary] -test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic] -test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary] -test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic] -test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary] -test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic] -test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary] -test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic] -test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary] -test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic] -test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0] -test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1] -test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0] -test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1] -test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0] -test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1] -test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0] -test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1] -test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database -test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database -test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database -test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database -test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine -test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine -test_mysql_database_engine/test.py::test_mysql_types[common_types_1] -test_mysql_database_engine/test.py::test_mysql_types[common_types_2] -test_mysql_database_engine/test.py::test_mysql_types[common_types_3] -test_mysql_database_engine/test.py::test_mysql_types[common_types_4] -test_mysql_database_engine/test.py::test_mysql_types[common_types_5] -test_mysql_database_engine/test.py::test_mysql_types[common_types_6] -test_mysql_database_engine/test.py::test_mysql_types[common_types_7] -test_mysql_database_engine/test.py::test_mysql_types[common_types_80] -test_mysql_database_engine/test.py::test_mysql_types[common_types_81] -test_mysql_database_engine/test.py::test_mysql_types[common_types_9] -test_mysql_database_engine/test.py::test_mysql_types[common_types_10] -test_mysql_database_engine/test.py::test_mysql_types[common_types_11] -test_mysql_database_engine/test.py::test_mysql_types[common_types_12] -test_mysql_database_engine/test.py::test_mysql_types[common_types_13] -test_mysql_database_engine/test.py::test_mysql_types[common_types_14] -test_mysql_database_engine/test.py::test_mysql_types[common_types_15] -test_mysql_database_engine/test.py::test_mysql_types[common_types_16] -test_mysql_database_engine/test.py::test_mysql_types[common_types_17] -test_mysql_database_engine/test.py::test_mysql_types[common_types_18] -test_mysql_database_engine/test.py::test_mysql_types[common_types_19] -test_mysql_database_engine/test.py::test_mysql_types[common_types_20] -test_mysql_database_engine/test.py::test_mysql_types[decimal_1] -test_mysql_database_engine/test.py::test_mysql_types[decimal_2] -test_mysql_database_engine/test.py::test_mysql_types[decimal_3] -test_mysql_database_engine/test.py::test_mysql_types[decimal_4] -test_mysql_database_engine/test.py::test_mysql_types[timestamp_default] -test_mysql_database_engine/test.py::test_mysql_types[timestamp_6] -test_mysql_database_engine/test.py::test_mysql_types[datetime_default] -test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1] -test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6] -test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1] -test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2] -test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2] -test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3] -test_mysql_protocol/test.py::test_mysql_client -test_mysql_protocol/test.py::test_mysql_client_exception -test_mysql_protocol/test.py::test_mysql_affected_rows -test_mysql_protocol/test.py::test_mysql_replacement_query -test_mysql_protocol/test.py::test_mysql_explain -test_mysql_protocol/test.py::test_mysql_federated -test_mysql_protocol/test.py::test_mysql_set_variables -test_mysql_protocol/test.py::test_python_client -test_mysql_protocol/test.py::test_golang_client -test_mysql_protocol/test.py::test_php_client -test_mysql_protocol/test.py::test_mysqljs_client -test_mysql_protocol/test.py::test_java_client -test_mysql_protocol/test.py::test_types -test_odbc_interaction/test.py::test_mysql_simple_select_works -test_odbc_interaction/test.py::test_mysql_insert -test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema -test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow -test_odbc_interaction/test.py::test_postgres_insert -test_odbc_interaction/test.py::test_odbc_postgres_date_data_type -test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl -test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl -test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries -test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays -test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache -test_postgresql_protocol/test.py::test_psql_is_ready -test_postgresql_protocol/test.py::test_psql_client -test_postgresql_protocol/test.py::test_python_client -test_postgresql_protocol/test.py::test_java_client -test_storage_kafka/test.py::test_kafka_json_as_string -test_storage_kafka/test.py::test_kafka_formats -test_storage_kafka/test.py::test_kafka_settings_old_syntax -test_storage_kafka/test.py::test_kafka_settings_new_syntax -test_storage_kafka/test.py::test_kafka_issue11308 -test_storage_kafka/test.py::test_kafka_issue4116 -test_storage_kafka/test.py::test_kafka_consumer_hang -test_storage_kafka/test.py::test_kafka_consumer_hang2 -test_storage_kafka/test.py::test_kafka_csv_with_delimiter -test_storage_kafka/test.py::test_kafka_tsv_with_delimiter -test_storage_kafka/test.py::test_kafka_select_empty -test_storage_kafka/test.py::test_kafka_json_without_delimiter -test_storage_kafka/test.py::test_kafka_protobuf -test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf -test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter -test_storage_kafka/test.py::test_kafka_materialized_view -test_storage_kafka/test.py::test_librdkafka_compression -test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery -test_storage_kafka/test.py::test_kafka_many_materialized_views -test_storage_kafka/test.py::test_kafka_flush_on_big_message -test_storage_kafka/test.py::test_kafka_virtual_columns -test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view -test_storage_kafka/test.py::test_kafka_insert -test_storage_kafka/test.py::test_kafka_produce_consume -test_storage_kafka/test.py::test_kafka_commit_on_block_write -test_storage_kafka/test.py::test_kafka_virtual_columns2 -test_storage_kafka/test.py::test_kafka_produce_key_timestamp -test_storage_kafka/test.py::test_kafka_flush_by_time -test_storage_kafka/test.py::test_kafka_flush_by_block_size -test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk -test_storage_kafka/test.py::test_kafka_rebalance -test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed -test_storage_kafka/test.py::test_exception_from_destructor -test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop -test_storage_kafka/test.py::test_bad_reschedule -test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed -test_storage_kafka/test.py::test_premature_flush_on_eof -test_storage_kafka/test.py::test_kafka_unavailable -test_storage_kafka/test.py::test_kafka_issue14202 -test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer -test_storage_kerberized_kafka/test.py::test_kafka_json_as_string -test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc -test_storage_mysql/test.py::test_many_connections -test_storage_mysql/test.py::test_insert_select -test_storage_mysql/test.py::test_replace_select -test_storage_mysql/test.py::test_insert_on_duplicate_select -test_storage_mysql/test.py::test_where -test_storage_mysql/test.py::test_table_function -test_storage_mysql/test.py::test_binary_type -test_storage_mysql/test.py::test_enum_type -test_storage_mysql/test.py::test_mysql_distributed -test_storage_postgresql/test.py::test_postgres_select_insert -test_storage_postgresql/test.py::test_postgres_conversions -test_storage_postgresql/test.py::test_non_default_scema -test_storage_postgresql/test.py::test_concurrent_queries -test_storage_postgresql/test.py::test_postgres_distributed diff --git a/tests/integration/parralel.json b/tests/integration/parralel.json new file mode 100644 index 00000000000..3853042f7d0 --- /dev/null +++ b/tests/integration/parralel.json @@ -0,0 +1,191 @@ +[ + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", + "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", + "test_dictionaries_postgresql/test.py::test_load_dictionaries", + "test_dictionaries_postgresql/test.py::test_invalidate_query", + "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", + "test_disabled_mysql_server/test.py::test_disabled_mysql_server", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", + "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", + "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", + "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", + "test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database", + "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", + "test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_1]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_10]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_11]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_12]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_13]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_14]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_15]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_16]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_17]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_18]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_19]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_20]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", + "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", + "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]", + "test_mysql_protocol/test.py::test_mysql_client", + "test_mysql_protocol/test.py::test_mysql_client_exception", + "test_mysql_protocol/test.py::test_mysql_affected_rows", + "test_mysql_protocol/test.py::test_mysql_replacement_query", + "test_mysql_protocol/test.py::test_mysql_explain", + "test_mysql_protocol/test.py::test_mysql_federated", + "test_mysql_protocol/test.py::test_mysql_set_variables", + "test_mysql_protocol/test.py::test_python_client", + "test_mysql_protocol/test.py::test_golang_client", + "test_mysql_protocol/test.py::test_php_client", + "test_mysql_protocol/test.py::test_mysqljs_client", + "test_mysql_protocol/test.py::test_java_client", + "test_mysql_protocol/test.py::test_types", + "test_odbc_interaction/test.py::test_mysql_simple_select_works", + "test_odbc_interaction/test.py::test_mysql_insert", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", + "test_odbc_interaction/test.py::test_postgres_insert", + "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", + "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", + "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache", + "test_postgresql_protocol/test.py::test_psql_is_ready", + "test_postgresql_protocol/test.py::test_psql_client", + "test_postgresql_protocol/test.py::test_python_client", + "test_postgresql_protocol/test.py::test_java_client", + "test_storage_kafka/test.py::test_kafka_json_as_string", + "test_storage_kafka/test.py::test_kafka_formats", + "test_storage_kafka/test.py::test_kafka_settings_old_syntax", + "test_storage_kafka/test.py::test_kafka_settings_new_syntax", + "test_storage_kafka/test.py::test_kafka_issue11308", + "test_storage_kafka/test.py::test_kafka_issue4116", + "test_storage_kafka/test.py::test_kafka_consumer_hang", + "test_storage_kafka/test.py::test_kafka_consumer_hang2", + "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_select_empty", + "test_storage_kafka/test.py::test_kafka_json_without_delimiter", + "test_storage_kafka/test.py::test_kafka_protobuf", + "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", + "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", + "test_storage_kafka/test.py::test_kafka_materialized_view", + "test_storage_kafka/test.py::test_librdkafka_compression", + "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", + "test_storage_kafka/test.py::test_kafka_many_materialized_views", + "test_storage_kafka/test.py::test_kafka_flush_on_big_message", + "test_storage_kafka/test.py::test_kafka_virtual_columns", + "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", + "test_storage_kafka/test.py::test_kafka_insert", + "test_storage_kafka/test.py::test_kafka_produce_consume", + "test_storage_kafka/test.py::test_kafka_commit_on_block_write", + "test_storage_kafka/test.py::test_kafka_virtual_columns2", + "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", + "test_storage_kafka/test.py::test_kafka_flush_by_time", + "test_storage_kafka/test.py::test_kafka_flush_by_block_size", + "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", + "test_storage_kafka/test.py::test_kafka_rebalance", + "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", + "test_storage_kafka/test.py::test_exception_from_destructor", + "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", + "test_storage_kafka/test.py::test_bad_reschedule", + "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", + "test_storage_kafka/test.py::test_premature_flush_on_eof", + "test_storage_kafka/test.py::test_kafka_unavailable", + "test_storage_kafka/test.py::test_kafka_issue14202", + "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", + "test_storage_mysql/test.py::test_many_connections", + "test_storage_mysql/test.py::test_insert_select", + "test_storage_mysql/test.py::test_replace_select", + "test_storage_mysql/test.py::test_insert_on_duplicate_select", + "test_storage_mysql/test.py::test_where", + "test_storage_mysql/test.py::test_table_function", + "test_storage_mysql/test.py::test_binary_type", + "test_storage_mysql/test.py::test_enum_type", + "test_storage_mysql/test.py::test_mysql_distributed", + "test_storage_postgresql/test.py::test_postgres_select_insert", + "test_storage_postgresql/test.py::test_postgres_conversions", + "test_storage_postgresql/test.py::test_non_default_scema", + "test_storage_postgresql/test.py::test_concurrent_queries", + "test_storage_postgresql/test.py::test_postgres_distributed" +] diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index ec15b6a0f9a..2b823b38e19 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -328,7 +328,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): cursor.execute("insert into clickhouse.test_table values(3, 'xxx')") for i in range(100): try: - node1.query("system reload dictionary postgres_odbc_hashed", timeout=5) + node1.query("system reload dictionary postgres_odbc_hashed", timeout=15) except Exception as ex: assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 1c859fa782e..61f74463063 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -147,7 +147,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): # Test put no data to S3. @pytest.mark.parametrize("auth", [ - pytest.param("'minio','minio123'", id="minio") + pytest.param("'minio','minio123',", id="minio") ]) def test_empty_put(started_cluster, auth): # type: (ClickHouseCluster) -> None From fb1e2c9006433afc784804321e1e7d03975d3c8f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 13 Apr 2021 14:27:52 +0300 Subject: [PATCH 096/652] fix --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index cd5867f972a..c1fe8e7ef0d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1657,7 +1657,7 @@ class ClickHouseInstance: "UserName": "postgres", "Password": "mysecretpassword", "Port": str(self.cluster.postgres_port), - "Servername": self.cluster.postgres_ip, + "Servername": self.cluster.postgres_host, "Protocol": "9.3", "ReadOnly": "No", "RowVersioning": "No", From c2707a8154eef77c0405209bcc446ebeb5568898 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 13 Apr 2021 17:55:31 +0300 Subject: [PATCH 097/652] better --- .../compose/docker_compose_mysql_client.yml | 2 +- tests/integration/helpers/cluster.py | 77 ++++++- .../{parralel.json => parallel.json} | 0 tests/integration/test_mysql_protocol/test.py | 193 ++++++++---------- 4 files changed, 156 insertions(+), 116 deletions(-) rename tests/integration/{parralel.json => parallel.json} (100%) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml index 5e4565d64c3..5b37b6e6c09 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: - mysql1: + mysql_client: image: mysql:5.7 restart: always environment: diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c1fe8e7ef0d..322e9a60e95 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -177,6 +177,7 @@ class ClickHouseCluster: self.pre_zookeeper_commands = [] self.instances = {} self.with_zookeeper = False + self.with_mysql_client = False self.with_mysql = False self.with_mysql8 = False self.with_mysql_cluster = False @@ -266,6 +267,10 @@ class ClickHouseCluster: self.postgres3_logs_dir = os.path.join(self.postgres_dir, "postgres3") self.postgres4_logs_dir = os.path.join(self.postgres_dir, "postgres4") + # available when with_mysql_client == True + self.mysql_client_host = "mysql_client" + self.mysql_client_container = None + # available when with_mysql == True self.mysql_host = "mysql57" self.mysql_port = 3306 @@ -297,12 +302,24 @@ class ClickHouseCluster: self.is_up = False logging.debug(f"CLUSTER INIT base_config_dir:{self.base_config_dir}") + def get_docker_handle(self, docker_id): + return self.docker_client.containers.get(docker_id) + def get_client_cmd(self): cmd = self.client_bin_path if p.basename(cmd) == 'clickhouse': cmd += " client" return cmd + def setup_mysql_client_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mysql_client = True + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_client.yml')]) + self.base_mysql_client_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_client.yml')] + + return self.base_mysql_client_cmd + + def setup_mysql_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_mysql = True env_variables['MYSQL_HOST'] = self.mysql_host @@ -464,7 +481,7 @@ class ClickHouseCluster: def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, - with_zookeeper=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, + with_zookeeper=False, with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_postgres_cluster=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -506,6 +523,7 @@ class ClickHouseCluster: macros=macros or {}, with_zookeeper=with_zookeeper, zookeeper_config_path=self.zookeeper_config_path, + with_mysql_client=with_mysql_client, with_mysql=with_mysql, with_mysql8=with_mysql8, with_mysql_cluster=with_mysql_cluster, @@ -522,6 +540,8 @@ class ClickHouseCluster: library_bridge_bin_path=self.library_bridge_bin_path, clickhouse_path_dir=clickhouse_path_dir, with_odbc_drivers=with_odbc_drivers, + with_postgres=with_postgres, + with_postgres_cluster=with_postgres_cluster, hostname=hostname, env_variables=env_variables, image=image, @@ -553,6 +573,9 @@ class ClickHouseCluster: '--file', zookeeper_docker_compose_path] cmds.append(self.base_zookeeper_cmd) + if with_mysql_client and not self.with_mysql_client: + cmds.append(self.setup_mysql_client_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_mysql and not self.with_mysql: cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -706,6 +729,28 @@ class ClickHouseCluster: ["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)], user='root') + def wait_mysql_client_to_start(self, timeout=180): + start = time.time() + errors = [] + self.mysql_client_container = self.get_docker_handle(self.get_instance_docker_id(self.mysql_client_host)) + + while time.time() - start < timeout: + try: + info = self.mysql_client_container.client.api.inspect_container(self.mysql_client_container.name) + if info['State']['Health']['Status'] == 'healthy': + logging.debug("Mysql Client Container Started") + break + time.sleep(1) + + return + except Exception as ex: + errors += [str(ex)] + time.sleep(1) + + subprocess_call(['docker-compose', 'ps', '--services', '--all']) + logging.error("Can't connect to MySQL Client:{}".format(errors)) + raise Exception("Cannot wait MySQL Client container") + def wait_mysql_to_start(self, timeout=180): self.mysql_ip = self.get_instance_ip('mysql57') start = time.time() @@ -996,6 +1041,11 @@ class ClickHouseCluster: self.run_kazoo_commands_with_retries(command, repeats=5) self.wait_zookeeper_to_start() + if self.with_mysql_client and self.base_mysql_client_cmd: + logging.debug('Setup MySQL Client') + subprocess_check_call(self.base_mysql_client_cmd + common_opts) + self.wait_mysql_client_to_start() + if self.with_mysql and self.base_mysql_cmd: logging.debug('Setup MySQL') if os.path.exists(self.mysql_dir): @@ -1295,9 +1345,9 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, - macros, with_zookeeper, zookeeper_config_path, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs, - with_mongo, with_redis, with_minio, - with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, + macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka, + with_rabbitmq, with_kerberized_hdfs, with_mongo, with_redis, with_minio, + with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster, hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag="latest", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): @@ -1323,9 +1373,12 @@ class ClickHouseInstance: self.odbc_bridge_bin_path = odbc_bridge_bin_path self.library_bridge_bin_path = library_bridge_bin_path + self.with_mysql_client = with_mysql_client self.with_mysql = with_mysql self.with_mysql8 = with_mysql8 self.with_mysql_cluster = with_mysql_cluster + self.with_postgres = with_postgres + self.with_postgres_cluster = with_postgres_cluster self.with_kafka = with_kafka self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq @@ -1581,7 +1634,7 @@ class ClickHouseInstance: assert_eq_with_retry(self, "select 1", "1", retry_count=retries) def get_docker_handle(self): - return self.docker_client.containers.get(self.docker_id) + return self.cluster.get_docker_handle(self.docker_id) def stop(self): self.get_docker_handle().stop() @@ -1760,12 +1813,26 @@ class ClickHouseInstance: depends_on = [] + if self.with_mysql_client: + depends_on.append(self.cluster.mysql_client_host) + if self.with_mysql: depends_on.append("mysql57") if self.with_mysql8: depends_on.append("mysql80") + if self.with_mysql_cluster: + depends_on.append("mysql57") + depends_on.append("mysql2") + depends_on.append("mysql3") + depends_on.append("mysql4") + + if self.with_postgres_cluster: + depends_on.append("postgres2") + depends_on.append("postgres3") + depends_on.append("postgres4") + if self.with_kafka: depends_on.append("kafka1") depends_on.append("schema-registry") diff --git a/tests/integration/parralel.json b/tests/integration/parallel.json similarity index 100% rename from tests/integration/parralel.json rename to tests/integration/parallel.json diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 74ece5ad433..5f76c9e098b 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -18,46 +18,19 @@ DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "configs/ssl_conf.xml", "configs/mysql.xml", "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], - user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) + user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}, with_mysql_client=True) server_port = 9001 @pytest.fixture(scope="module") -def server_address(): +def started_cluster(): cluster.start() try: - yield cluster.get_instance_ip('node') + yield cluster finally: cluster.shutdown() -@pytest.fixture(scope='module') -def mysql_client(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') - - -@pytest.fixture(scope='module') -def mysql_server(mysql_client): - """Return MySQL container when it is healthy. - - :type mysql_client: Container - :rtype: Container - """ - retries = 60 - for i in range(retries): - info = mysql_client.client.api.inspect_container(mysql_client.name) - if info['State']['Health']['Status'] == 'healthy': - break - time.sleep(1) - else: - raise Exception('Mysql server has not started in %d seconds.' % retries) - - return mysql_client - - @pytest.fixture(scope='module') def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') @@ -90,42 +63,42 @@ def java_container(): yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') -def test_mysql_client(mysql_client, server_address): +def test_mysql_client(started_cluster): # type: (Container, str) -> None - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba -e "SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) logging.debug(f"test_mysql_client code:{code} stdout:{stdout}, stderr:{stderr}") assert stdout.decode() == '\n'.join(['1', '1', '']) - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "SELECT 1 as a;" -e "SELECT 'тест' as b;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stdout.decode() == '\n'.join(['a', '1', 'b', 'тест', '']) - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=abc -e "select 1 as a;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stderr.decode() == 'mysql: [Warning] Using a password on the command line interface can be insecure.\n' \ 'ERROR 516 (00000): default: Authentication failed: password is incorrect or there is no user with such name\n' - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "use system;" -e "select count(*) from (select name from tables limit 1);" -e "use system2;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stdout.decode() == 'count()\n1\n' assert stderr[0:182].decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ "ERROR 81 (00000) at line 1: Code: 81, e.displayText() = DB::Exception: Database system2 doesn't exist" - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "CREATE DATABASE x;" -e "USE x;" @@ -137,127 +110,127 @@ def test_mysql_client(mysql_client, server_address): -e "CREATE TEMPORARY TABLE tmp (tmp_column UInt32);" -e "INSERT INTO tmp VALUES (0), (1);" -e "SELECT * FROM tmp ORDER BY tmp_column;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stdout.decode() == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', '']) -def test_mysql_client_exception(mysql_client, server_address): +def test_mysql_client_exception(started_cluster): # Poco exception. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "CREATE TABLE default.t1_remote_mysql AS mysql('127.0.0.1:10086','default','t1_local','default','');" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stderr[0:258].decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ "ERROR 1000 (00000) at line 1: Poco::Exception. Code: 1000, e.code() = 0, e.displayText() = Exception: Connections to all replicas failed: default@127.0.0.1:10086 as user default" -def test_mysql_affected_rows(mysql_client, server_address): - code, (stdout, stderr) = mysql_client.exec_run(''' +def test_mysql_affected_rows(started_cluster): + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "CREATE TABLE IF NOT EXISTS default.t1 (n UInt64) ENGINE MergeTree() ORDER BY tuple();" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -vvv --protocol tcp -h {host} -P {port} default -u default --password=123 -e "INSERT INTO default.t1(n) VALUES(1);" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert "1 row affected" in stdout.decode() - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -vvv --protocol tcp -h {host} -P {port} default -u default --password=123 -e "INSERT INTO default.t1(n) SELECT * FROM numbers(1000)" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert "1000 rows affected" in stdout.decode() - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "DROP TABLE default.t1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 -def test_mysql_replacement_query(mysql_client, server_address): +def test_mysql_replacement_query(started_cluster): # SHOW TABLE STATUS LIKE. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "show table status like 'xx';" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # SHOW VARIABLES. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "show variables;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # KILL QUERY. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "kill query 0;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "kill query where query_id='mysql:0';" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # SELECT DATABASE(). - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "select database();" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'DATABASE()\ndefault\n' - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "select DATABASE();" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'DATABASE()\ndefault\n' -def test_mysql_explain(mysql_client, server_address): +def test_mysql_explain(started_cluster): # EXPLAIN SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # EXPLAIN AST SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN AST SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # EXPLAIN PLAN SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN PLAN SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # EXPLAIN PIPELINE graph=1 SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN PIPELINE graph=1 SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 -def test_mysql_federated(mysql_server, server_address): +def test_mysql_federated(started_cluster): # For some reason it occasionally fails without retries. retries = 100 for try_num in range(retries): @@ -272,14 +245,14 @@ def test_mysql_federated(mysql_server, server_address): or "MySQL server has gone away" in stderr or "Server shutdown in progress" in stderr) - code, (stdout, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -e "DROP SERVER IF EXISTS clickhouse;" -e "CREATE SERVER clickhouse FOREIGN DATA WRAPPER mysql OPTIONS (USER 'default', PASSWORD '123', HOST '{host}', PORT {port}, DATABASE 'mysql_federated');" -e "DROP DATABASE IF EXISTS mysql_federated;" -e "CREATE DATABASE mysql_federated;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) if code != 0: print(("stdout", stdout)) @@ -289,11 +262,11 @@ def test_mysql_federated(mysql_server, server_address): continue assert code == 0 - code, (stdout, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -e "CREATE TABLE mysql_federated.test(`col` int UNSIGNED) ENGINE=FEDERATED CONNECTION='clickhouse';" -e "SELECT * FROM mysql_federated.test ORDER BY col;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) if code != 0: print(("stdout", stdout)) @@ -305,11 +278,11 @@ def test_mysql_federated(mysql_server, server_address): assert stdout.decode() == '\n'.join(['col', '0', '1', '5', '']) - code, (stdout, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -e "INSERT INTO mysql_federated.test VALUES (0), (1), (5);" -e "SELECT * FROM mysql_federated.test ORDER BY col;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) if code != 0: print(("stdout", stdout)) @@ -322,8 +295,8 @@ def test_mysql_federated(mysql_server, server_address): assert stdout.decode() == '\n'.join(['col', '0', '0', '1', '1', '5', '5', '']) -def test_mysql_set_variables(mysql_client, server_address): - code, (stdout, stderr) = mysql_client.exec_run(''' +def test_mysql_set_variables(started_cluster): + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e " @@ -335,12 +308,12 @@ def test_mysql_set_variables(mysql_client, server_address): SET @@wait_timeout = 2147483; SET SESSION TRANSACTION ISOLATION LEVEL READ; " - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 -def test_python_client(server_address): - client = pymysql.connections.Connection(host=server_address, user='user_with_double_sha1', password='abacaba', +def test_python_client(started_cluster): + client = pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='user_with_double_sha1', password='abacaba', database='default', port=server_port) with pytest.raises(pymysql.InternalError) as exc_info: @@ -354,13 +327,13 @@ def test_python_client(server_address): assert cursor.fetchall() == [{'a': 1, 'b': 'тест'}] with pytest.raises(pymysql.InternalError) as exc_info: - pymysql.connections.Connection(host=server_address, user='default', password='abacab', database='default', + pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='default', password='abacab', database='default', port=server_port) assert exc_info.value.args == ( 516, 'default: Authentication failed: password is incorrect or there is no user with such name') - client = pymysql.connections.Connection(host=server_address, user='default', password='123', database='default', + client = pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='default', password='123', database='default', port=server_port) with pytest.raises(pymysql.InternalError) as exc_info: @@ -390,80 +363,80 @@ def test_python_client(server_address): assert cursor.fetchall() == [{'a': 1}, {'a': 1}, {'a': 3}, {'a': 4}] -def test_golang_client(server_address, golang_container): +def test_golang_client(started_cluster, golang_container): # type: (str, Container) -> None with open(os.path.join(SCRIPT_DIR, 'golang.reference'), 'rb') as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user default --password 123 --database ' - 'abc'.format(host=server_address, port=server_port), demux=True) + 'abc'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 assert stderr.decode() == "Error 81: Database abc doesn't exist\n" code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user default --password 123 --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout == reference code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout == reference -def test_php_client(server_address, php_container): +def test_php_client(started_cluster, php_container): # type: (str, Container) -> None code, (stdout, stderr) = php_container.exec_run( - 'php -f test.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) + 'php -f test.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( - 'php -f test_ssl.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) + 'php -f test_ssl.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( - 'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), + 'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( - 'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), + 'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' -def test_mysqljs_client(server_address, nodejs_container): +def test_mysqljs_client(started_cluster, nodejs_container): code, (_, stderr) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_sha256 abacaba'.format(host=server_address, port=server_port), demux=True) + 'node test.js {host} {port} user_with_sha256 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 assert 'MySQL is requesting the sha256_password authentication method, which is not supported.' in stderr.decode() code, (_, stderr) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_empty_password ""'.format(host=server_address, port=server_port), + 'node test.js {host} {port} user_with_empty_password ""'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 code, (_, _) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), + 'node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 code, (_, _) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_empty_password 123'.format(host=server_address, port=server_port), + 'node test.js {host} {port} user_with_empty_password 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 -def test_java_client(server_address, java_container): +def test_java_client(started_cluster, java_container): # type: (str, Container) -> None with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() @@ -471,33 +444,33 @@ def test_java_client(server_address, java_container): # database not exists exception. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database ' - 'abc'.format(host=server_address, port=server_port), demux=True) + 'abc'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 # empty password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == reference # non-empty password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user default --password 123 --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == reference # double-sha1 password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == reference -def test_types(server_address): - client = pymysql.connections.Connection(host=server_address, user='default', password='123', database='default', +def test_types(started_cluster): + client = pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='default', password='123', database='default', port=server_port) cursor = client.cursor(pymysql.cursors.DictCursor) From b635f17b1b7728a9693262fad7f995e05b29baa8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 13 Apr 2021 19:08:29 +0300 Subject: [PATCH 098/652] fix --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 322e9a60e95..684fd14b4dd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -377,7 +377,7 @@ class ClickHouseCluster: env_variables['POSTGRES3_DIR'] = self.postgres3_logs_dir env_variables['POSTGRES4_DIR'] = self.postgres4_logs_dir env_variables['POSTGRES_LOGS_FS'] = "bind" - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')]) self.base_postgres_cluster_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')] From 70d65c0092f1b70fc9a2c40ecb7c05740106df2a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 13 Apr 2021 19:22:10 +0300 Subject: [PATCH 099/652] longer timeout --- tests/integration/helpers/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 684fd14b4dd..a151e3795f3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -769,7 +769,7 @@ class ClickHouseCluster: logging.error("Can't connect to MySQL:{}".format(errors)) raise Exception("Cannot wait MySQL container") - def wait_mysql8_to_start(self, timeout=60): + def wait_mysql8_to_start(self, timeout=180): self.mysql8_ip = self.get_instance_ip('mysql80') start = time.time() while time.time() - start < timeout: @@ -854,7 +854,7 @@ class ClickHouseCluster: raise Exception("Cannot wait ZooKeeper container") - def make_hdfs_api(self, timeout=60, kerberized=False): + def make_hdfs_api(self, timeout=180, kerberized=False): hdfs_api = None if kerberized: keytab = p.abspath(p.join(self.instances['node1'].path, "secrets/clickhouse.keytab")) From f6d455732e3d42db7501e393afd935a3d31832af Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 14 Apr 2021 19:04:17 +0300 Subject: [PATCH 100/652] better --- docker/test/integration/runner/Dockerfile | 1 - tests/integration/runner | 18 ++++++++++++++++-- tests/integration/test_storage_kafka/test.py | 2 +- 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index ff8fed0001e..1c538058243 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -92,7 +92,6 @@ RUN set -x \ && echo 'dockremap:165536:65536' >> /etc/subuid \ && echo 'dockremap:165536:65536' >> /etc/subgid -VOLUME /var/lib/docker EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] CMD ["sh", "-c", "pytest $PYTEST_OPTS"] diff --git a/tests/integration/runner b/tests/integration/runner index 34bbbb29751..6c152b249b0 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -167,6 +167,13 @@ if __name__ == "__main__": dest="tests_list", help="List of tests to run") + parser.add_argument( + "--tmpfs", + action='store_true', + default=False, + dest="tmpfs", + help="Use tmpfs for dockerd files") + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() @@ -208,19 +215,25 @@ if __name__ == "__main__": # to avoid redownloading it every time # # should be removed manually when not needed - subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) + dockerd_internal_volume = "" + if args.tmpfs: + dockerd_internal_volume = "--tmpfs /var/lib/docker -e DOCKER_RAMDISK=true" + else: + subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) + dockerd_internal_volume = "--volume /var/lib/docker--volume={}_volume:/var/lib/docker".format(CONTAINER_NAME) # enable tty mode & interactive for docker if we have real tty tty = "" if sys.stdout.isatty() and sys.stdin.isatty(): tty = "-it" + cmd = "docker run {net} {tty} --rm --name {name} --privileged \ --volume={odbc_bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={library_bridge_bin}:/clickhouse-library-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=300 \ + {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=300 \ {env_tags} -e PYTEST_OPTS='{parallel} {opts} {tests_list}' {img} {command}".format( net=net, tty=tty, @@ -234,6 +247,7 @@ if __name__ == "__main__": parallel=parallel_args, opts=' '.join(args.pytest_args), tests_list=' '.join(args.tests_list), + dockerd_internal_volume=dockerd_internal_volume, img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, command=args.command diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index a2f526741a3..51136d578a3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2975,7 +2975,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): data_prefix = data_prefix + [''] if format_opts.get('printable', False) == False: raw_message = 'hex(_raw_message)' - kafka_produce(topic_name, data_prefix + data_sample) + kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; From a27f269e6b127a14ba8ee32a306c060b43ad3d6d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Apr 2021 07:15:11 +0300 Subject: [PATCH 101/652] catch --- tests/integration/test_profile_events_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index c70c29d1bf7..52f14c99b4a 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -57,7 +57,7 @@ def get_minio_stat(cluster): "rx_bytes": 0, "tx_bytes": 0, } - stat = requests.get(url="http://{}:{}/minio/prometheus/metrics".format("localhost", cluster.minio_port)).text.split( + stat = requests.get(url="http://{}:{}/minio/prometheus/metrics".format(cluster.minio_ip, cluster.minio_port)).text.split( "\n") for line in stat: x = re.search("s3_requests_total(\{.*\})?\s(\d+)(\s.*)?", line) From 647fd1b53bd75b5a651da7ca5f88233e2c25dd8f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Apr 2021 15:06:49 +0300 Subject: [PATCH 102/652] Add dockerhub-proxy to runner --- docker/test/integration/runner/dockerd-entrypoint.sh | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 714bf333757..a850a3a9086 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,6 +1,16 @@ #!/bin/bash set -e +cat > /etc/docker/daemon.json << EOF +{ + "ipv6": true, + "fixed-cidr-v6": "fd00::/8", + "ip-forward": true, + "insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"], + "registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"] +} +EOF + dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e From 31f2c65e4f9555f10406d3d20536fc97117776d3 Mon Sep 17 00:00:00 2001 From: fuqi Date: Sun, 18 Apr 2021 17:38:50 +0800 Subject: [PATCH 103/652] support join table mutation --- src/Storages/StorageJoin.cpp | 73 +++++++++++++++++-- src/Storages/StorageJoin.h | 5 ++ .../01821_join_table_mutation.reference | 5 ++ .../0_stateless/01821_join_table_mutation.sql | 23 ++++++ 4 files changed, 99 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01821_join_table_mutation.reference create mode 100644 tests/queries/0_stateless/01821_join_table_mutation.sql diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 983b9213a35..a392c71ad92 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -1,24 +1,25 @@ #include #include +#include #include #include #include -#include #include #include #include #include #include #include +#include #include #include -#include #include +#include -#include /// toLower -#include -#include +#include #include +#include +#include /// toLower namespace DB @@ -70,6 +71,7 @@ StorageJoin::StorageJoin( void StorageJoin::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder&) { + std::lock_guard lock(mutex); disk->removeRecursive(path); disk->createDirectories(path); disk->createDirectories(path + "tmp/"); @@ -78,6 +80,64 @@ void StorageJoin::truncate( join = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); } +void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const +{ + for(const auto& command: commands) { + switch (command.type) { + case MutationCommand::Type::DELETE: + break; + case MutationCommand::Type::UPDATE: + throw Exception("Table engine Join doesn't support update mutation, please use insert instead", ErrorCodes::NOT_IMPLEMENTED); + default: + throw Exception("Table engine Join doesn't support this mutation", ErrorCodes::NOT_IMPLEMENTED); + } + } +} + +void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) +{ + // Only delete is supported + std::lock_guard lock(mutex); + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto storage = getStorageID(); + auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); + auto in = interpreter->execute(); + in->readPrefix(); + + auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); + + const String backup_file_name = "1.bin"; // id starts from 1 + auto backup_buf = disk->writeFile(path + "tmp/" + backup_file_name); + auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); + auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); + + while (const Block & block = in->read()) { + new_data->addJoinedBlock(block, true); + if (persistent) + backup_stream.write(block); + } + + join = std::move(new_data); + increment = 1; + + if (persistent) { + backup_stream.flush(); + compressed_backup_buf.next(); + backup_buf->next(); + backup_buf->finalize(); + + std::vector files; + disk->listFiles(path, files); + for (const auto& file_name: files) { + if (file_name.ends_with(".bin")) { + disk->removeFileIfExists(path + file_name); + } + } + + disk->replaceFile(path + "tmp/" + backup_file_name, path + backup_file_name); + } +} HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) const { @@ -104,8 +164,7 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) void StorageJoin::insertBlock(const Block & block) -{ - std::unique_lock lock(rwlock); +{std::unique_lock lock(rwlock); join->addJoinedBlock(block, true); } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 4baac53c69c..2485fe015b0 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -29,6 +29,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; + void mutate(const MutationCommands & commands, ContextPtr context) override; + /// Return instance of HashJoin holding lock that protects from insertions to StorageJoin. /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join) const; @@ -69,6 +72,8 @@ private: /// Lock is stored in HashJoin instance during query and blocks concurrent insertions. mutable std::shared_mutex rwlock; + mutable std::mutex mutex; + void insertBlock(const Block & block) override; void finishInsert() override {} size_t getSize() const override; diff --git a/tests/queries/0_stateless/01821_join_table_mutation.reference b/tests/queries/0_stateless/01821_join_table_mutation.reference new file mode 100644 index 00000000000..0d7ca32ed70 --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_mutation.reference @@ -0,0 +1,5 @@ +100 +10 +99 +m10 +50 \ No newline at end of file diff --git a/tests/queries/0_stateless/01821_join_table_mutation.sql b/tests/queries/0_stateless/01821_join_table_mutation.sql new file mode 100644 index 00000000000..9662a197b88 --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_mutation.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS join_table_mutation; + +CREATE TABLE join_table_mutation(id Int32, name String) ENGINE = Join(ANY, LEFT, id); + +INSERT INTO join_table_mutation select number, toString(number) from numbers(100); + +SELECT count(1) FROM join_table_mutation; + +SELECT name FROM join_table_mutation WHERE id = 10; + +ALTER TABLE join_table_mutation DELETE WHERE id = 10; + +SELECT count(1) FROM join_table_mutation; + +SELECT name FROM join_table_mutation WHERE id = 10; + +INSERT INTO join_table_mutation VALUES (10, 'm10'); + +SELECT name FROM join_table_mutation WHERE id = 10; + +ALTER TABLE join_table_mutation DELETE WHERE id % 2 = 0; + +SELECT count(1) FROM join_table_mutation; \ No newline at end of file From 9f4c6adb43b94dff88663cea3aac3678dc3dc9ce Mon Sep 17 00:00:00 2001 From: fuqi Date: Sun, 18 Apr 2021 18:05:18 +0800 Subject: [PATCH 104/652] fix code style --- src/Storages/StorageJoin.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a392c71ad92..d5b858b46bc 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -82,7 +82,7 @@ void StorageJoin::truncate( void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for(const auto& command: commands) { + for (const auto & command: commands) { switch (command.type) { case MutationCommand::Type::DELETE: break; @@ -129,7 +129,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) std::vector files; disk->listFiles(path, files); - for (const auto& file_name: files) { + for (const auto & file_name: files) { if (file_name.ends_with(".bin")) { disk->removeFileIfExists(path + file_name); } @@ -164,7 +164,8 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) void StorageJoin::insertBlock(const Block & block) -{std::unique_lock lock(rwlock); +{ + std::unique_lock lock(rwlock); join->addJoinedBlock(block, true); } From 73a5e7a7c3dd8960ace220f0080667bb25125f36 Mon Sep 17 00:00:00 2001 From: fuqi Date: Mon, 19 Apr 2021 10:22:18 +0800 Subject: [PATCH 105/652] fix test case new line --- tests/queries/0_stateless/01821_join_table_mutation.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01821_join_table_mutation.reference b/tests/queries/0_stateless/01821_join_table_mutation.reference index 0d7ca32ed70..e79d145b39b 100644 --- a/tests/queries/0_stateless/01821_join_table_mutation.reference +++ b/tests/queries/0_stateless/01821_join_table_mutation.reference @@ -2,4 +2,4 @@ 10 99 m10 -50 \ No newline at end of file +50 From 2fd04ec9353e27f85fd09bbcbfc0053ab4a7ecc5 Mon Sep 17 00:00:00 2001 From: fuqi Date: Mon, 19 Apr 2021 11:22:35 +0800 Subject: [PATCH 106/652] fix code style --- src/Storages/StorageJoin.cpp | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index d5b858b46bc..92f16d703c9 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -82,8 +82,10 @@ void StorageJoin::truncate( void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for (const auto & command: commands) { - switch (command.type) { + for (const auto & command: commands) + { + switch (command.type) + { case MutationCommand::Type::DELETE: break; case MutationCommand::Type::UPDATE: @@ -112,7 +114,8 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); - while (const Block & block = in->read()) { + while (const Block & block = in->read()) + { new_data->addJoinedBlock(block, true); if (persistent) backup_stream.write(block); @@ -121,7 +124,8 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) join = std::move(new_data); increment = 1; - if (persistent) { + if (persistent) + { backup_stream.flush(); compressed_backup_buf.next(); backup_buf->next(); @@ -129,10 +133,10 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) std::vector files; disk->listFiles(path, files); - for (const auto & file_name: files) { - if (file_name.ends_with(".bin")) { + for (const auto & file_name: files) + { + if (file_name.ends_with(".bin")) disk->removeFileIfExists(path + file_name); - } } disk->replaceFile(path + "tmp/" + backup_file_name, path + backup_file_name); From edabf8f5c334ef4af745895a1ed68c6e8ac88d31 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Apr 2021 12:07:04 +0300 Subject: [PATCH 107/652] wip --- docker/test/integration/runner/dockerd-entrypoint.sh | 1 + tests/integration/test_adaptive_granularity/test.py | 6 ++++++ tests/integration/test_allowed_client_hosts/test.py | 9 +++++---- .../configs/dictionaries/.gitkeep | 1 - 4 files changed, 12 insertions(+), 5 deletions(-) delete mode 100644 tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index a850a3a9086..a04a88f541d 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,6 +1,7 @@ #!/bin/bash set -e +mkdir -p /etc/docker/ cat > /etc/docker/daemon.json << EOF { "ipv6": true, diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 2ea2f29212f..f10bfddb01b 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -30,6 +30,12 @@ def prepare_single_pair_with_setting(first_node, second_node, group): for node in (first_node, second_node): node.query("CREATE DATABASE IF NOT EXISTS test") + first_node.query("DROP TABLE IF EXISTS table_by_default") + second_node.query("DROP TABLE IF EXISTS table_by_default") + first_node.query("DROP TABLE IF EXISTS table_with_fixed_granularity") + second_node.query("DROP TABLE IF EXISTS table_with_fixed_granularity") + + # Two tables with adaptive granularity first_node.query( ''' diff --git a/tests/integration/test_allowed_client_hosts/test.py b/tests/integration/test_allowed_client_hosts/test.py index cf83718e25c..7b803fd50f3 100644 --- a/tests/integration/test_allowed_client_hosts/test.py +++ b/tests/integration/test_allowed_client_hosts/test.py @@ -37,8 +37,9 @@ def query(node, query): def setup_nodes(): try: cluster.start() - query(server, "CREATE TABLE test_table (x Int32) ENGINE = MergeTree() ORDER BY tuple()") - query(server, "INSERT INTO test_table VALUES (5)") + query(server, "DROP TABLE IF EXISTS test_allowed_client_hosts") + query(server, "CREATE TABLE test_allowed_client_hosts (x Int32) ENGINE = MergeTree() ORDER BY tuple()") + query(server, "INSERT INTO test_allowed_client_hosts VALUES (5)") yield cluster @@ -57,8 +58,8 @@ def test_allowed_host(): # expected_to_fail.extend([clientC3, clientD2]) for client_node in expected_to_pass: - assert query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") == "5\n" + assert query_from_one_node_to_another(client_node, server, "SELECT * FROM test_allowed_client_hosts") == "5\n" for client_node in expected_to_fail: with pytest.raises(Exception, match=r'default: Authentication failed'): - query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") + query_from_one_node_to_another(client_node, server, "SELECT * FROM test_allowed_client_hosts") diff --git a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep deleted file mode 100644 index c693f138c81..00000000000 --- a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep +++ /dev/null @@ -1 +0,0 @@ -keep \ No newline at end of file From 73c593f0300cf02252ce423b4516ef91c7c17065 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Apr 2021 12:07:21 +0300 Subject: [PATCH 108/652] better --- tests/integration/helpers/cluster.py | 61 +++++++++++++++++++--------- 1 file changed, 42 insertions(+), 19 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 2c25cb7a614..0fe732e8f88 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -323,6 +323,45 @@ class ClickHouseCluster: self.is_up = False logging.debug(f"CLUSTER INIT base_config_dir:{self.base_config_dir}") + def cleanup(self): + # Just in case kill unstopped containers from previous launch + try: + logging.debug("Trying to kill unstopped containers...") + subprocess_call(['docker', 'kill', f'`docker container list -a -f name={self.project_name}`']) + subprocess_call(['docker', 'rm', f'`docker container list -a -f name={self.project_name}`']) + logging.debug("Unstopped containers killed") + subprocess_call(['docker-compose', 'ps', '--services', '--all']) + except: + pass + + # # Just in case remove unused networks + # try: + # logging.debug("Trying to prune unused networks...") + + # subprocess_call(['docker', 'network', 'prune', '-f']) + # logging.debug("Networks pruned") + # except: + # pass + + # Remove unused containers + try: + logging.debug("Trying to prune unused containers...") + + subprocess_call(['docker', 'container', 'prune', '-f']) + logging.debug("Networks pruned") + except: + pass + + # Remove unused volumes + try: + logging.debug("Trying to prune unused volumes...") + + subprocess_call(['docker', 'volume', 'prune', '-f']) + logging.debug("Volumes pruned") + except: + pass + + def get_docker_handle(self, docker_id): return self.docker_client.containers.get(docker_id) @@ -1009,26 +1048,10 @@ class ClickHouseCluster: if self.is_up: return - # Just in case kill unstopped containers from previous launch try: - logging.debug("Trying to kill unstopped containers...") - - if not subprocess_call(['docker-compose', 'kill']): - subprocess_call(['docker-compose', 'down', '--volumes']) - logging.debug("Unstopped containers killed") - subprocess_call(['docker-compose', 'ps', '--services', '--all']) - - except: - pass - - # # Just in case remove unused networks - # try: - # logging.debug("Trying to prune unused networks...") - - # subprocess_call(['docker', 'network', 'prune', '-f']) - # logging.debug("Networks pruned") - # except: - # pass + self.cleanup() + except Exception as e: + logging.warning("Cleanup failed:{e}") try: # clickhouse_pull_cmd = self.base_cmd + ['pull'] From b80d4f8e963c3e5ce3ae9148a1fddf84d20cc89c Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Apr 2021 20:18:09 +0300 Subject: [PATCH 109/652] fix --- .../test_match_process_uid_against_data_owner/test.py | 2 +- tests/integration/test_storage_hdfs/test.py | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_match_process_uid_against_data_owner/test.py b/tests/integration/test_match_process_uid_against_data_owner/test.py index ed37227ab15..8adcc4463c9 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -31,7 +31,7 @@ def test_different_user(): with open(os.path.join(node.path, 'logs/clickhouse-server.err.log')) as log: expected_message = "Effective user of the process \(.*\) does not match the owner of the data \(.*\)\. Run under 'sudo -u .*'\." - last_message = log.readlines()[-1].strip() + last_messages = ";".join(log.readlines()[-2:-1]) if re.search(expected_message, last_message) is None: pytest.fail( diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9454b0902d3..e816cc500f0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -216,6 +216,7 @@ def test_write_gzip_storage(started_cluster): def test_virtual_columns(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query("create table virtual_cols (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/file*', 'TSV')") started_cluster.hdfs_api.write_data("/file1", "1\n") started_cluster.hdfs_api.write_data("/file2", "2\n") @@ -225,6 +226,7 @@ def test_virtual_columns(started_cluster): def test_read_files_with_spaces(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() started_cluster.hdfs_api.write_data("/test test test 1.txt", "1\n") started_cluster.hdfs_api.write_data("/test test test 2.txt", "2\n") started_cluster.hdfs_api.write_data("/test test test 3.txt", "3\n") From 30b05fec4294e3c29f065866e4425e2f6945beb1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Apr 2021 20:20:13 +0300 Subject: [PATCH 110/652] fix --- tests/integration/test_storage_hdfs/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index e816cc500f0..b3b5f9e6ded 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -218,18 +218,18 @@ def test_write_gzip_storage(started_cluster): def test_virtual_columns(started_cluster): hdfs_api = started_cluster.make_hdfs_api() node1.query("create table virtual_cols (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/file*', 'TSV')") - started_cluster.hdfs_api.write_data("/file1", "1\n") - started_cluster.hdfs_api.write_data("/file2", "2\n") - started_cluster.hdfs_api.write_data("/file3", "3\n") + hdfs_api.write_data("/file1", "1\n") + hdfs_api.write_data("/file2", "2\n") + hdfs_api.write_data("/file3", "3\n") expected = "1\tfile1\thdfs://hdfs1:9000//file1\n2\tfile2\thdfs://hdfs1:9000//file2\n3\tfile3\thdfs://hdfs1:9000//file3\n" assert node1.query("select id, _file as file_name, _path as file_path from virtual_cols order by id") == expected def test_read_files_with_spaces(started_cluster): hdfs_api = started_cluster.make_hdfs_api() - started_cluster.hdfs_api.write_data("/test test test 1.txt", "1\n") - started_cluster.hdfs_api.write_data("/test test test 2.txt", "2\n") - started_cluster.hdfs_api.write_data("/test test test 3.txt", "3\n") + hdfs_api.write_data("/test test test 1.txt", "1\n") + hdfs_api.write_data("/test test test 2.txt", "2\n") + hdfs_api.write_data("/test test test 3.txt", "3\n") node1.query("create table test (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/test*', 'TSV')") assert node1.query("select * from test order by id") == "1\n2\n3\n" From cb101e46bbd5003caeb025610ebaa3c767edf82f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 29 Apr 2021 14:57:48 +0300 Subject: [PATCH 111/652] finally --- tests/integration/helpers/cluster.py | 9 +- tests/integration/helpers/hdfs_api.py | 81 ++++++------ .../integration/test_odbc_interaction/test.py | 4 +- tests/integration/test_storage_s3/test.py | 15 +++ .../test_storage_s3/test_redirect.py | 122 ------------------ .../test.py | 5 +- .../test.py | 8 +- 7 files changed, 72 insertions(+), 172 deletions(-) delete mode 100644 tests/integration/test_storage_s3/test_redirect.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index be09147d0fe..d5c16ae9aa7 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -15,6 +15,7 @@ import time import traceback import urllib.parse import shlex +import urllib3 from cassandra.policies import RoundRobinPolicy import cassandra.cluster @@ -224,6 +225,7 @@ class ClickHouseCluster: self.minio_port = 9001 self.minio_client = None # type: Minio self.minio_redirect_host = "proxy1" + self.minio_redirect_ip = None self.minio_redirect_port = 8080 # available when with_hdfs == True @@ -351,7 +353,7 @@ class ClickHouseCluster: try: logging.debug("Trying to prune unused images...") - subprocess_call(['docker', 'images', 'prune', '-f']) + subprocess_call(['docker', 'image', 'prune', '-f']) logging.debug("Images pruned") except: pass @@ -985,12 +987,15 @@ class ClickHouseCluster: def wait_minio_to_start(self, timeout=180, secure=False): self.minio_ip = self.get_instance_ip(self.minio_host) + self.minio_redirect_ip = self.get_instance_ip(self.minio_redirect_host) + os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') minio_client = Minio(f'{self.minio_ip}:{self.minio_port}', access_key='minio', secret_key='minio123', - secure=secure) + secure=secure, + http_client=urllib3.PoolManager(cert_reqs='CERT_NONE')) # disable SSL check as we test ClickHouse and not Python library start = time.time() while time.time() - start < timeout: try: diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index a77d7cf4c7a..09df9b9dcb9 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -57,10 +57,6 @@ class HDFSApi(object): if kerberized: self._run_kinit() self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override="kerberizedhdfs1", principal=self.principal) - #principal=self.principal, - #hostname_override=self.host, principal=self.principal) - # , mutual_authentication=reqkerb.REQUIRED, force_preemptive=True) - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) if self.kerberos_auth is None: print("failed to obtain kerberos_auth") else: @@ -98,23 +94,29 @@ class HDFSApi(object): raise Exception("Kinit running failure") def req_wrapper(self, func, expected_code, cnt=2, **kwargs): - with dns_hook(self): - for i in range(0, cnt): - response_data = func(**kwargs) - if response_data.status_code == expected_code: - return response_data - else: - print("unexpected response_data.status_code {}", response_data.status_code) + for i in range(0, cnt): + response_data = func(**kwargs) + if response_data.status_code == expected_code: + return response_data + else: + print("unexpected response_data.status_code {}", response_data.status_code) response_data.raise_for_status() + def read_data(self, path, universal_newlines=True): + logging.debug("read_data protocol:{} host:{} port:{} path: {}".format(self.protocol, self.host, self.proxy_port, path)) response = self.req_wrapper(requests.get, 307, url="{protocol}://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(protocol=self.protocol, host=self.host, port=self.proxy_port, path=path), headers={'host': 'localhost'}, allow_redirects=False, verify=False, auth=self.kerberos_auth) # additional_params = '&'.join(response.headers['Location'].split('&')[1:2]) - url = "{location}".format(location=response.headers['Location']) - # print("redirected to ", url) - response_data = self.req_wrapper(requests.get, 200, url=url, - headers={'host': 'localhost'}, - verify=False, auth=self.kerberos_auth) + location = None + if self.kerberized: + location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port)) + else: + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) + logging.debug("redirected to {}".format(location)) + + response_data = self.req_wrapper(requests.get, 200, url=location, headers={'host': 'localhost'}, + verify=False, auth=self.kerberos_auth) + if universal_newlines: return response_data.text else: @@ -130,37 +132,36 @@ class HDFSApi(object): named_file.write(content) named_file.flush() - - if self.kerberized: - self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) - # print(self.kerberos_auth) - response = self.req_wrapper(requests.put, 307, - url="{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format( - protocol=self.protocol, host=self.host, - port=self.proxy_port, - path=path, user=self.user), - allow_redirects=False, - headers={'host': 'localhost'}, - params={'overwrite' : 'true'}, - verify=False, auth=self.kerberos_auth - ) - additional_params = '&'.join( - response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) + url="{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', + port=self.proxy_port, + path=path, user=self.user), + allow_redirects=False, + headers={'host': 'localhost'}, + params={'overwrite' : 'true'}, + verify=False, auth=self.kerberos_auth + ) + + logging.debug("HDFS api response:{}".format(response.headers)) + + # additional_params = '&'.join( + # response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) + if self.kerberized: + location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port)) + else: + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) with open(fpath, mode="rb") as fh: file_data = fh.read() protocol = "http" # self.protocol response = self.req_wrapper(requests.put, 201, - url="{location}".format( - location=response.headers['Location']), - data=file_data, - headers={'content-type':'text/plain', 'host': 'localhost'}, - params={'file': path, 'user.name' : self.user}, - allow_redirects=False, verify=False, auth=self.kerberos_auth + url="{location}".format(location=location), + data=file_data, + headers={'content-type':'text/plain', 'host': 'localhost'}, + params={'file': path, 'user.name' : self.user}, + allow_redirects=False, verify=False, auth=self.kerberos_auth ) - # print(response) + logging.debug(response) def write_gzip_data(self, path, content): diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 569358a8718..903350c28c9 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -540,7 +540,7 @@ def test_concurrent_queries(started_cluster): def test_odbc_long_column_names(started_cluster): - conn = get_postgres_conn(); + conn = get_postgres_conn(started_cluster); cursor = conn.cursor() column_name = "column" * 8 @@ -572,7 +572,7 @@ def test_odbc_long_column_names(started_cluster): def test_odbc_long_text(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("drop table if exists clickhouse.test_long_text") cursor.execute("create table clickhouse.test_long_text(flen int, field1 text)"); diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 7802fb8ad68..350e276a394 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -464,6 +464,21 @@ def test_custom_auth_headers_exclusion(started_cluster): assert ei.value.returncode == 243 assert '403 Forbidden' in ei.value.stderr + +def test_infinite_redirect(started_cluster): + bucket = "redirected" + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + filename = "test.csv" + get_query = f"select * from s3('http://resolver:{started_cluster.minio_redirect_port}/{bucket}/{filename}', 'CSV', '{table_format}')" + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + exception_raised = False + try: + run_query(instance, get_query) + except Exception as e: + assert str(e).find("Too many redirects while trying to access") != -1 + exception_raised = True + finally: + assert exception_raised @pytest.mark.parametrize("extension,method", [ pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz"), diff --git a/tests/integration/test_storage_s3/test_redirect.py b/tests/integration/test_storage_s3/test_redirect.py deleted file mode 100644 index 10a20184722..00000000000 --- a/tests/integration/test_storage_s3/test_redirect.py +++ /dev/null @@ -1,122 +0,0 @@ -import gzip -import json -import logging -import os -import io -import random -import threading -import time - -import helpers.client -import pytest -from helpers.cluster import ClickHouseCluster, ClickHouseInstance - -# Creates S3 bucket for tests and allows anonymous read-write access to it. -def prepare_s3_bucket(cluster): - # Allows read-write access for bucket without authorization. - bucket_read_write_policy = {"Version": "2012-10-17", - "Statement": [ - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:GetBucketLocation", - "Resource": "arn:aws:s3:::root" - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:ListBucket", - "Resource": "arn:aws:s3:::root" - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:GetObject", - "Resource": "arn:aws:s3:::root/*" - }, - { - "Sid": "", - "Effect": "Allow", - "Principal": {"AWS": "*"}, - "Action": "s3:PutObject", - "Resource": "arn:aws:s3:::root/*" - } - ]} - - minio_client = cluster.minio_client - minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) - - cluster.minio_restricted_bucket = "{}-with-auth".format(cluster.minio_bucket) - if minio_client.bucket_exists(cluster.minio_restricted_bucket): - minio_client.remove_bucket(cluster.minio_restricted_bucket) - - minio_client.make_bucket(cluster.minio_restricted_bucket) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__, name="redirect") - cluster.add_instance("dummy", with_minio=True, main_configs=["configs/defaultS3.xml"]) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") - - prepare_s3_bucket(cluster) - logging.info("S3 bucket created") - run_s3_mock(cluster) - - yield cluster - finally: - cluster.shutdown() - -def run_query(instance, query, stdin=None, settings=None): - # type: (ClickHouseInstance, str, object, dict) -> str - - logging.info("Running query '{}'...".format(query)) - result = instance.query(query, stdin=stdin, settings=settings) - logging.info("Query finished") - - return result - -def run_s3_mock(cluster): - logging.info("Starting s3 mock") - container_id = cluster.get_container_id('resolver') - current_dir = os.path.dirname(__file__) - cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") - cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) - - # Wait for S3 mock start - for attempt in range(10): - ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), - ["curl", "-s", "http://resolver:8080/"], nothrow=True) - if ping_response != 'OK': - if attempt == 9: - assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) - else: - time.sleep(1) - else: - break - - logging.info("S3 mock started") - -def test_infinite_redirect(started_cluster): - bucket = "redirected" - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - filename = "test.csv" - get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=bucket, - file=filename, - table_format=table_format) - instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - exception_raised = False - try: - run_query(instance, get_query) - except Exception as e: - assert str(e).find("Too many redirects while trying to access") != -1 - exception_raised = True - finally: - assert exception_raised \ No newline at end of file diff --git a/tests/integration/test_system_clusters_actual_information/test.py b/tests/integration/test_system_clusters_actual_information/test.py index afbaf53089f..48f654dc30a 100644 --- a/tests/integration/test_system_clusters_actual_information/test.py +++ b/tests/integration/test_system_clusters_actual_information/test.py @@ -17,11 +17,12 @@ node_1 = cluster.add_instance('node_1', with_zookeeper=True) def started_cluster(): try: cluster.start() + node_1.query_with_retry('DROP TABLE IF EXISTS replicated') - node_1.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + node_1.query_with_retry('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_1') ORDER BY id PARTITION BY toYYYYMM(date)''') - node.query("CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')") + node.query_with_retry("CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')") yield cluster diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 79e5dece174..4527e7a796f 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -32,28 +32,28 @@ def test_mutate_and_upgrade(start_cluster): node1.query("INSERT INTO mt VALUES ('2020-02-13', 1), ('2020-02-13', 2);") node1.query("ALTER TABLE mt DELETE WHERE id = 2", settings={"mutations_sync": "2"}) - node2.query("SYSTEM SYNC REPLICA mt", timeout=5) + node2.query("SYSTEM SYNC REPLICA mt", timeout=15) node1.restart_with_latest_version() node2.restart_with_latest_version() node2.query("INSERT INTO mt VALUES ('2020-02-13', 3);") - node1.query("SYSTEM SYNC REPLICA mt", timeout=5) + node1.query("SYSTEM SYNC REPLICA mt", timeout=15) assert node1.query("SELECT COUNT() FROM mt") == "2\n" assert node2.query("SELECT COUNT() FROM mt") == "2\n" node1.query("INSERT INTO mt VALUES ('2020-02-13', 4);") - node2.query("SYSTEM SYNC REPLICA mt", timeout=5) + node2.query("SYSTEM SYNC REPLICA mt", timeout=15) assert node1.query("SELECT COUNT() FROM mt") == "3\n" assert node2.query("SELECT COUNT() FROM mt") == "3\n" node2.query("ALTER TABLE mt DELETE WHERE id = 3", settings={"mutations_sync": "2"}) - node1.query("SYSTEM SYNC REPLICA mt", timeout=5) + node1.query("SYSTEM SYNC REPLICA mt", timeout=15) assert node1.query("SELECT COUNT() FROM mt") == "2\n" assert node2.query("SELECT COUNT() FROM mt") == "2\n" From dd192bb0ab0af08fb0ed2a73d0c5aa0730a138c4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 29 Apr 2021 17:26:41 +0300 Subject: [PATCH 112/652] better --- tests/integration/test_storage_s3/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 350e276a394..f40642f7f75 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -116,12 +116,12 @@ def run_query(instance, query, stdin=None, settings=None): @pytest.mark.parametrize("maybe_auth,positive,compression", [ pytest.param("", True, 'auto', id="positive"), pytest.param("'minio','minio123',", True, 'auto', id="auth_positive"), - pytest.param("'wrongid','wrongkey',", False, 'auto', id="auto-"), - pytest.param("'wrongid','wrongkey',", False, 'gzip', id=""), - pytest.param("'wrongid','wrongkey',", False, 'deflate', id=""), - pytest.param("'wrongid','wrongkey',", False, 'brotli', id=""), - pytest.param("'wrongid','wrongkey',", False, 'xz', id=""), - pytest.param("'wrongid','wrongkey',", False, 'zstd, id=""') + pytest.param("'wrongid','wrongkey',", False, 'auto', id="auto"), + pytest.param("'wrongid','wrongkey',", False, 'gzip', id="gzip"), + pytest.param("'wrongid','wrongkey',", False, 'deflate', id="deflate"), + pytest.param("'wrongid','wrongkey',", False, 'brotli', id="brotli"), + pytest.param("'wrongid','wrongkey',", False, 'xz', id="xz"), + pytest.param("'wrongid','wrongkey',", False, 'zstd', id="zstd") ]) def test_put(started_cluster, maybe_auth, positive, compression): # type: (ClickHouseCluster) -> None From 764ac307d11eda044e789b323053417ea6e72e06 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 29 Apr 2021 19:01:28 +0300 Subject: [PATCH 113/652] fix --- .../integration/test_adaptive_granularity_replicated/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_adaptive_granularity_replicated/test.py b/tests/integration/test_adaptive_granularity_replicated/test.py index a2057a5677b..5903cb85603 100644 --- a/tests/integration/test_adaptive_granularity_replicated/test.py +++ b/tests/integration/test_adaptive_granularity_replicated/test.py @@ -6,8 +6,8 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True) -node2 = cluster.add_instance('node2') -node3 = cluster.add_instance('node3', image='yandex/clickhouse-server', tag='19.1.14', +node2 = cluster.add_instance('node2', with_zookeeper=True) +node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.14', with_installed_binary=True) From 28fe11158a1b3ea5d51f12505b01aed4585094e1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 30 Apr 2021 12:18:12 +0300 Subject: [PATCH 114/652] rabbitmq --- tests/integration/helpers/cluster.py | 49 ++++++- tests/integration/helpers/network.py | 13 +- tests/integration/parallel.json | 127 +++++++++++++++++- tests/integration/runner | 14 +- .../test_redirect_url_storage/test.py | 28 ++-- .../integration/test_storage_rabbitmq/test.py | 126 +++++------------ 6 files changed, 244 insertions(+), 113 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d5c16ae9aa7..35adf00c75b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -142,6 +142,27 @@ def check_kafka_is_available(kafka_id, kafka_port): p.communicate() return p.returncode == 0 +def check_rabbitmq_is_available(rabbitmq_id): + p = subprocess.Popen(('docker', + 'exec', + '-i', + rabbitmq_id, + 'rabbitmqctl', + 'await_startup'), + stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + +def enable_consistent_hash_plugin(rabbitmq_id): + p = subprocess.Popen(('docker', + 'exec', + '-i', + rabbitmq_id, + "rabbitmq-plugins", "enable", "rabbitmq_consistent_hash_exchange"), + stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -267,8 +288,8 @@ class ClickHouseCluster: # available when with_rabbitmq == True self.rabbitmq_host = "rabbitmq1" - self.rabbitmq_port = get_open_port() - self.rabbitmq_http_port = get_open_port() + self.rabbitmq_ip = None + self.rabbitmq_port = 5672 # available when with_redis == True self.redis_host = "redis1" @@ -512,10 +533,7 @@ class ClickHouseCluster: def setup_rabbitmq_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_rabbitmq = True env_variables['RABBITMQ_HOST'] = self.rabbitmq_host - env_variables['RABBITMQ_EXTERNAL_PORT'] = str(self.rabbitmq_port) - env_variables['RABBITMQ_INTERNAL_PORT'] = "5672" - env_variables['RABBITMQ_EXTERNAL_HTTP_PORT'] = str(self.rabbitmq_http_port) - env_variables['RABBITMQ_INTERNAL_HTTP_PORT'] = "15672" + env_variables['RABBITMQ_PORT'] = str(self.rabbitmq_port) self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, @@ -904,6 +922,24 @@ class ClickHouseCluster: raise Exception("Cannot wait Postgres container") + def wait_rabbitmq_to_start(self, timeout=30): + self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host) + + start = time.time() + while time.time() - start < timeout: + try: + if check_rabbitmq_is_available(self.rabbitmq_docker_id): + logging.debug("RabbitMQ is available") + if enable_consistent_hash_plugin(self.rabbitmq_docker_id): + logging.debug("RabbitMQ consistent hash plugin is available") + return + time.sleep(0.5) + except Exception as ex: + logging.debug("Can't connect to RabbitMQ " + str(ex)) + time.sleep(0.5) + + raise Exception("Cannot wait RabbitMQ container") + def wait_zookeeper_to_start(self, timeout=180): start = time.time() while time.time() - start < timeout: @@ -1166,6 +1202,7 @@ class ClickHouseCluster: if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') + self.wait_rabbitmq_to_start() if self.with_hdfs and self.base_hdfs_cmd: logging.debug('Setup HDFS') diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 120ec9e873d..e4b9501a18f 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -1,7 +1,7 @@ import os import subprocess import time - +import logging import docker @@ -129,12 +129,12 @@ class _NetworkManager: def add_iptables_rule(self, **kwargs): cmd = ['iptables', '-I', 'DOCKER-USER', '1'] cmd.extend(self._iptables_cmd_suffix(**kwargs)) - self._exec_run(cmd, privileged=True) + self._exec_run_with_retry(cmd, retry_count=3, privileged=True) def delete_iptables_rule(self, **kwargs): cmd = ['iptables', '-D', 'DOCKER-USER'] cmd.extend(self._iptables_cmd_suffix(**kwargs)) - self._exec_run(cmd, privileged=True) + self._exec_run_with_retry(cmd, retry_count=3, privileged=True) @staticmethod def _iptables_cmd_suffix( @@ -207,6 +207,13 @@ class _NetworkManager: return self._container + def _exec_run_with_retry(self, cmd, retry_count, **kwargs): + for i in range(retry_count): + try: + self._exec_run(cmd, **kwargs) + except subprocess.CalledProcessError as e: + logging.error(f"_exec_run failed for {cmd}, {e}") + def _exec_run(self, cmd, **kwargs): container = self._ensure_container() diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index 3853042f7d0..c783798055d 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -16,6 +16,10 @@ "test_dictionaries_postgresql/test.py::test_invalidate_query", "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", "test_disabled_mysql_server/test.py::test_disabled_mysql_server", + "test_disk_access_storage/test.py::test_create", + "test_disk_access_storage/test.py::test_alter", + "test_disk_access_storage/test.py::test_drop", + "test_distributed_storage_configuration/test.py::test_insert", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", @@ -64,6 +68,8 @@ "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", @@ -119,10 +125,22 @@ "test_mysql_protocol/test.py::test_types", "test_odbc_interaction/test.py::test_mysql_simple_select_works", "test_odbc_interaction/test.py::test_mysql_insert", + "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", + "test_odbc_interaction/test.py::test_sqlite_table_function", + "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", + "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", + "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", "test_odbc_interaction/test.py::test_postgres_insert", + "test_odbc_interaction/test.py::test_bridge_dies_with_parent", "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", + "test_odbc_interaction/test.py::test_odbc_postgres_conversions", + "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", + "test_odbc_interaction/test.py::test_many_connections", + "test_odbc_interaction/test.py::test_concurrent_queries", + "test_odbc_interaction/test.py::test_odbc_long_column_names", + "test_odbc_interaction/test.py::test_odbc_long_text", "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", @@ -132,6 +150,36 @@ "test_postgresql_protocol/test.py::test_psql_client", "test_postgresql_protocol/test.py::test_python_client", "test_postgresql_protocol/test.py::test_java_client", + "test_redirect_url_storage/test.py::test_url_without_redirect", + "test_redirect_url_storage/test.py::test_url_with_globs", + "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", + "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", + "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", + "test_reloading_storage_configuration/test.py::test_add_disk", + "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", + "test_reloading_storage_configuration/test.py::test_add_policy", + "test_reloading_storage_configuration/test.py::test_new_policy_works", + "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", + "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", + "test_reloading_storage_configuration/test.py::test_remove_disk", + "test_reloading_storage_configuration/test.py::test_remove_policy", + "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", + "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", + "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", + "test_storage_hdfs/test.py::test_read_write_storage", + "test_storage_hdfs/test.py::test_read_write_storage_with_globs", + "test_storage_hdfs/test.py::test_read_write_table", + "test_storage_hdfs/test.py::test_write_table", + "test_storage_hdfs/test.py::test_bad_hdfs_uri", + "test_storage_hdfs/test.py::test_globs_in_read_table", + "test_storage_hdfs/test.py::test_read_write_gzip_table", + "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", + "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", + "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", + "test_storage_hdfs/test.py::test_write_gz_storage", + "test_storage_hdfs/test.py::test_write_gzip_storage", + "test_storage_hdfs/test.py::test_virtual_columns", + "test_storage_hdfs/test.py::test_read_files_with_spaces", "test_storage_kafka/test.py::test_kafka_json_as_string", "test_storage_kafka/test.py::test_kafka_formats", "test_storage_kafka/test.py::test_kafka_settings_old_syntax", @@ -172,8 +220,21 @@ "test_storage_kafka/test.py::test_kafka_unavailable", "test_storage_kafka/test.py::test_kafka_issue14202", "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", + "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", + "test_storage_kerberized_hdfs/test.py::test_read_table", + "test_storage_kerberized_hdfs/test.py::test_read_write_storage", + "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", + "test_storage_kerberized_hdfs/test.py::test_two_users", + "test_storage_kerberized_hdfs/test.py::test_read_table_expired", + "test_storage_kerberized_hdfs/test.py::test_prohibited", + "test_storage_kerberized_hdfs/test.py::test_cache_path", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", + "test_storage_mongodb/test.py::test_simple_select", + "test_storage_mongodb/test.py::test_complex_data_type", + "test_storage_mongodb/test.py::test_incorrect_data_type", "test_storage_mysql/test.py::test_many_connections", "test_storage_mysql/test.py::test_insert_select", "test_storage_mysql/test.py::test_replace_select", @@ -183,9 +244,73 @@ "test_storage_mysql/test.py::test_binary_type", "test_storage_mysql/test.py::test_enum_type", "test_storage_mysql/test.py::test_mysql_distributed", + "test_storage_mysql/test.py::test_external_settings", "test_storage_postgresql/test.py::test_postgres_select_insert", "test_storage_postgresql/test.py::test_postgres_conversions", "test_storage_postgresql/test.py::test_non_default_scema", "test_storage_postgresql/test.py::test_concurrent_queries", - "test_storage_postgresql/test.py::test_postgres_distributed" + "test_storage_postgresql/test.py::test_postgres_distributed", + "test_storage_rabbitmq/test.py::test_rabbitmq_select", + "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", + "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", + "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", + "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", + "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", + "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", + "test_storage_rabbitmq/test.py::test_rabbitmq_big_message", + "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", + "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", + "test_storage_rabbitmq/test.py::test_rabbitmq_insert", + "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", + "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", + "test_storage_rabbitmq/test.py::test_rabbitmq_direct_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_fanout_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_hash_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", + "test_storage_rabbitmq/test.py::test_rabbitmq_headers_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", + "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_consumers_to_each_queue", + "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", + "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", + "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", + "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", + "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", + "test_storage_rabbitmq/test.py::test_rabbitmq_vhost", + "test_storage_s3/test.py::test_put[positive]", + "test_storage_s3/test.py::test_put[auth_positive]", + "test_storage_s3/test.py::test_put[auto]", + "test_storage_s3/test.py::test_put[gzip]", + "test_storage_s3/test.py::test_put[deflate]", + "test_storage_s3/test.py::test_put[brotli]", + "test_storage_s3/test.py::test_put[xz]", + "test_storage_s3/test.py::test_put[zstd]", + "test_storage_s3/test.py::test_empty_put[minio]", + "test_storage_s3/test.py::test_put_csv[positive]", + "test_storage_s3/test.py::test_put_csv[auth_positive]", + "test_storage_s3/test.py::test_put_csv[negative]", + "test_storage_s3/test.py::test_put_get_with_redirect", + "test_storage_s3/test.py::test_put_with_zero_redirect", + "test_storage_s3/test.py::test_put_get_with_globs", + "test_storage_s3/test.py::test_multipart_put[positive]", + "test_storage_s3/test.py::test_multipart_put[negative]", + "test_storage_s3/test.py::test_remote_host_filter", + "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", + "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", + "test_storage_s3/test.py::test_s3_glob_scheherazade", + "test_storage_s3/test.py::test_custom_auth_headers", + "test_storage_s3/test.py::test_custom_auth_headers_exclusion", + "test_storage_s3/test.py::test_infinite_redirect", + "test_storage_s3/test.py::test_storage_s3_get_gzip[bin]", + "test_storage_s3/test.py::test_storage_s3_get_gzip[gz]", + "test_storage_s3/test.py::test_storage_s3_get_unstable", + "test_storage_s3/test.py::test_storage_s3_put_uncompressed", + "test_storage_s3/test.py::test_storage_s3_put_gzip[bin]", + "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", + "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", + "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]", + "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]" ] diff --git a/tests/integration/runner b/tests/integration/runner index 6c152b249b0..d203d74218f 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -70,6 +70,10 @@ def check_args_and_update_paths(args): if not os.path.exists(path): raise Exception("Path {} doesn't exist".format(path)) + if args.dockerd_volume: + if not os.path.isabs(args.dockerd_volume): + args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.dockerd_volume)) + if not os.path.exists(os.path.join(args.base_configs_dir, "config.xml")): raise Exception("No configs.xml in {}".format(args.base_configs_dir)) @@ -174,6 +178,12 @@ if __name__ == "__main__": dest="tmpfs", help="Use tmpfs for dockerd files") + parser.add_argument( + "--dockerd-volume-dir", + action='store', + dest="dockerd_volume", + help="Bind volume to this dir to use for dockerd files") + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() @@ -218,9 +228,11 @@ if __name__ == "__main__": dockerd_internal_volume = "" if args.tmpfs: dockerd_internal_volume = "--tmpfs /var/lib/docker -e DOCKER_RAMDISK=true" + elif args.dockerd_volume: + dockerd_internal_volume = f"--mount type=bind,source={args.dockerd_volume},target=/var/lib/docker" else: subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) - dockerd_internal_volume = "--volume /var/lib/docker--volume={}_volume:/var/lib/docker".format(CONTAINER_NAME) + dockerd_internal_volume = "--volume={}_volume:/var/lib/docker".format(CONTAINER_NAME) # enable tty mode & interactive for docker if we have real tty tty = "" diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index d51b7e3287e..7b9adba32e7 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -29,12 +29,14 @@ def test_url_without_redirect(started_cluster): def test_url_with_globs(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage_1_1", "1\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_2", "2\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_3", "3\n") - started_cluster.hdfs_api.write_data("/simple_storage_2_1", "4\n") - started_cluster.hdfs_api.write_data("/simple_storage_2_2", "5\n") - started_cluster.hdfs_api.write_data("/simple_storage_2_3", "6\n") + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage_1_1", "1\n") + hdfs_api.write_data("/simple_storage_1_2", "2\n") + hdfs_api.write_data("/simple_storage_1_3", "3\n") + hdfs_api.write_data("/simple_storage_2_1", "4\n") + hdfs_api.write_data("/simple_storage_2_2", "5\n") + hdfs_api.write_data("/simple_storage_2_3", "6\n") result = node1.query( "select * from url('http://hdfs1:50075/webhdfs/v1/simple_storage_{1..2}_{1..3}?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'data String') as data order by data") @@ -42,12 +44,14 @@ def test_url_with_globs(started_cluster): def test_url_with_globs_and_failover(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage_1_1", "1\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_2", "2\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_3", "3\n") - started_cluster.hdfs_api.write_data("/simple_storage_3_1", "4\n") - started_cluster.hdfs_api.write_data("/simple_storage_3_2", "5\n") - started_cluster.hdfs_api.write_data("/simple_storage_3_3", "6\n") + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage_1_1", "1\n") + hdfs_api.write_data("/simple_storage_1_2", "2\n") + hdfs_api.write_data("/simple_storage_1_3", "3\n") + hdfs_api.write_data("/simple_storage_3_1", "4\n") + hdfs_api.write_data("/simple_storage_3_2", "5\n") + hdfs_api.write_data("/simple_storage_3_3", "6\n") result = node1.query( "select * from url('http://hdfs1:50075/webhdfs/v1/simple_storage_{0|1|2|3}_{1..3}?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'data String') as data order by data") diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 4ccdc404fc9..713a6063db1 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -3,6 +3,7 @@ import os.path as p import random import subprocess import threading +import logging import time from random import randrange @@ -19,61 +20,10 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/rabbitmq.xml', 'configs/log_conf.xml'], with_rabbitmq=True) -# clickhouse_path_dir='clickhouse_path') -rabbitmq_id = '' # Helpers -def check_rabbitmq_is_available(): - p = subprocess.Popen(('docker', - 'exec', - '-i', - rabbitmq_id, - 'rabbitmqctl', - 'await_startup'), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def enable_consistent_hash_plugin(): - p = subprocess.Popen(('docker', - 'exec', - '-i', - rabbitmq_id, - "rabbitmq-plugins", "enable", "rabbitmq_consistent_hash_exchange"), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_rabbitmq_is_available(max_retries=50): - retries = 0 - while True: - if check_rabbitmq_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise "RabbitMQ is not available" - print("Waiting for RabbitMQ to start up") - time.sleep(1) - - -def wait_rabbitmq_plugin_enabled(max_retries=50): - retries = 0 - while True: - if enable_consistent_hash_plugin(): - break - else: - retries += 1 - if retries > max_retries: - raise "RabbitMQ plugin is not available" - print("Waiting for plugin") - time.sleep(1) - - def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.reference'): fpath = p.join(p.dirname(__file__), ref_file) with open(fpath) as reference: @@ -83,13 +33,13 @@ def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.refe return TSV(result) == TSV(reference) -def kill_rabbitmq(): +def kill_rabbitmq(rabbitmq_id): p = subprocess.Popen(('docker', 'stop', rabbitmq_id), stdout=subprocess.PIPE) p.communicate() return p.returncode == 0 -def revive_rabbitmq(): +def revive_rabbitmq(rabbitmq_id): p = subprocess.Popen(('docker', 'start', rabbitmq_id), stdout=subprocess.PIPE) p.communicate() return p.returncode == 0 @@ -100,10 +50,8 @@ def revive_rabbitmq(): @pytest.fixture(scope="module") def rabbitmq_cluster(): try: - global rabbitmq_id cluster.start() - rabbitmq_id = instance.cluster.rabbitmq_docker_id - print(("rabbitmq_id is {}".format(rabbitmq_id))) + logging.debug("rabbitmq_id is {}".format(instance.cluster.rabbitmq_docker_id)) instance.query('CREATE DATABASE test') yield cluster @@ -114,8 +62,6 @@ def rabbitmq_cluster(): @pytest.fixture(autouse=True) def rabbitmq_setup_teardown(): - wait_rabbitmq_is_available() - wait_rabbitmq_plugin_enabled() print("RabbitMQ is available - running test") yield # run test instance.query('DROP TABLE IF EXISTS test.rabbitmq') @@ -135,7 +81,7 @@ def test_rabbitmq_select(rabbitmq_cluster): '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -184,7 +130,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -227,7 +173,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -270,7 +216,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -310,7 +256,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -353,7 +299,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -403,7 +349,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -451,7 +397,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -504,7 +450,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(rabbitmq_messages)] credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -568,7 +514,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -645,7 +591,7 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -703,7 +649,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -756,7 +702,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -985,7 +931,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1058,7 +1004,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1144,7 +1090,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1222,7 +1168,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1301,7 +1247,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1398,7 +1344,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1452,7 +1398,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1519,7 +1465,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1597,7 +1543,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -1677,7 +1623,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1700,9 +1646,9 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): while int(instance.query('SELECT count() FROM test.view')) == 0: time.sleep(0.1) - kill_rabbitmq() + kill_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) time.sleep(4) - revive_rabbitmq() + revive_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) while True: result = instance.query('SELECT count(DISTINCT key) FROM test.view') @@ -1737,7 +1683,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): messages_num = 150000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1763,9 +1709,9 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): while int(instance.query('SELECT count() FROM test.view')) == 0: time.sleep(0.1) - kill_rabbitmq() + kill_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) time.sleep(8) - revive_rabbitmq() + revive_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) # while int(instance.query('SELECT count() FROM test.view')) == 0: # time.sleep(0.1) @@ -1809,7 +1755,7 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1895,7 +1841,7 @@ def test_rabbitmq_no_connection_at_startup(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() for i in range(messages_num): @@ -1931,7 +1877,7 @@ def test_rabbitmq_format_factory_settings(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1983,7 +1929,7 @@ def test_rabbitmq_vhost(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() channel.basic_publish(exchange='vhost', routing_key='', body=json.dumps({'key': 1, 'value': 2})) From fddc72fb494e2b4eb8659d360cf08568e97984d5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 30 Apr 2021 15:22:37 +0300 Subject: [PATCH 115/652] fixes --- .../runner/compose/docker_compose_rabbitmq.yml | 5 ++--- tests/integration/helpers/cluster.py | 12 ++++++++++-- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index d16a19f7ac2..997a346779c 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -4,9 +4,8 @@ services: rabbitmq1: image: rabbitmq:3-management hostname: rabbitmq1 - ports: - - ${RABBITMQ_EXTERNAL_PORT}:${RABBITMQ_INTERNAL_PORT} - - ${RABBITMQ_EXTERNAL_HTTP_PORT}:${RABBITMQ_INTERNAL_HTTP_PORT} + expose: + - ${RABBITMQ_PORT} environment: RABBITMQ_DEFAULT_USER: "root" RABBITMQ_DEFAULT_PASS: "clickhouse" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 35adf00c75b..a4cfc0cdc4d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -29,6 +29,7 @@ from dict2xml import dict2xml from kazoo.client import KazooClient from kazoo.exceptions import KazooException from minio import Minio +from minio.deleteobjects import DeleteObject import docker @@ -922,7 +923,7 @@ class ClickHouseCluster: raise Exception("Cannot wait Postgres container") - def wait_rabbitmq_to_start(self, timeout=30): + def wait_rabbitmq_to_start(self, timeout=180): self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host) start = time.time() @@ -1043,6 +1044,13 @@ class ClickHouseCluster: for bucket in buckets: if minio_client.bucket_exists(bucket): + delete_object_list = map( + lambda x: DeleteObject(x.object_name), + minio_client.list_objects(bucket, recursive=True), + ) + errors = minio_client.remove_objects(bucket, delete_object_list) + for error in errors: + logging.error(f"Error occured when deleting object {error}") minio_client.remove_bucket(bucket) minio_client.make_bucket(bucket) logging.debug("S3 bucket '%s' created", bucket) @@ -1650,7 +1658,7 @@ class ClickHouseInstance: def contains_in_log(self, substring): result = self.exec_in_container( - ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) + ["bash", "-c", '[ -f /var/log/clickhouse-server/clickhouse-server.log ] && grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) return len(result) > 0 def grep_in_log(self, substring): From d12e78609f1f2ff24b8643cef952fe2453b3f065 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 30 Apr 2021 17:34:21 +0300 Subject: [PATCH 116/652] py2 runner --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index d203d74218f..bbd7f73ae0d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -229,7 +229,7 @@ if __name__ == "__main__": if args.tmpfs: dockerd_internal_volume = "--tmpfs /var/lib/docker -e DOCKER_RAMDISK=true" elif args.dockerd_volume: - dockerd_internal_volume = f"--mount type=bind,source={args.dockerd_volume},target=/var/lib/docker" + dockerd_internal_volume = "--mount type=bind,source={},target=/var/lib/docker".format(args.dockerd_volume) else: subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) dockerd_internal_volume = "--volume={}_volume:/var/lib/docker".format(CONTAINER_NAME) From cb53bbb7b09d551fe276b59abe38a57694e091dd Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 14:57:22 +0300 Subject: [PATCH 117/652] add experimental codecs flag, add integration test for experimental codecs --- programs/compressor/Compressor.cpp | 37 ++++++ src/CMakeLists.txt | 24 ++++ src/Client/Connection.cpp | 2 +- src/Compression/CompressionCodecDensity.cpp | 106 ++++++++++++++++++ src/Compression/CompressionCodecDensity.h | 35 ++++++ src/Compression/CompressionCodecLZSSE2.cpp | 91 +++++++++++++++ src/Compression/CompressionCodecLZSSE2.h | 33 ++++++ src/Compression/CompressionCodecLZSSE4.cpp | 91 +++++++++++++++ src/Compression/CompressionCodecLZSSE4.h | 33 ++++++ src/Compression/CompressionCodecLZSSE8.cpp | 90 +++++++++++++++ src/Compression/CompressionCodecLZSSE8.h | 33 ++++++ src/Compression/CompressionCodecLizard.cpp | 88 +++++++++++++++ src/Compression/CompressionCodecLizard.h | 34 ++++++ src/Compression/CompressionFactory.cpp | 46 +++++++- src/Compression/CompressionFactory.h | 10 +- src/Compression/CompressionInfo.h | 10 +- src/Core/Settings.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 3 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/AlterCommands.cpp | 8 +- src/Storages/ColumnsDescription.cpp | 2 +- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- .../test_non_default_compression/test.py | 19 +++- 24 files changed, 781 insertions(+), 21 deletions(-) create mode 100644 src/Compression/CompressionCodecDensity.cpp create mode 100644 src/Compression/CompressionCodecDensity.h create mode 100644 src/Compression/CompressionCodecLZSSE2.cpp create mode 100644 src/Compression/CompressionCodecLZSSE2.h create mode 100644 src/Compression/CompressionCodecLZSSE4.cpp create mode 100644 src/Compression/CompressionCodecLZSSE4.h create mode 100644 src/Compression/CompressionCodecLZSSE8.cpp create mode 100644 src/Compression/CompressionCodecLZSSE8.h create mode 100644 src/Compression/CompressionCodecLizard.cpp create mode 100644 src/Compression/CompressionCodecLizard.h diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index d47372631fe..d5471a62afe 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -19,6 +20,8 @@ #include #include +#include +#include namespace DB { @@ -77,6 +80,12 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") + ("lizard", "use Lizard instread of LZ4") + ("lzsse2", "use lzsse2 instread of LZ4") + ("lzsse4", "use lzsse4 instread of LZ4") + ("lzsse8", "use lzsse8 instread of LZ4") + ("density", "use Density instread of LZ4") + ("param", po::value(), "extra params for compresion algorithm") ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") ("level", po::value(), "compression level for codecs specified via flags") ("none", "use no compression instead of LZ4") @@ -103,6 +112,11 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool decompress = options.count("decompress"); bool use_lz4hc = options.count("hc"); bool use_zstd = options.count("zstd"); + bool use_lizard = options.count("lizard"); + bool use_lzsse2 = options.count("lzsse2"); + bool use_lzsse4 = options.count("lzsse4"); + bool use_lzsse8 = options.count("lzsse8"); + bool use_density = options.count("density"); bool stat_mode = options.count("stat"); bool use_none = options.count("none"); unsigned block_size = options["block-size"].as(); @@ -110,6 +124,10 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) if (options.count("codec")) codecs = options["codec"].as>(); + std::optional param; + if (options.count("param")) + param = options["param"].as(); + if ((use_lz4hc || use_zstd || use_none) && !codecs.empty()) throw Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", ErrorCodes::BAD_ARGUMENTS); @@ -122,6 +140,16 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) method_family = "LZ4HC"; else if (use_zstd) method_family = "ZSTD"; + else if (use_lizard) + method_family = "Lizard"; + else if (use_lzsse2) + method_family = "LZSSE2"; + else if (use_lzsse4) + method_family = "LZSSE4"; + else if (use_lzsse8) + method_family = "LZSSE8"; + else if (use_density) + method_family = "Density"; else if (use_none) method_family = "NONE"; @@ -137,6 +165,8 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) std::string codecs_line = boost::algorithm::join(codecs, ","); auto ast = parseQuery(codec_parser, "(" + codecs_line + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); codec = CompressionCodecFactory::instance().get(ast, nullptr); + } else if (param.has_value()) { + codec = CompressionCodecFactory::instance().get(method_family, level, param); } else codec = CompressionCodecFactory::instance().get(method_family, level); @@ -155,6 +185,9 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else wb = std::make_unique(STDOUT_FILENO); + + struct tms tv1, tv2; + times(&tv1); if (stat_mode) { /// Output statistic for compressed file. @@ -185,6 +218,10 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) CompressedWriteBuffer to(*wb, codec, block_size); copyData(*rb, to); } + times(&tv2); + + int tics_per_second = sysconf(_SC_CLK_TCK); + std::cerr << static_cast(tv2.tms_utime - tv1.tms_utime) / tics_per_second << std::endl; } catch (...) { diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 69a84dbeb2c..ea8525212cc 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -377,6 +377,30 @@ if (XZ_LIBRARY) target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${XZ_INCLUDE_DIR}) endif() +set (LIZARD_LIBRARY lizard) +set (LIZARD_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/lizard/lib/) +if (LIZARD_LIBRARY) + dbms_target_link_libraries(PRIVATE ${LIZARD_LIBRARY}) + #target_link_libraries (clickhouse_compression PUBLIC ${LZMA_LIBRARY}) + #target_include_directories (clickhouse_compression SYSTEM BEFORE PUBLIC ${}) +endif() + +set (DENSITY_LIBRARY density) +set (DENSITY_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/density/src) +if (DENSITY_LIBRARY) + dbms_target_link_libraries(PRIVATE ${DENSITY_LIBRARY}) + #target_link_libraries (clickhouse_compression PUBLIC ${LZMA_LIBRARY}) + #target_include_directories (clickhouse_compression SYSTEM BEFORE PUBLIC ${}) +endif() + +set (LZSSE_LIBRARY lzsse) +set (LZSSE_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/lzsse/) +if (LZSSE_LIBRARY) + dbms_target_link_libraries(PRIVATE ${LZSSE_LIBRARY}) + #target_link_libraries (clickhouse_compression PUBLIC ${LZMA_LIBRARY}) + #target_include_directories (clickhouse_compression SYSTEM BEFORE PUBLIC ${}) +endif() + if (USE_ICU) dbms_target_link_libraries (PRIVATE ${ICU_LIBRARIES}) dbms_target_include_directories (SYSTEM PRIVATE ${ICU_INCLUDE_DIRS}) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d26f7454bcc..87f768d7e75 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -424,7 +424,7 @@ void Connection::sendQuery( if (method == "ZSTD") level = settings->network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs); compression_codec = CompressionCodecFactory::instance().get(method, level); } else diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp new file mode 100644 index 00000000000..841638ac6ed --- /dev/null +++ b/src/Compression/CompressionCodecDensity.cpp @@ -0,0 +1,106 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +CompressionCodecDensity::CompressionCodecDensity(DENSITY_ALGORITHM algo_) : algo(algo_) +{ + setCodecDescription("Density", {std::make_shared(static_cast(algo))}); +} + +uint8_t CompressionCodecDensity::getMethodByte() const +{ + return static_cast(CompressionMethodByte::Density); +} + +void CompressionCodecDensity::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +UInt32 CompressionCodecDensity::getMaxCompressedDataSize(UInt32 uncompressed_size) const +{ + return density_compress_safe_size(uncompressed_size); +} + +UInt32 CompressionCodecDensity::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + density_processing_result res = density_compress(reinterpret_cast(source), source_size, reinterpret_cast(dest), density_compress_safe_size(source_size), algo); + if (res.state != DENSITY_STATE_OK) + throw Exception("Cannot compress block with Density; ", ErrorCodes::CANNOT_COMPRESS); + return res.bytesWritten; +} + +void CompressionCodecDensity::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +{ + density_processing_result res = density_decompress(reinterpret_cast(source), source_size, reinterpret_cast(dest), density_decompress_safe_size(uncompressed_size)); + if (res.state != DENSITY_STATE_OK) + throw Exception("Cannot decompress block with Density; ", ErrorCodes::CANNOT_DECOMPRESS); +} + +void registerCodecDensity(CompressionCodecFactory & factory) +{ + UInt8 method_code = UInt8(CompressionMethodByte::Density); + factory.registerCompressionCodec( + "Density", + method_code, + [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + DENSITY_ALGORITHM algo = CompressionCodecDensity::DENSITY_DEFAULT_ALGO; + //std::cerr << arguments << std::endl; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() != 1) + throw Exception( + "Deisnty codec must have 1 parameter, given " + std::to_string(arguments->children.size()), + ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + const auto children = arguments->children; + + const auto * algo_literal = children[0]->as(); + if (!algo_literal) + throw Exception("Density codec argument must be string (algorithm)", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + + + if (algo_literal->value.getType() == Field::Types::Which::UInt64) { + const auto algorithm = algo_literal->value.safeGet(); + if (algorithm == 3) { + algo = DENSITY_ALGORITHM_LION; + } else if (algorithm == 2) { + algo = DENSITY_ALGORITHM_CHEETAH; + } else if (algorithm == 1) { + algo = DENSITY_ALGORITHM_CHAMELEON; + } else { + throw Exception("Density codec argument may be LION, CHAMELEON, CHEETAH", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + } + } else { + const auto algorithm = algo_literal->value.safeGet(); + if (algorithm == "LION") { + algo = DENSITY_ALGORITHM_LION; + } else if (algorithm == "CHAMELEON") { + algo = DENSITY_ALGORITHM_CHAMELEON; + } else if (algorithm == "CHEETAH") { + algo = DENSITY_ALGORITHM_CHEETAH; + } else { + throw Exception("Density codec argument may be LION, CHAMELEON, CHEETAH", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + } + } + } + + return std::make_shared(algo); + }); +} + +} diff --git a/src/Compression/CompressionCodecDensity.h b/src/Compression/CompressionCodecDensity.h new file mode 100644 index 00000000000..cfc8bfbd76a --- /dev/null +++ b/src/Compression/CompressionCodecDensity.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class CompressionCodecDensity : public ICompressionCodec +{ +public: + static constexpr auto DENSITY_DEFAULT_ALGO = DENSITY_ALGORITHM_CHAMELEON; // by default aim on speed + + CompressionCodecDensity(DENSITY_ALGORITHM algo_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + + bool isGenericCompression() const override { return true; } + +private: + const DENSITY_ALGORITHM algo; +}; + +} \ No newline at end of file diff --git a/src/Compression/CompressionCodecLZSSE2.cpp b/src/Compression/CompressionCodecLZSSE2.cpp new file mode 100644 index 00000000000..215b93e6721 --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE2.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +CompressionCodecLZSSE2::CompressionCodecLZSSE2(int level_) : level(level_) +{ + + setCodecDescription("LZSSE2", {std::make_shared(static_cast(level))}); +} + +uint8_t CompressionCodecLZSSE2::getMethodByte() const +{ + return static_cast(CompressionMethodByte::LZSSE2); +} + +void CompressionCodecLZSSE2::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +UInt32 CompressionCodecLZSSE2::getMaxCompressedDataSize(UInt32 uncompressed_size) const +{ + return uncompressed_size; +} + +UInt32 CompressionCodecLZSSE2::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + UInt32 res; + LZSSE2_OptimalParseState* state = LZSSE2_MakeOptimalParseState(source_size); + res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE2_FreeOptimalParseState(state); + + if (res == 0) + throw Exception("Cannot compress block with LZSSE2; ", ErrorCodes::CANNOT_COMPRESS); + + return res; +} + +void CompressionCodecLZSSE2::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +{ + UInt32 res; + res = LZSSE2_Decompress(source, source_size, dest, uncompressed_size); + + if (res < uncompressed_size) + throw Exception("Cannot decompress block with LZSSE2; ", ErrorCodes::CANNOT_DECOMPRESS); +} + +void registerCodecLZSSE2(CompressionCodecFactory & factory) +{ + UInt8 method_code = UInt8(CompressionMethodByte::LZSSE2); + factory.registerCompressionCodec( + "LZSSE2", + method_code, + [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = 1; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() != 1) + throw Exception( + "LZSSE2 codec must have 1 parameter, given " + std::to_string(arguments->children.size()), + ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + const auto children = arguments->children; + const auto * level_literal = children[0]->as(); + if (!level_literal) + throw Exception("LZSSE2 first codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + + level = level_literal->value.safeGet(); + + } + + return std::make_shared(level); + }); +} + +} diff --git a/src/Compression/CompressionCodecLZSSE2.h b/src/Compression/CompressionCodecLZSSE2.h new file mode 100644 index 00000000000..d26bb3e2cf2 --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE2.h @@ -0,0 +1,33 @@ +#pragma once + +#include + + +namespace DB +{ +class CompressionCodecLZSSE2 : public ICompressionCodec +{ +public: + + CompressionCodecLZSSE2(int level_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + + bool isGenericCompression() const override { return true; } + +private: + const int level; +}; + +} \ No newline at end of file diff --git a/src/Compression/CompressionCodecLZSSE4.cpp b/src/Compression/CompressionCodecLZSSE4.cpp new file mode 100644 index 00000000000..9b687d866a1 --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE4.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +CompressionCodecLZSSE4::CompressionCodecLZSSE4(int level_) : level(level_) +{ + + setCodecDescription("LZSSE4", {std::make_shared(static_cast(level))}); +} + +uint8_t CompressionCodecLZSSE4::getMethodByte() const +{ + return static_cast(CompressionMethodByte::LZSSE4); +} + +void CompressionCodecLZSSE4::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +UInt32 CompressionCodecLZSSE4::getMaxCompressedDataSize(UInt32 uncompressed_size) const +{ + return uncompressed_size; +} + +UInt32 CompressionCodecLZSSE4::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + UInt32 res; + LZSSE4_OptimalParseState* state = LZSSE4_MakeOptimalParseState(source_size); + res = LZSSE4_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE4_FreeOptimalParseState(state); + + if (res == 0) + throw Exception("Cannot compress block with LZSSE4; ", ErrorCodes::CANNOT_COMPRESS); + + return res; +} + +void CompressionCodecLZSSE4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +{ + UInt32 res; + res = LZSSE4_Decompress(source, source_size, dest, uncompressed_size); + + if (res < uncompressed_size) + throw Exception("Cannot decompress block with LZSSE4; ", ErrorCodes::CANNOT_DECOMPRESS); +} + +void registerCodecLZSSE4(CompressionCodecFactory & factory) +{ + UInt8 method_code = UInt8(CompressionMethodByte::LZSSE4); + factory.registerCompressionCodec( + "LZSSE4", + method_code, + [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = 1; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() != 1) + throw Exception( + "LZSSE4 codec must have 1 parameter, given " + std::to_string(arguments->children.size()), + ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + const auto children = arguments->children; + const auto * level_literal = children[0]->as(); + if (!level_literal) + throw Exception("LZSSE4 first codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + + level = level_literal->value.safeGet(); + + } + + return std::make_shared(level); + }); +} + +} diff --git a/src/Compression/CompressionCodecLZSSE4.h b/src/Compression/CompressionCodecLZSSE4.h new file mode 100644 index 00000000000..ed1cc8fb26b --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE4.h @@ -0,0 +1,33 @@ +#pragma once + +#include + + +namespace DB +{ +class CompressionCodecLZSSE4 : public ICompressionCodec +{ +public: + + CompressionCodecLZSSE4(int level_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + + bool isGenericCompression() const override { return true; } + +private: + const int level; +}; + +} \ No newline at end of file diff --git a/src/Compression/CompressionCodecLZSSE8.cpp b/src/Compression/CompressionCodecLZSSE8.cpp new file mode 100644 index 00000000000..04e73121fc6 --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE8.cpp @@ -0,0 +1,90 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +CompressionCodecLZSSE8::CompressionCodecLZSSE8(int level_) : level(level_) +{ + setCodecDescription("LZSSE8", {std::make_shared(static_cast(level))}); +} + +uint8_t CompressionCodecLZSSE8::getMethodByte() const +{ + return static_cast(CompressionMethodByte::LZSSE8); +} + +void CompressionCodecLZSSE8::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +UInt32 CompressionCodecLZSSE8::getMaxCompressedDataSize(UInt32 uncompressed_size) const +{ + return uncompressed_size; +} + +UInt32 CompressionCodecLZSSE8::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + UInt32 res; + LZSSE8_OptimalParseState* state = LZSSE8_MakeOptimalParseState(source_size); + res = LZSSE8_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE8_FreeOptimalParseState(state); + + if (res == 0) + throw Exception("Cannot compress block with LZSSE; ", ErrorCodes::CANNOT_COMPRESS); + + return res; +} + +void CompressionCodecLZSSE8::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +{ + UInt32 res; + res = LZSSE8_Decompress(source, source_size, dest, uncompressed_size); + + if (res < uncompressed_size) + throw Exception("Cannot decompress block with LZSSE; ", ErrorCodes::CANNOT_DECOMPRESS); +} + +void registerCodecLZSSE8(CompressionCodecFactory & factory) +{ + UInt8 method_code = UInt8(CompressionMethodByte::LZSSE8); + factory.registerCompressionCodec( + "LZSSE8", + method_code, + [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = 1; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() != 1) + throw Exception( + "LZSSE8 codec must have 1 parameter, given " + std::to_string(arguments->children.size()), + ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + const auto children = arguments->children; + const auto * level_literal = children[0]->as(); + if (!level_literal) + throw Exception("LZSSE8 first codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + + level = level_literal->value.safeGet(); + + } + + return std::make_shared(level); + }); +} + +} diff --git a/src/Compression/CompressionCodecLZSSE8.h b/src/Compression/CompressionCodecLZSSE8.h new file mode 100644 index 00000000000..c9d006ca13b --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE8.h @@ -0,0 +1,33 @@ +#pragma once + +#include + + +namespace DB +{ +class CompressionCodecLZSSE8 : public ICompressionCodec +{ +public: + + CompressionCodecLZSSE8(int level_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + + bool isGenericCompression() const override { return true; } + +private: + const int level; +}; + +} \ No newline at end of file diff --git a/src/Compression/CompressionCodecLizard.cpp b/src/Compression/CompressionCodecLizard.cpp new file mode 100644 index 00000000000..a4106965bd5 --- /dev/null +++ b/src/Compression/CompressionCodecLizard.cpp @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; +} + +CompressionCodecLizard::CompressionCodecLizard(int level_) : level(level_) +{ + setCodecDescription("Lizard", {std::make_shared(static_cast(level))}); +} + +uint8_t CompressionCodecLizard::getMethodByte() const +{ + return static_cast(CompressionMethodByte::Lizard); +} + +void CompressionCodecLizard::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +UInt32 CompressionCodecLizard::getMaxCompressedDataSize(UInt32 uncompressed_size) const +{ + return Lizard_compressBound(uncompressed_size); +} + +UInt32 CompressionCodecLizard::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + int res = Lizard_compress(source, dest, source_size, Lizard_compressBound(source_size), level); + + if (res == 0) + throw Exception("Cannot compress block with Lizard; ", ErrorCodes::CANNOT_COMPRESS); + return res; +} + +void CompressionCodecLizard::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +{ + int res = Lizard_decompress_safe(source, dest, source_size, uncompressed_size); + + if (res < 0) + throw Exception("Cannot compress block with Lizard; ", ErrorCodes::CANNOT_DECOMPRESS); +} + +void registerCodecLizard(CompressionCodecFactory & factory) +{ + UInt8 method_code = UInt8(CompressionMethodByte::Lizard); + factory.registerCompressionCodec( + "Lizard", + method_code, + [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = CompressionCodecLizard::LIZARD_DEFAULT_LEVEL; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() > 1) + throw Exception( + "Lizard codec must have 1 parameter, given " + std::to_string(arguments->children.size()), + ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + const auto children = arguments->children; + const auto * literal = children[0]->as(); + if (!literal) + throw Exception("Lizard codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + + level = literal->value.safeGet(); + // compression level will be truncated to LIZARD_MAX_CLEVEL if it is greater and to LIZARD_MIN_CLEVEL if it is less + //if (level > 1)//ZSTD_maxCLevel()) + // throw Exception("Lizard codec can't have level more that " + toString(1/*ZSTD_maxCLevel()*/) + ", given " + toString(level), ErrorCodes::ILLEGAL_CODEC_PARAMETER); + } + + return std::make_shared(level); + }); +} + +} diff --git a/src/Compression/CompressionCodecLizard.h b/src/Compression/CompressionCodecLizard.h new file mode 100644 index 00000000000..95bef7ab39e --- /dev/null +++ b/src/Compression/CompressionCodecLizard.h @@ -0,0 +1,34 @@ +#pragma once + +#include + + +namespace DB +{ +class CompressionCodecLizard : public ICompressionCodec +{ +public: + static constexpr auto LIZARD_DEFAULT_LEVEL = 1; + + CompressionCodecLizard(int level_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + + bool isGenericCompression() const override { return true; } + +private: + const int level; +}; + +} \ No newline at end of file diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 0ff9797aeaf..9ca64e36468 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -44,7 +44,24 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std } } -void CompressionCodecFactory::validateCodec(const String & family_name, std::optional level, bool sanity_check) const +CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level, std::optional param) const +{ + if (level && param) + { + auto level_literal = std::make_shared(static_cast(*level)); + auto param_literal = std::make_shared(static_cast(*param)); + return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), level_literal, param_literal)), {}); + } + else if (param) + { + auto param_literal = std::make_shared(static_cast(*param)); + return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), param_literal)), {}); + } else { + return get(family_name, level); + } +} + +void CompressionCodecFactory::validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const { if (family_name.empty()) throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS); @@ -52,16 +69,16 @@ void CompressionCodecFactory::validateCodec(const String & family_name, std::opt if (level) { auto literal = std::make_shared(static_cast(*level)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check, allow_experimental_codecs); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), {}, sanity_check); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), {}, sanity_check, allow_experimental_codecs); } } -ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const +ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const { if (const auto * func = ast->as()) { @@ -90,6 +107,16 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr else throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + if (sanity_check && !allow_experimental_codecs) { + if (codec_family_name == "Lizard" || + codec_family_name == "Density" || + codec_family_name == "LZSSE2" || + codec_family_name == "LZSSE4" || + codec_family_name == "LZSSE8") { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Experimental codecs Lizard, Density and LZSSE* are not allowed, please enable allow_experimental_codecs flag."); + } + } /// Default codec replaced with current default codec which may depend on different /// settings (and properties of data) in runtime. CompressionCodecPtr result_codec; @@ -172,6 +199,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); } + /// For columns with nested types like Tuple(UInt32, UInt64) we /// obviously cannot substitute parameters for codecs which depend on /// data type, because for the first column Delta(4) is suitable and @@ -318,6 +346,11 @@ void registerCodecT64(CompressionCodecFactory & factory); void registerCodecDoubleDelta(CompressionCodecFactory & factory); void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); +void registerCodecLizard(CompressionCodecFactory & factory); +void registerCodecDensity(CompressionCodecFactory & factory); +void registerCodecLZSSE2(CompressionCodecFactory & factory); +void registerCodecLZSSE4(CompressionCodecFactory & factory); +void registerCodecLZSSE8(CompressionCodecFactory & factory); CompressionCodecFactory::CompressionCodecFactory() { @@ -330,6 +363,11 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecDoubleDelta(*this); registerCodecGorilla(*this); registerCodecMultiple(*this); + registerCodecLizard(*this); + registerCodecDensity(*this); + registerCodecLZSSE2(*this); + registerCodecLZSSE4(*this); + registerCodecLZSSE8(*this); default_codec = get("LZ4", {}); } diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index dc06e853898..34dd8633611 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -38,16 +38,16 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const; /// Just wrapper for previous method. - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check) const + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const { - return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check); + return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check, allow_experimental_codecs); } /// Validate codecs AST specified by user - void validateCodec(const String & family_name, std::optional level, bool sanity_check) const; + void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const; /// Get codec by AST and possible column_type. Some codecs can use /// information about type to improve inner settings, but every codec should @@ -72,6 +72,8 @@ public: /// For backward compatibility with config settings CompressionCodecPtr get(const String & family_name, std::optional level) const; + CompressionCodecPtr get(const String & family_name, std::optional level, std::optional param) const; + /// Register codec with parameters and column type void registerCompressionCodecWithType(const String & family_name, std::optional byte_code, CreatorWithType creator); /// Register codec with parameters diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h index 290ebaeb504..d1d4ad4618f 100644 --- a/src/Compression/CompressionInfo.h +++ b/src/Compression/CompressionInfo.h @@ -41,8 +41,14 @@ enum class CompressionMethodByte : uint8_t Multiple = 0x91, Delta = 0x92, T64 = 0x93, - DoubleDelta = 0x94, - Gorilla = 0x95, + DoubleDelta = 0x94, + Gorilla = 0x95, + Lizard = 0x96, + Density = 0x97, + LZSSE2 = 0x98, + LZSSE4 = 0x99, + LZSSE8 = 0xa0, + }; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 42a20441a2e..9ce06542d2c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -238,6 +238,7 @@ class IColumn; M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ + M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs(LZSSE*, Lizard, Density).", 0) \ M(UInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \ M(UInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d45d02243fb..6da714ee6b9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -437,6 +437,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context_); bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs; + bool allow_experimental_codecs = context_->getSettingsRef().allow_experimental_codecs; ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); for (auto ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it) @@ -471,7 +472,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.default_specifier == "ALIAS") throw Exception{"Cannot specify codec for column type ALIAS", ErrorCodes::BAD_ARGUMENTS}; column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - col_decl.codec, column.type, sanity_check_compression_codecs); + col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs); } if (col_decl.ttl) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 916f29ba1d4..e0abbd63b1d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1398,7 +1398,7 @@ void TCPHandler::initBlockOutput(const Block & block) if (state.compression == Protocol::Compression::Enable) { - CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs); state.maybe_compressed_out = std::make_shared( *out, CompressionCodecFactory::instance().get(method, level)); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index e3177c167c5..b7e7893487b 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -313,7 +313,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) column.comment = *comment; if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true); column.ttl = ttl; @@ -354,7 +354,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) else { if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true); if (comment) column.comment = *comment; @@ -907,7 +907,7 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPt ErrorCodes::BAD_ARGUMENTS}; if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -927,7 +927,7 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPt ErrorCodes::NOT_IMPLEMENTED}; if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); auto column_default = all_columns.getDefault(column_name); if (column_default) { diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 545911f1465..43a385bda1d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -128,7 +128,7 @@ void ColumnDescription::readText(ReadBuffer & buf) comment = col_ast->comment->as().value.get(); if (col_ast->codec) - codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false); + codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true); if (col_ast->ttl) ttl = col_ast->ttl; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index a4aa2779771..b91f5b10cf2 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -632,7 +632,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: if (compression_method == "ZSTD") compression_level = settings.network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs); CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 95ea4f07f18..9bb7d0c22f2 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -289,7 +289,7 @@ TTLDescription TTLDescription::getTTLFromAST( { result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs); + ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); } } diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index 03210e47081..678b2167595 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -17,7 +17,8 @@ node4 = cluster.add_instance('node4', user_configs=['configs/enable_uncompressed node5 = cluster.add_instance('node5', main_configs=['configs/zstd_compression_by_default.xml'], user_configs=['configs/enable_uncompressed_cache.xml', 'configs/allow_suspicious_codecs.xml']) - +node6 = cluster.add_instance('node6', main_configs=['configs/allow_experimental_codecs.xml'], + user_configs=['configs/allow_suspicious_codecs.xml']) @pytest.fixture(scope="module") def start_cluster(): @@ -137,3 +138,19 @@ def test_uncompressed_cache_plus_zstd_codec(start_cluster): assert node5.query( "SELECT max(length(data)) from compression_codec_multiple_with_key GROUP BY data ORDER BY max(length(data)) DESC LIMIT 1") == "10000\n" + +def test_experimental_codecs(start_cluster): + node6.query(""" + CREATE TABLE compression_experimental_codecs ( + somedate Date CODEC(Lizard(12)), + id UInt64 CODEC(Density('LION')), + data String CODEC(LZSSE4(3)) + ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; + """) + + node6.query( + "INSERT INTO compression_experimental_codecs VALUES(toDate('2018-10-12'), 100000, '{}')".format( + 'c' * 10000)) + + assert node6.query( + "SELECT max(length(data)) from compression_experimental_codecs GROUP BY data ORDER BY max(length(data)) DESC LIMIT 1") == "10000\n" From 9cfddff290e347b26558366a88951d5b3665ba66 Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 15:08:58 +0300 Subject: [PATCH 118/652] add new codecs from contrib to cmake --- contrib/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 9eafec23f51..18da3ab5dd5 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -54,6 +54,10 @@ if (USE_INTERNAL_XZ_LIBRARY) add_subdirectory (xz) endif() +add_subdirectory (lizard-cmake) +add_subdirectory (density-cmake) +add_subdirectory (lzsse-cmake) + add_subdirectory (poco-cmake) add_subdirectory (croaring-cmake) From 7c9acb9f532709daa0a50ee3656086d6d670ba0a Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 15:26:03 +0300 Subject: [PATCH 119/652] add dedicated cmake files for compression libraries --- contrib/density-cmake/CMakeLists.txt | 47 ++++++++++++++++++++++++++++ contrib/lizard-cmake/CMakeLists.txt | 19 +++++++++++ contrib/lzsse-cmake/CMakeLists.txt | 21 +++++++++++++ 3 files changed, 87 insertions(+) create mode 100644 contrib/density-cmake/CMakeLists.txt create mode 100644 contrib/lizard-cmake/CMakeLists.txt create mode 100644 contrib/lzsse-cmake/CMakeLists.txt diff --git a/contrib/density-cmake/CMakeLists.txt b/contrib/density-cmake/CMakeLists.txt new file mode 100644 index 00000000000..d48bbbf05b9 --- /dev/null +++ b/contrib/density-cmake/CMakeLists.txt @@ -0,0 +1,47 @@ +SET (USE_INTERNAL_DENSITY_LIBRARY 1) +SET (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/density") + +# cd contrib/density +# find . -name '*.c' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ ${LIBRARY_DIR}/' +SET (Sources + ${LIBRARY_DIR}/src/algorithms/algorithms.c + ${LIBRARY_DIR}/src/algorithms/chameleon/core/chameleon_decode.c + ${LIBRARY_DIR}/src/algorithms/chameleon/core/chameleon_encode.c + ${LIBRARY_DIR}/src/algorithms/cheetah/core/cheetah_decode.c + ${LIBRARY_DIR}/src/algorithms/cheetah/core/cheetah_encode.c + ${LIBRARY_DIR}/src/algorithms/dictionaries.c + ${LIBRARY_DIR}/src/algorithms/lion/core/lion_decode.c + ${LIBRARY_DIR}/src/algorithms/lion/core/lion_encode.c + ${LIBRARY_DIR}/src/algorithms/lion/forms/lion_form_model.c + ${LIBRARY_DIR}/src/buffers/buffer.c + ${LIBRARY_DIR}/src/globals.c + ${LIBRARY_DIR}/src/structure/header.c +) + +# cd contrib/density +# find . -name '*.h' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ ${LIBRARY_DIR}/' +SET (Headers + ${LIBRARY_DIR}/src/algorithms/algorithms.h + ${LIBRARY_DIR}/src/algorithms/chameleon/chameleon.h + ${LIBRARY_DIR}/src/algorithms/chameleon/core/chameleon_decode.h + ${LIBRARY_DIR}/src/algorithms/chameleon/core/chameleon_encode.h + ${LIBRARY_DIR}/src/algorithms/chameleon/dictionary/chameleon_dictionary.h + ${LIBRARY_DIR}/src/algorithms/cheetah/cheetah.h + ${LIBRARY_DIR}/src/algorithms/cheetah/core/cheetah_decode.h + ${LIBRARY_DIR}/src/algorithms/cheetah/core/cheetah_encode.h + ${LIBRARY_DIR}/src/algorithms/cheetah/dictionary/cheetah_dictionary.h + ${LIBRARY_DIR}/src/algorithms/dictionaries.h + ${LIBRARY_DIR}/src/algorithms/lion/core/lion_decode.h + ${LIBRARY_DIR}/src/algorithms/lion/core/lion_encode.h + ${LIBRARY_DIR}/src/algorithms/lion/dictionary/lion_dictionary.h + ${LIBRARY_DIR}/src/algorithms/lion/forms/lion_form_model.h + ${LIBRARY_DIR}/src/algorithms/lion/lion.h + ${LIBRARY_DIR}/src/buffers/buffer.h + ${LIBRARY_DIR}/src/density_api.h + ${LIBRARY_DIR}/src/globals.h + ${LIBRARY_DIR}/src/structure/header.h +) + +ADD_LIBRARY(density ${Sources} ${Headers}) + +target_include_directories (density PUBLIC ${LIBRARY_DIR}) \ No newline at end of file diff --git a/contrib/lizard-cmake/CMakeLists.txt b/contrib/lizard-cmake/CMakeLists.txt new file mode 100644 index 00000000000..eaf63db0acd --- /dev/null +++ b/contrib/lizard-cmake/CMakeLists.txt @@ -0,0 +1,19 @@ +SET (USE_INTERNAL_LIZARD_LIBRARY 1) +SET (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/lizard") + +SET (Sources + "${LIBRARY_DIR}/lib/lizard_decompress.c" + "${LIBRARY_DIR}/lib/lizard_compress.c" + "${LIBRARY_DIR}/lib/lizard_frame.c" + "${LIBRARY_DIR}/lib/xxhash/xxhash.c" +) + +SET (Headers + "${LIBRARY_DIR}/lib/lizard_compress.h" + "${LIBRARY_DIR}/lib/lizard_common.h" + "${LIBRARY_DIR}/lib/lizard_frame.h" +) + +ADD_LIBRARY(lizard ${Sources} ${Headers}) + +target_include_directories (lizard PUBLIC ${LIBRARY_DIR}) \ No newline at end of file diff --git a/contrib/lzsse-cmake/CMakeLists.txt b/contrib/lzsse-cmake/CMakeLists.txt new file mode 100644 index 00000000000..a463089a5b6 --- /dev/null +++ b/contrib/lzsse-cmake/CMakeLists.txt @@ -0,0 +1,21 @@ +SET (USE_INTERNAL_LZSSE_LIBRARY 1) +SET (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/lzsse") + +SET (Sources + "${LIBRARY_DIR}/lzsse2/lzsse2.cpp" + "${LIBRARY_DIR}/lzsse4/lzsse4.cpp" + "${LIBRARY_DIR}/lzsse8/lzsse8.cpp" +) + +SET (Headers + "${LIBRARY_DIR}/lzsse2/lzsse2.h" + "${LIBRARY_DIR}/lzsse4/lzsse4.h" + "${LIBRARY_DIR}/lzsse8/lzsse8.h" + "${LIBRARY_DIR}/lzsse2/lzsse2_platform.h" + "${LIBRARY_DIR}/lzsse4/lzsse4_platform.h" + "${LIBRARY_DIR}/lzsse8/lzsse8_platform.h" +) + +ADD_LIBRARY(lzsse ${Sources} ${Headers}) + +target_include_directories (lzsse PUBLIC ${LIBRARY_DIR}) \ No newline at end of file From 9f7733567b8198558da9a2b4450d5c99050a2d28 Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 15:36:49 +0300 Subject: [PATCH 120/652] extend modules --- .gitmodules | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.gitmodules b/.gitmodules index f7dcf5f4ac1..3e572e5280e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -227,3 +227,12 @@ [submodule "contrib/datasketches-cpp"] path = contrib/datasketches-cpp url = https://github.com/ClickHouse-Extras/datasketches-cpp.git +[submodule "contrib/lizard"] + path = contrib/lizard + url = https://github.com/inikep/lizard +[submodule "contrib/density"] + path = contrib/density + url = https://github.com/centaurean/density.git +[submodule "contrib/lzsse"] + path = contrib/lzsse + url = https://github.com/ConorStokes/LZSSE.git From a2f70306dff7fce1be79059713cfbd21c1720ed6 Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 15:55:03 +0300 Subject: [PATCH 121/652] add contribs --- contrib/density/.gitignore | 2 + contrib/density/.gitmodules | 6 + contrib/density/.travis.yml | 272 +++ contrib/density/CHANGELOG.md | 168 ++ contrib/density/LICENSE.md | 28 + contrib/density/README.md | 261 +++ contrib/density/appveyor.yml | 37 + contrib/density/benchmark/src/benchmark.c | 345 ++++ contrib/density/benchmark/src/benchmark.h | 129 ++ contrib/density/msvc/Density.sln | 21 + contrib/density/msvc/benchmark.vcxproj | 78 + .../density/msvc/benchmark.vcxproj.filters | 63 + contrib/density/msvc/density.vcxproj | 93 + contrib/density/msvc/density.vcxproj.filters | 135 ++ contrib/density/src/algorithms/algorithms.c | 43 + contrib/density/src/algorithms/algorithms.h | 78 + .../src/algorithms/chameleon/chameleon.h | 70 + .../chameleon/core/chameleon_decode.c | 254 +++ .../chameleon/core/chameleon_decode.h | 53 + .../chameleon/core/chameleon_encode.c | 179 ++ .../chameleon/core/chameleon_encode.h | 53 + .../dictionary/chameleon_dictionary.h | 63 + .../density/src/algorithms/cheetah/cheetah.h | 73 + .../algorithms/cheetah/core/cheetah_decode.c | 266 +++ .../algorithms/cheetah/core/cheetah_decode.h | 54 + .../algorithms/cheetah/core/cheetah_encode.c | 202 +++ .../algorithms/cheetah/core/cheetah_encode.h | 54 + .../cheetah/dictionary/cheetah_dictionary.h | 70 + contrib/density/src/algorithms/dictionaries.c | 48 + contrib/density/src/algorithms/dictionaries.h | 45 + .../src/algorithms/lion/core/lion_decode.c | 327 ++++ .../src/algorithms/lion/core/lion_decode.h | 54 + .../src/algorithms/lion/core/lion_encode.c | 298 ++++ .../src/algorithms/lion/core/lion_encode.h | 54 + .../lion/dictionary/lion_dictionary.h | 73 + .../algorithms/lion/forms/lion_form_model.c | 153 ++ .../algorithms/lion/forms/lion_form_model.h | 96 + contrib/density/src/algorithms/lion/lion.h | 94 + contrib/density/src/buffers/buffer.c | 220 +++ contrib/density/src/buffers/buffer.h | 58 + contrib/density/src/density_api.h | 220 +++ contrib/density/src/globals.c | 47 + contrib/density/src/globals.h | 232 +++ contrib/density/src/structure/header.c | 57 + contrib/density/src/structure/header.h | 58 + contrib/lizard/.gitattributes | 21 + contrib/lizard/.gitignore | 37 + contrib/lizard/.travis.yml | 268 +++ contrib/lizard/LICENSE | 15 + contrib/lizard/NEWS | 41 + contrib/lizard/README.md | 95 + contrib/lizard/appveyor.yml | 143 ++ contrib/lizard/contrib/djgpp/LICENSE | 24 + contrib/lizard/contrib/djgpp/README.MD | 21 + contrib/lizard/doc/lizard_Block_format.md | 181 ++ contrib/lizard/doc/lizard_Frame_format.md | 312 ++++ contrib/lizard/examples/.gitignore | 8 + .../lizard/examples/HCStreaming_ringBuffer.c | 241 +++ contrib/lizard/examples/README.md | 8 + .../examples/blockStreaming_doubleBuffer.c | 203 +++ .../examples/blockStreaming_doubleBuffer.md | 100 ++ .../examples/blockStreaming_lineByLine.c | 210 +++ .../examples/blockStreaming_lineByLine.md | 122 ++ .../examples/blockStreaming_ringBuffer.c | 202 +++ contrib/lizard/examples/compress_functions.c | 303 ++++ contrib/lizard/examples/frameCompress.c | 169 ++ contrib/lizard/examples/printVersion.c | 13 + contrib/lizard/examples/simple_buffer.c | 91 + .../lizard/examples/streaming_api_basics.md | 87 + contrib/lizard/lib/.gitignore | 3 + contrib/lizard/lib/LICENSE | 25 + contrib/lizard/lib/README.md | 76 + contrib/lizard/lib/dll/liblizard.def | 19 + contrib/lizard/lib/entropy/README.md | 38 + contrib/lizard/lib/entropy/bitstream.h | 414 +++++ contrib/lizard/lib/entropy/entropy_common.c | 231 +++ contrib/lizard/lib/entropy/error_private.h | 115 ++ contrib/lizard/lib/entropy/error_public.h | 64 + contrib/lizard/lib/entropy/fse.h | 694 ++++++++ contrib/lizard/lib/entropy/fse_compress.c | 848 +++++++++ contrib/lizard/lib/entropy/fse_decompress.c | 329 ++++ contrib/lizard/lib/entropy/huf.h | 250 +++ contrib/lizard/lib/entropy/huf_compress.c | 612 +++++++ contrib/lizard/lib/entropy/huf_decompress.c | 885 ++++++++++ contrib/lizard/lib/entropy/mem.h | 372 ++++ contrib/lizard/lib/liblizard.pc.in | 15 + contrib/lizard/lib/lizard_common.h | 504 ++++++ contrib/lizard/lib/lizard_compress.c | 630 +++++++ contrib/lizard/lib/lizard_compress.h | 208 +++ contrib/lizard/lib/lizard_compress_liz.h | 301 ++++ contrib/lizard/lib/lizard_compress_lz4.h | 162 ++ contrib/lizard/lib/lizard_decompress.c | 372 ++++ contrib/lizard/lib/lizard_decompress.h | 152 ++ contrib/lizard/lib/lizard_decompress_liz.h | 220 +++ contrib/lizard/lib/lizard_decompress_lz4.h | 163 ++ contrib/lizard/lib/lizard_frame.c | 1362 ++++++++++++++ contrib/lizard/lib/lizard_frame.h | 303 ++++ contrib/lizard/lib/lizard_frame_static.h | 81 + contrib/lizard/lib/lizard_parser_fast.h | 196 +++ contrib/lizard/lib/lizard_parser_fastbig.h | 175 ++ contrib/lizard/lib/lizard_parser_fastsmall.h | 189 ++ contrib/lizard/lib/lizard_parser_hashchain.h | 369 ++++ .../lizard/lib/lizard_parser_lowestprice.h | 376 ++++ contrib/lizard/lib/lizard_parser_nochain.h | 318 ++++ contrib/lizard/lib/lizard_parser_optimal.h | 679 +++++++ contrib/lizard/lib/lizard_parser_pricefast.h | 250 +++ contrib/lizard/lib/xxhash/xxhash.c | 888 ++++++++++ contrib/lizard/lib/xxhash/xxhash.h | 293 +++ contrib/lizard/programs/.gitignore | 15 + contrib/lizard/programs/COPYING | 339 ++++ contrib/lizard/programs/README.md | 73 + contrib/lizard/programs/bench.c | 502 ++++++ contrib/lizard/programs/bench.h | 37 + contrib/lizard/programs/datagen.c | 188 ++ contrib/lizard/programs/datagen.h | 39 + contrib/lizard/programs/lizard.1 | 229 +++ contrib/lizard/programs/lizardcli.c | 581 ++++++ contrib/lizard/programs/lizardio.c | 896 ++++++++++ contrib/lizard/programs/lizardio.h | 101 ++ contrib/lizard/programs/platform.h | 145 ++ contrib/lizard/programs/util.h | 497 ++++++ contrib/lizard/tests/.gitignore | 13 + contrib/lizard/tests/COPYING | 339 ++++ contrib/lizard/tests/README.md | 15 + contrib/lizard/tests/datagencli.c | 192 ++ contrib/lizard/tests/frametest.c | 866 +++++++++ contrib/lizard/tests/fullbench.c | 732 ++++++++ contrib/lizard/tests/fuzzer.c | 1086 ++++++++++++ contrib/lizard/visual/.gitignore | 11 + contrib/lizard/visual/README.md | 53 + .../visual/VS2010/datagen/datagen.vcxproj | 165 ++ .../visual/VS2010/frametest/frametest.vcxproj | 177 ++ .../visual/VS2010/fullbench/fullbench.vcxproj | 175 ++ .../visual/VS2010/fuzzer/fuzzer.vcxproj | 174 ++ .../VS2010/liblizard-dll/liblizard-dll.rc | 51 + .../liblizard-dll/liblizard-dll.vcxproj | 179 ++ .../visual/VS2010/liblizard/liblizard.vcxproj | 176 ++ contrib/lizard/visual/VS2010/lizard.sln | 86 + contrib/lizard/visual/VS2010/lizard/lizard.rc | 51 + .../visual/VS2010/lizard/lizard.vcxproj | 190 ++ contrib/lzsse/.gitignore | 4 + contrib/lzsse/.travis.yml | 185 ++ contrib/lzsse/LICENSE | 23 + contrib/lzsse/README.md | 15 + contrib/lzsse/example/main.cpp | 389 ++++ contrib/lzsse/lzsse2/lzsse2.cpp | 1080 ++++++++++++ contrib/lzsse/lzsse2/lzsse2.h | 90 + contrib/lzsse/lzsse2/lzsse2_platform.h | 73 + contrib/lzsse/lzsse4/lzsse4.cpp | 1499 ++++++++++++++++ contrib/lzsse/lzsse4/lzsse4.h | 117 ++ contrib/lzsse/lzsse4/lzsse4_platform.h | 73 + contrib/lzsse/lzsse8/lzsse8.cpp | 1568 +++++++++++++++++ contrib/lzsse/lzsse8/lzsse8.h | 117 ++ contrib/lzsse/lzsse8/lzsse8_platform.h | 73 + contrib/lzsse/premake4.exe | Bin 0 -> 518144 bytes contrib/lzsse/premake4.lua | 26 + 156 files changed, 35085 insertions(+) create mode 100644 contrib/density/.gitignore create mode 100644 contrib/density/.gitmodules create mode 100644 contrib/density/.travis.yml create mode 100644 contrib/density/CHANGELOG.md create mode 100644 contrib/density/LICENSE.md create mode 100644 contrib/density/README.md create mode 100644 contrib/density/appveyor.yml create mode 100644 contrib/density/benchmark/src/benchmark.c create mode 100644 contrib/density/benchmark/src/benchmark.h create mode 100644 contrib/density/msvc/Density.sln create mode 100644 contrib/density/msvc/benchmark.vcxproj create mode 100644 contrib/density/msvc/benchmark.vcxproj.filters create mode 100644 contrib/density/msvc/density.vcxproj create mode 100644 contrib/density/msvc/density.vcxproj.filters create mode 100644 contrib/density/src/algorithms/algorithms.c create mode 100644 contrib/density/src/algorithms/algorithms.h create mode 100644 contrib/density/src/algorithms/chameleon/chameleon.h create mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_decode.c create mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_decode.h create mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_encode.c create mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_encode.h create mode 100644 contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h create mode 100644 contrib/density/src/algorithms/cheetah/cheetah.h create mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_decode.c create mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_decode.h create mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_encode.c create mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_encode.h create mode 100644 contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h create mode 100644 contrib/density/src/algorithms/dictionaries.c create mode 100644 contrib/density/src/algorithms/dictionaries.h create mode 100644 contrib/density/src/algorithms/lion/core/lion_decode.c create mode 100644 contrib/density/src/algorithms/lion/core/lion_decode.h create mode 100644 contrib/density/src/algorithms/lion/core/lion_encode.c create mode 100644 contrib/density/src/algorithms/lion/core/lion_encode.h create mode 100644 contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h create mode 100644 contrib/density/src/algorithms/lion/forms/lion_form_model.c create mode 100644 contrib/density/src/algorithms/lion/forms/lion_form_model.h create mode 100644 contrib/density/src/algorithms/lion/lion.h create mode 100644 contrib/density/src/buffers/buffer.c create mode 100644 contrib/density/src/buffers/buffer.h create mode 100644 contrib/density/src/density_api.h create mode 100644 contrib/density/src/globals.c create mode 100644 contrib/density/src/globals.h create mode 100644 contrib/density/src/structure/header.c create mode 100644 contrib/density/src/structure/header.h create mode 100644 contrib/lizard/.gitattributes create mode 100644 contrib/lizard/.gitignore create mode 100644 contrib/lizard/.travis.yml create mode 100644 contrib/lizard/LICENSE create mode 100644 contrib/lizard/NEWS create mode 100644 contrib/lizard/README.md create mode 100644 contrib/lizard/appveyor.yml create mode 100644 contrib/lizard/contrib/djgpp/LICENSE create mode 100644 contrib/lizard/contrib/djgpp/README.MD create mode 100644 contrib/lizard/doc/lizard_Block_format.md create mode 100644 contrib/lizard/doc/lizard_Frame_format.md create mode 100644 contrib/lizard/examples/.gitignore create mode 100644 contrib/lizard/examples/HCStreaming_ringBuffer.c create mode 100644 contrib/lizard/examples/README.md create mode 100644 contrib/lizard/examples/blockStreaming_doubleBuffer.c create mode 100644 contrib/lizard/examples/blockStreaming_doubleBuffer.md create mode 100644 contrib/lizard/examples/blockStreaming_lineByLine.c create mode 100644 contrib/lizard/examples/blockStreaming_lineByLine.md create mode 100644 contrib/lizard/examples/blockStreaming_ringBuffer.c create mode 100644 contrib/lizard/examples/compress_functions.c create mode 100644 contrib/lizard/examples/frameCompress.c create mode 100644 contrib/lizard/examples/printVersion.c create mode 100644 contrib/lizard/examples/simple_buffer.c create mode 100644 contrib/lizard/examples/streaming_api_basics.md create mode 100644 contrib/lizard/lib/.gitignore create mode 100644 contrib/lizard/lib/LICENSE create mode 100644 contrib/lizard/lib/README.md create mode 100644 contrib/lizard/lib/dll/liblizard.def create mode 100644 contrib/lizard/lib/entropy/README.md create mode 100644 contrib/lizard/lib/entropy/bitstream.h create mode 100644 contrib/lizard/lib/entropy/entropy_common.c create mode 100644 contrib/lizard/lib/entropy/error_private.h create mode 100644 contrib/lizard/lib/entropy/error_public.h create mode 100644 contrib/lizard/lib/entropy/fse.h create mode 100644 contrib/lizard/lib/entropy/fse_compress.c create mode 100644 contrib/lizard/lib/entropy/fse_decompress.c create mode 100644 contrib/lizard/lib/entropy/huf.h create mode 100644 contrib/lizard/lib/entropy/huf_compress.c create mode 100644 contrib/lizard/lib/entropy/huf_decompress.c create mode 100644 contrib/lizard/lib/entropy/mem.h create mode 100644 contrib/lizard/lib/liblizard.pc.in create mode 100644 contrib/lizard/lib/lizard_common.h create mode 100644 contrib/lizard/lib/lizard_compress.c create mode 100644 contrib/lizard/lib/lizard_compress.h create mode 100644 contrib/lizard/lib/lizard_compress_liz.h create mode 100644 contrib/lizard/lib/lizard_compress_lz4.h create mode 100644 contrib/lizard/lib/lizard_decompress.c create mode 100644 contrib/lizard/lib/lizard_decompress.h create mode 100644 contrib/lizard/lib/lizard_decompress_liz.h create mode 100644 contrib/lizard/lib/lizard_decompress_lz4.h create mode 100644 contrib/lizard/lib/lizard_frame.c create mode 100644 contrib/lizard/lib/lizard_frame.h create mode 100644 contrib/lizard/lib/lizard_frame_static.h create mode 100644 contrib/lizard/lib/lizard_parser_fast.h create mode 100644 contrib/lizard/lib/lizard_parser_fastbig.h create mode 100644 contrib/lizard/lib/lizard_parser_fastsmall.h create mode 100644 contrib/lizard/lib/lizard_parser_hashchain.h create mode 100644 contrib/lizard/lib/lizard_parser_lowestprice.h create mode 100644 contrib/lizard/lib/lizard_parser_nochain.h create mode 100644 contrib/lizard/lib/lizard_parser_optimal.h create mode 100644 contrib/lizard/lib/lizard_parser_pricefast.h create mode 100644 contrib/lizard/lib/xxhash/xxhash.c create mode 100644 contrib/lizard/lib/xxhash/xxhash.h create mode 100644 contrib/lizard/programs/.gitignore create mode 100644 contrib/lizard/programs/COPYING create mode 100644 contrib/lizard/programs/README.md create mode 100644 contrib/lizard/programs/bench.c create mode 100644 contrib/lizard/programs/bench.h create mode 100644 contrib/lizard/programs/datagen.c create mode 100644 contrib/lizard/programs/datagen.h create mode 100644 contrib/lizard/programs/lizard.1 create mode 100644 contrib/lizard/programs/lizardcli.c create mode 100644 contrib/lizard/programs/lizardio.c create mode 100644 contrib/lizard/programs/lizardio.h create mode 100644 contrib/lizard/programs/platform.h create mode 100644 contrib/lizard/programs/util.h create mode 100644 contrib/lizard/tests/.gitignore create mode 100644 contrib/lizard/tests/COPYING create mode 100644 contrib/lizard/tests/README.md create mode 100644 contrib/lizard/tests/datagencli.c create mode 100644 contrib/lizard/tests/frametest.c create mode 100644 contrib/lizard/tests/fullbench.c create mode 100644 contrib/lizard/tests/fuzzer.c create mode 100644 contrib/lizard/visual/.gitignore create mode 100644 contrib/lizard/visual/README.md create mode 100644 contrib/lizard/visual/VS2010/datagen/datagen.vcxproj create mode 100644 contrib/lizard/visual/VS2010/frametest/frametest.vcxproj create mode 100644 contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj create mode 100644 contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj create mode 100644 contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc create mode 100644 contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj create mode 100644 contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj create mode 100644 contrib/lizard/visual/VS2010/lizard.sln create mode 100644 contrib/lizard/visual/VS2010/lizard/lizard.rc create mode 100644 contrib/lizard/visual/VS2010/lizard/lizard.vcxproj create mode 100644 contrib/lzsse/.gitignore create mode 100644 contrib/lzsse/.travis.yml create mode 100644 contrib/lzsse/LICENSE create mode 100644 contrib/lzsse/README.md create mode 100644 contrib/lzsse/example/main.cpp create mode 100644 contrib/lzsse/lzsse2/lzsse2.cpp create mode 100644 contrib/lzsse/lzsse2/lzsse2.h create mode 100644 contrib/lzsse/lzsse2/lzsse2_platform.h create mode 100644 contrib/lzsse/lzsse4/lzsse4.cpp create mode 100644 contrib/lzsse/lzsse4/lzsse4.h create mode 100644 contrib/lzsse/lzsse4/lzsse4_platform.h create mode 100644 contrib/lzsse/lzsse8/lzsse8.cpp create mode 100644 contrib/lzsse/lzsse8/lzsse8.h create mode 100644 contrib/lzsse/lzsse8/lzsse8_platform.h create mode 100644 contrib/lzsse/premake4.exe create mode 100644 contrib/lzsse/premake4.lua diff --git a/contrib/density/.gitignore b/contrib/density/.gitignore new file mode 100644 index 00000000000..5192545dd3f --- /dev/null +++ b/contrib/density/.gitignore @@ -0,0 +1,2 @@ +build/ +benchmark/build diff --git a/contrib/density/.gitmodules b/contrib/density/.gitmodules new file mode 100644 index 00000000000..91eee619841 --- /dev/null +++ b/contrib/density/.gitmodules @@ -0,0 +1,6 @@ +[submodule "benchmark/libs/cputime"] + path = benchmark/libs/cputime + url = https://github.com/k0dai/cputime.git +[submodule "benchmark/libs/spookyhash"] + path = benchmark/libs/spookyhash + url = https://github.com/k0dai/spookyhash.git diff --git a/contrib/density/.travis.yml b/contrib/density/.travis.yml new file mode 100644 index 00000000000..62e7655a4cb --- /dev/null +++ b/contrib/density/.travis.yml @@ -0,0 +1,272 @@ +language: c +matrix: + fast_finish: true + include: + - os: osx + env: + - C_COMPILER=clang + + - os: osx + env: + - C_COMPILER=gcc + + - dist: trusty + sudo: required + addons: + apt: + packages: + - qemu-system-arm + - qemu-user-static + - gcc-aarch64-linux-gnu + - g++-aarch64-linux-gnu + env: + - C_COMPILER=aarch64-linux-gnu-gcc MAKE_OPTIONS="AR=gcc-ar NATIVE=off" QEMU_INTERPRETER=qemu-aarch64-static PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/aarch64-linux-gnu/" + + - dist: trusty + sudo: required + addons: + apt: + packages: + - qemu-system-arm + - qemu-user-static + - gcc-arm-linux-gnueabi + - g++-arm-linux-gnueabi + env: + - C_COMPILER=arm-linux-gnueabi-gcc MAKE_OPTIONS="AR=gcc-ar NATIVE=off" QEMU_INTERPRETER=qemu-arm-static PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/arm-linux-gnueabi/" + + - dist: trusty + sudo: required + addons: + apt: + packages: + - qemu-system-ppc + - qemu-user-static + - gcc-powerpc-linux-gnu + - g++-powerpc-linux-gnu + env: + - C_COMPILER=powerpc-linux-gnu-gcc MAKE_OPTIONS="AR=gcc-ar ARCH=64" QEMU_INTERPRETER="qemu-ppc64-static" PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/powerpc-linux-gnu/" + + - dist: trusty + sudo: required + addons: + apt: + packages: + - qemu-system-ppc + - qemu-user-static + - gcc-powerpc-linux-gnu + - g++-powerpc-linux-gnu + env: + - C_COMPILER=powerpc-linux-gnu-gcc MAKE_OPTIONS="AR=gcc-ar NATIVE=off" QEMU_INTERPRETER="qemu-ppc-static" PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/powerpc-linux-gnu/" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-4.8 + env: + - C_COMPILER=gcc-4.8 MAKE_OPTIONS="AR=gcc-ar" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-4.8 + - gcc-4.8-multilib + env: + - C_COMPILER=gcc-4.8 MAKE_OPTIONS="AR=gcc-ar ARCH=32" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-4.9 + env: + - C_COMPILER=gcc-4.9 MAKE_OPTIONS="AR=gcc-ar" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-4.9 + - gcc-4.9-multilib + env: + - C_COMPILER=gcc-4.9 MAKE_OPTIONS="AR=gcc-ar ARCH=32" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-5 + env: + - C_COMPILER=gcc-5 MAKE_OPTIONS="AR=gcc-ar-5" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-5 + - gcc-5-multilib + env: + - C_COMPILER=gcc-5 MAKE_OPTIONS="AR=gcc-ar-5 ARCH=32" + +# - dist: trusty +# sudo: required +# addons: +# apt: +# sources: +# - ubuntu-toolchain-r-test +# packages: +# - gcc-6 +# env: +# - C_COMPILER=gcc-6 MAKE_OPTIONS="AR=gcc-ar-6" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-6 + - gcc-6-multilib + env: + - C_COMPILER=gcc-6 MAKE_OPTIONS="AR=gcc-ar-6 ARCH=32" + +# - dist: trusty +# sudo: required +# addons: +# apt: +# sources: +# - ubuntu-toolchain-r-test +# packages: +# - gcc-7 +# env: +# - C_COMPILER=gcc-7 MAKE_OPTIONS="AR=gcc-ar-7" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-7 + - gcc-7-multilib + env: + - C_COMPILER=gcc-7 MAKE_OPTIONS="AR=gcc-ar-7 ARCH=32" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-trusty-3.5 + packages: + - clang-3.5 + - llvm-3.5-dev + env: + - C_COMPILER=clang-3.5 MAKE_OPTIONS="AR=llvm-ar-3.5" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-trusty-3.6 + packages: + - clang-3.6 + - llvm-3.6-dev + env: + - C_COMPILER=clang-3.6 MAKE_OPTIONS="AR=llvm-ar-3.6" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + +# - dist: precise +# sudo: required +# addons: +# apt: +# sources: +# - ubuntu-toolchain-r-test +# - llvm-toolchain-precise-3.7 +# packages: +# - clang-3.7 +# - llvm-3.7-dev +# env: +# - C_COMPILER=clang-3.7 MAKE_OPTIONS="AR=llvm-ar-3.7" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + +# - dist: precise +# sudo: required +# addons: +# apt: +# sources: +# - ubuntu-toolchain-r-test +# - llvm-toolchain-precise-3.8 +# packages: +# - clang-3.8 +# - llvm-3.8-dev +# env: +# - C_COMPILER=clang-3.8 MAKE_OPTIONS="AR=llvm-ar-3.8" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-trusty-3.9 + packages: + - clang-3.9 + - llvm-3.9-dev + env: + - C_COMPILER=clang-3.9 MAKE_OPTIONS="AR=llvm-ar-3.9" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + + - dist: trusty + sudo: required + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-trusty-4.0 + packages: + - clang-4.0 + - llvm-4.0-dev + env: + - C_COMPILER=clang-4.0 MAKE_OPTIONS="AR=llvm-ar-4.0" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + +# - dist: trusty +# sudo: required +# addons: +# apt: +# sources: +# - ubuntu-toolchain-r-test +# - llvm-toolchain-trusty-5.0 +# packages: +# - clang-5.0 +# - llvm-5.0-dev +# env: +# - C_COMPILER=clang-5.0 MAKE_OPTIONS="AR=llvm-ar-5.0" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" + +script: + - ${PRE_SCRIPT} + - $C_COMPILER -v + - make CC=$C_COMPILER $MAKE_OPTIONS + - file ./build/* + - $QEMU_INTERPRETER ./build/benchmark -h -f diff --git a/contrib/density/CHANGELOG.md b/contrib/density/CHANGELOG.md new file mode 100644 index 00000000000..6d5535bcc7c --- /dev/null +++ b/contrib/density/CHANGELOG.md @@ -0,0 +1,168 @@ +0.14.2 +------ +*February 12, 2018* +* Improved chameleon decode speed +* Added data hash checks and display option in benchmark +* Now using makefiles as build system +* Big endian support correctly implemented and tested +* Improved continuous integration tests + +0.14.1 +------ +*January 20, 2018* +* Added MSVC support +* Added continuous integration on travis and appveyor +* Premake script improvement +* Various codebase improvements + +0.14.0 +------ +*January 16, 2018* +* First stable version of DENSITY +* Complete project reorganization and API rewrite +* Many stability fixes and improvements +* Fast revert to conditional copy for incompressible input +* Custom dictionaries in API +* Improvements in compression ratio and speed + +0.12.5 beta +----------- +*June 20, 2015* + +* Added conditional main footer read/write +* Improved teleport staging buffer management +* Regression - a minimum buffer output size has to be ensured to avoid signature loss +* Modified the minimum lookahead and the resulting minimum buffer size in the API +* Lion : corrected a signature interception problem due to an increase in process unit size +* Lion : corrected chunk count conditions for new block / mode marker detection +* Lion : modified end of stream marker conditions +* Stability fixes and improvements + +0.12.4 beta +----------- +*May 25, 2015* + +* Removed remaining undefined behavior potential occurences +* Implemented parallelizable decompressible output block header reads/writes (disabled by default) + +0.12.3 beta +----------- +*May 20, 2015* + +* New lion algorithm, faster and more efficient +* Compiler specific optimizations +* Switched to premake 5 to benefit from link time optimizations +* Various fixes and improvements + +0.12.2 beta +----------- +*May 4, 2015* + +* Added an integrated in-memory benchmark +* Better Windows compatibility +* Fixed misaligned load/stores +* Switched to the premake build system +* Performance optimizations (pointers, branches, loops ...) +* Various fixes and improvements + +0.12.1 beta +----------- +*April 3, 2015* + +* Better unrolling readability and efficiency +* Improved read speed of dictionary/predictions entries +* Implemented case generators in cheetah to speed up decoding by using less branches +* Added signatures interception in lion to cancel the need for large output buffers +* Improved lion decode speed with specific form data access and use of ctz in form read +* Enabled decompression to exact-sized buffer for all algorithms +* Various fixes and improvements + +0.12.0 beta +----------- +*March 24, 2015* + +* Added new lion kernel +* Renamed kernel mandala to cheetah +* Kernel chameleon and cheetah improvements in encoding/decoding speeds +* Generic function macros to avoid code rewrite +* Improved memory teleport IO flexibility and speed, bytes issued by memory teleport can now be partially read +* Various fixes and improvements + +0.11.3 beta +----------- +*February 5, 2015* + +* Added integrity check system +* Corrected pointer usage and update on footer read/writes +* Now freeing kernel state memory only when compression mode is not copy +* Updated Makefiles +* Improved memory teleport +* Fixed sequencing problem after kernels request a new block + +0.11.2 beta +----------- +*February 3, 2015* + +* Added an algorithms overview in README +* Removed ssc references +* Now initializing last hash to zero on mandala kernel inits +* Reimplemented the buffer API +* Various corrections and improvements + +0.11.1 beta +----------- +*January 19, 2015* + +* Added a sharc benchmark in README +* Stateless memory teleport +* Improved event management and dispatching +* Improved compression/decompression finishes +* Improved streams API +* Various bug fixes, robustness improvements + +0.10.2 beta +----------- +*January 7, 2015* + +* Improved organization of compile-time switches and run-time options in the API +* Removed method density_stream_decompress_utilities_get_header from the API, header info is now returned in the density_stream_decompress_init function +* Corrected readme to reflect API changes + +0.10.1 beta +----------- +*January 5, 2015* + +* Re-added mandala kernel +* Corrected available bytes adjustment problem +* Added missing restrict keywords +* Cleaned unnecessary defines + +0.10.0 beta +----------- +*January 2, 2015* + +* Complete stream API redesign to greatly improve flexibility +* Only one supported algorithm for now : Chameleon + +0.9.12 beta +----------- +*December 2, 2013* + +* Mandala kernel addition, replacing dual pass chameleon +* Simplified, faster hash function +* Fixed memory freeing issue during main encoding/decoding finish +* Implemented no footer encode output type +* Namespace migration, kernel structure reorganization +* Corrected copy mode problem +* Implemented efficiency checks and mode reversions +* Corrected lack of main header parameters retrieval +* Fixed stream not being properly ended when mode reversion occurred +* Updated metadata computations + +0.9.11 beta +----------- +*November 2, 2013* + +* First beta release of DENSITY, including all the compression code from SHARC in a standalone, BSD licensed library +* Added copy mode (useful for enhancing data security via the density block checksums for example) +* Makefile produces static and dynamic libraries diff --git a/contrib/density/LICENSE.md b/contrib/density/LICENSE.md new file mode 100644 index 00000000000..c2b699c9b83 --- /dev/null +++ b/contrib/density/LICENSE.md @@ -0,0 +1,28 @@ +Copyright (c) 2013, Guillaume Voirin + +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/density/README.md b/contrib/density/README.md new file mode 100644 index 00000000000..6c660dea1f5 --- /dev/null +++ b/contrib/density/README.md @@ -0,0 +1,261 @@ +DENSITY +======== +Superfast compression library + +DENSITY is a free C99, open-source, BSD licensed compression library. + +It is focused on high-speed compression, at the best ratio possible. **All three** of DENSITY's algorithms are currently at the **pareto frontier** of compression speed vs ratio (cf. [here](https://github.com/inikep/lzbench/blob/master/lzbench18_sorted.md) for an independent benchmark). + +DENSITY features a simple API to enable quick integration in any project. + +Branch|Linux & MacOS|Windows +--- | --- | --- +master|[![Build Status](https://travis-ci.org/k0dai/density.svg?branch=master)](https://travis-ci.org/k0dai/density)|[![Build status](https://ci.appveyor.com/api/projects/status/rf7x3x829il72cii/branch/master?svg=true)](https://ci.appveyor.com/project/gpnuma/density/branch/master) +dev|[![Build Status](https://travis-ci.org/k0dai/density.svg?branch=dev)](https://travis-ci.org/k0dai/density)|[![Build status](https://ci.appveyor.com/api/projects/status/rf7x3x829il72cii/branch/dev?svg=true)](https://ci.appveyor.com/project/gpnuma/density/branch/dev) + +Why is it so fast ? +------------------- + +One of the biggest assets of DENSITY is that its work unit is **not a byte** like other libraries, but **a group of 4 bytes**. + +When other libraries consume one byte of data and then apply an algorithmic processing to it, DENSITY consumes 4 bytes and then applies its algorithmic processing. + +That's why DENSITY's algorithms were designed from scratch. They have to alleviate for 4-byte work units and still provide interesting compression ratios. + +**Speed pedigree traits** + +* 4-byte work units +* heavy use of registers as opposed to memory for processing +* avoidance of or use of minimal branching when possible +* use of low memory data structures to favor processor cache Lx accesses +* library wide inlining +* specific unrollings +* prefetching and branching hints +* restricted pointers to maximize compiler optimizations + +A "blowup protection" is provided, dramatically increasing the processing speed of incompressible input data. Also, the output, compressed data size will **never exceed** the original uncompressed data size by more than 1% in case of incompressible, reasonably-sized inputs. + +Benchmarks +---------- + +**Quick benchmark** + +DENSITY features an **integrated in-memory benchmark**. After building the project (see [build](#build)), a *benchmark* executable will be present in the build directory. If run without arguments, usage help will be displayed. + +File used : enwik8 (100 MB) + +Platform : MacBook Pro, MacOS 10.13.3, 2.3 GHz Intel Core i7, 8Gb 1600 MHz DDR, SSD, compiling with Clang/LLVM 9.0.0 + +Timing : using the *time* function, and taking the best *user* output after multiple runs. In the case of density, the in-memory integrated benchmark's best value (which uses the same usermode CPU timing) is used. + +Library|Algorithm|Compress|Decompress|Size|Ratio|Round trip +---|---|---|---|---|---|--- +**density** 0.14.2|Chameleon|0.092s (1085 MB/s)|0.059s (1684 MB/s)|61 524 084|61,52%|0.151s +lz4 r129|-1|0.468s (214 MB/s)|0.115s (870 MB/s)|57 285 990|57,29%|0.583s +lzo 2.08|-1|0.367s (272 MB/s)|0.309s (324 MB/s)|56 709 096|56,71%|0.676s +**density** 0.14.2|Cheetah|0.170s (587 MB/s)|0.126s (796 MB/s)|53 156 668|53,16%|0.296s +**density** 0.14.2|Lion|0.303s (330 MB/s)|0.288s (347 MB/s)|47 817 692|47,82%|0.591s +lz4 r129|-3|1.685s (59 MB/s)|0.118s (847 MB/s)|44 539 940|44,54%|1.803s +lzo 2.08|-7|9.562s (10 MB/s)|0.319s (313 MB/s)|41 720 721|41,72%|9.881s + +**Other benchmarks** + +Here are a few other benchmarks featuring DENSITY (non exhaustive list) : + +* [**squash**](https://github.com/quixdb/squash) is an abstraction layer for compression algorithms, and has an extremely exhaustive set of benchmark results, including density's, [available here](https://quixdb.github.io/squash-benchmark/?dataset=dickens&machine=s-desktop). + +* [**lzbench**](https://github.com/inikep/lzbench) is an in-memory benchmark of open-source LZ77/LZSS/LZMA compressors. + +* [**fsbench**](https://github.com/gpnuma/fsbench-density) is a command line utility that enables real-time testing of compression algorithms, but also hashes and much more. A fork with density releases is [available here](https://github.com/gpnuma/fsbench-density) for easy access. +The original author's repository [can be found here](https://chiselapp.com/user/Justin_be_my_guide/repository/fsbench/). + +Build +----- +DENSITY can be built on a number of platforms, via the provided makefiles. + +It was developed and optimized against Clang/LLVM which makes it the preferred compiler, but GCC and MSVC are also supported. Please use the latest compiler versions for best performance. + +**MacOS** + +On MacOS, Clang/LLVM is the default compiler, which makes things simpler. + +1) Get the source code : + +``` + git clone https://github.com/k0dai/density.git + cd density +``` + +2) Build and test : + +``` + make + build/benchmark -f +``` + +Alternatively, thanks to the [Homebrew project](https://brew.sh), DENSITY can also be installed with a single command on MacOS: + +``` + brew install density +``` + +**Linux** + +On Linux, Clang/LLVM is not always available by default, but can be easily added thanks to the provided package managers. +The following example assumes a Debian or Ubuntu distribution with *apt-get*. + +1) From the command line, install Clang/LLVM (*optional*, GCC is also supported if Clang/LLVM can't be used) and other prerequisites. + +``` + sudo apt-get install clang git +``` + +2) Get the source code : + +``` + git clone https://github.com/k0dai/density.git + cd density +``` + +3) Build and test : + +``` + make +``` +or +``` + make CC=gcc-... AR=gcc-ar-... +``` +or +``` + make CC=clang-... AR=llvm-ar-... +``` +to choose alternative compilers. For a quick test of resulting binaries, run +``` + build/benchmark -f +``` + +**Windows** + +Please install [git for Windows](https://git-scm.com/download/win) to begin with. + +On Windows, density can be built in different ways. +The **first method** is to use mingw's gcc compiler; for that it is necessary to download and install [mingw-w64](https://sourceforge.net/projects/mingw-w64/). + +1) Once mingw-w64 is installed, get the source : + +``` + git clone https://github.com/k0dai/density.git + cd density +``` + +2) Build and test : + +``` + mingw32-make.exe + build/benchmark.exe -f +``` + +As an alternative, [MSYS2](http://www.msys2.org/) also offers a linux-like environment for Windows. + +The **second method** is to download and install Microsoft's [Visual Studio IDE community edition](https://www.visualstudio.com/thank-you-downloading-visual-studio/?sku=Community). It comes with Microsoft's own compilers and is free. + +1) Once Visual Studio is installed, open a [developer command prompt](https://docs.microsoft.com/en-us/dotnet/framework/tools/developer-command-prompt-for-vs) and type : + +``` + git clone https://github.com/k0dai/density.git + cd density\msvc +``` + +2) Build and test : + +``` + msbuild Density.sln + bin\Release\benchmark.exe -f +``` + +An extra **recommended step** would be to install *Clang/LLVM* for Windows. It is downloadable from [this link](http://releases.llvm.org/5.0.1/LLVM-5.0.1-win64.exe). Once installed, open the Visual Studio IDE by double-clicking on *Density.sln*, then right-click on project names and change the platform toolsets to *LLVM*. Rebuild the solution to generate binaries with Clang/LLVM. + +Output format +------------- +DENSITY outputs compressed data in a simple format, which enables file storage and optional parallelization for both compression and decompression. + +A very short header holding vital informations (like DENSITY version and algorithm used) precedes the binary compressed data. + +APIs +---- +DENSITY features a straightforward *API*, simple yet powerful enough to keep users' creativity unleashed. + +For advanced developers, it allows use of custom dictionaries and exportation of generated dictionaries after a compression session. Although using the default, blank dictionary is perfectly fine in most cases, setting up your own, tailored dictionaries could somewhat improve compression ratio especially for low sized input datum. + +Please see the [*quick start*](#quick-start-a-simple-example-using-the-api) at the bottom of this page. + +About the algorithms +-------------------- + +**Chameleon** ( *DENSITY_ALGORITHM_CHAMELEON* ) + +Chameleon is a dictionary lookup based compression algorithm. It is designed for absolute speed and usually reaches a 60% compression ratio on compressible data. +Decompression is just as fast. This algorithm is a great choice when main concern is speed. + +**Cheetah** ( *DENSITY_ALGORITHM_CHEETAH* ) + +Cheetah was developed with inputs from [Piotr Tarsa](https://github.com/tarsa). +It is derived from chameleon and uses swapped double dictionary lookups and predictions. It can be extremely good with highly compressible data (ratio reaching 10% or less). +On typical compressible data compression ratio is about 50% or less. It is still extremely fast for both compression and decompression and is a great, efficient all-rounder algorithm. + +**Lion** ( *DENSITY_ALGORITHM_LION* ) + +Lion is a multiform compression algorithm derived from cheetah. It goes further in the areas of dynamic adaptation and fine-grained analysis. +It uses multiple swapped dictionary lookups and predictions, and forms rank entropy coding. +Lion provides the best compression ratio of all three algorithms under any circumstance, and is still very fast. + +Quick start (a simple example using the API) +-------------------------------------------- +Using DENSITY in your application couldn't be any simpler. + +First you need to include this file in your project : + +* density_api.h + +When this is done you can start using the **DENSITY API** : + +```C + #include + #include "density_api.h" + + char* text = "This is a simple example on how to use the simple Density API. This is a simple example on how to use the simple Density API."; + uint64_t text_length = (uint64_t)strlen(text); + + // Determine safe buffer sizes + uint_fast64_t compress_safe_size = density_compress_safe_size(text_length); + uint_fast64_t decompress_safe_size = density_decompress_safe_size(text_length); + + // Allocate required memory + uint8_t *outCompressed = malloc(compress_safe_size * sizeof(char)); + uint8_t *outDecompressed = malloc(decompress_safe_size * sizeof(char)); + density_processing_result result; + + // Compress + result = density_compress(text, text_length, outCompressed, compress_safe_size, DENSITY_COMPRESSION_MODE_CHAMELEON_ALGORITHM); + if(!result.state) + printf("Compressed %llu bytes to %llu bytes\n", result.bytesRead, result.bytesWritten); + + // Decompress + result = density_decompress(outCompressed, result.bytesWritten, outDecompressed, decompress_safe_size); + if(!result.state) + printf("Decompressed %llu bytes to %llu bytes\n", result.bytesRead, result.bytesWritten); + + // Free memory_allocated + free(outCompressed); + free(outDecompressed); +``` + +And that's it ! We've done a compression/decompression round trip with a few lines ! + +Related projects +---------------- + +* **SHARC** (archiver using density algorithms) [https://github.com/gpnuma/sharc](https://github.com/gpnuma/sharc) +* **fsbench-density** (in-memory transformations benchmark) [https://github.com/gpnuma/fsbench-density](https://github.com/gpnuma/fsbench-density) +* **densityxx** (c++ port of density) [https://github.com/charlesw1234/densityxx](https://github.com/charlesw1234/densityxx) diff --git a/contrib/density/appveyor.yml b/contrib/density/appveyor.yml new file mode 100644 index 00000000000..0aabedebc8b --- /dev/null +++ b/contrib/density/appveyor.yml @@ -0,0 +1,37 @@ +version: 0.14.2.{build} +image: Visual Studio 2015 +init: +- cmd: +environment: + matrix: + - EXTRA_PATH: C:\mingw-w64\i686-5.3.0-posix-dwarf-rt_v4-rev0\mingw32\bin\ + LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar + LAUNCH_PATH: build\ + - EXTRA_PATH: C:\mingw-w64\i686-6.3.0-posix-dwarf-rt_v5-rev1\mingw32\bin\ + LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar + LAUNCH_PATH: build\ + - EXTRA_PATH: C:\mingw-w64\x86_64-6.3.0-posix-seh-rt_v5-rev1\mingw64\bin\ + LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar + LAUNCH_PATH: build\ + - EXTRA_PATH: C:\mingw-w64\x86_64-7.2.0-posix-seh-rt_v5-rev1\mingw64\bin\ + LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar + LAUNCH_PATH: build\ + - EXTRA_PATH: '%programfiles(x86)%\Microsoft Visual Studio\2017\Community\MSBuild\15.0\Bin\amd64\' + LAUNCH_BUILD: cd msvc && msbuild.exe Density.sln + EXTRA_GIT: git submodule update --init --recursive + LAUNCH_PATH: bin\Release\ +install: +- cmd: +build_script: +- cmd: >- + %EXTRA_GIT% + + set PATH=%EXTRA_PATH%;%PATH% + + %LAUNCH_BUILD% + + file %LAUNCH_PATH:\=/%* + + %LAUNCH_PATH%benchmark.exe -h -f +test_script: +- cmd: diff --git a/contrib/density/benchmark/src/benchmark.c b/contrib/density/benchmark/src/benchmark.c new file mode 100644 index 00000000000..9781ef361c6 --- /dev/null +++ b/contrib/density/benchmark/src/benchmark.c @@ -0,0 +1,345 @@ +/* + * Density benchmark + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program. If not, see . + * + * 5/04/15 19:30 + */ + +#include "benchmark.h" + +void density_benchmark_version() { + printf("\nSingle threaded "); + DENSITY_BENCHMARK_BOLD(printf("in-memory benchmark")); + printf(" powered by "); + DENSITY_BENCHMARK_BOLD(printf("Density %i.%i.%i\n", density_version_major(), density_version_minor(), density_version_revision())); + printf("Copyright (C) 2015 Guillaume Voirin\n"); + printf("Built for %s (%s endian system, %u bits) using " DENSITY_BENCHMARK_COMPILER ", %s %s\n", DENSITY_BENCHMARK_PLATFORM_STRING, DENSITY_BENCHMARK_ENDIAN_STRING, (unsigned int) (8 * sizeof(void *)), DENSITY_BENCHMARK_COMPILER_VERSION, __DATE__, __TIME__); +} + +void density_benchmark_client_usage() { + printf("\n"); + DENSITY_BENCHMARK_BOLD(printf("Usage :\n")); + printf(" benchmark [OPTIONS ?]... [FILE ?]\n\n"); + DENSITY_BENCHMARK_BOLD(printf("Available options :\n")); + printf(" -[LEVEL] Test file using only the specified compression LEVEL\n"); + printf(" If unspecified, all algorithms are tested (default).\n"); + printf(" LEVEL can have the following values (as values become higher,\n"); + printf(" compression ratio increases and speed diminishes) :\n"); + printf(" 0 = Copy (no compression)\n"); + printf(" 1 = Chameleon algorithm\n"); + printf(" 2 = Cheetah algorithm\n"); + printf(" 3 = Lion algorithm\n"); + printf(" -c Compress only\n"); + printf(" -f Activate fuzzer mode (pseudorandom generated data)\n"); + printf(" -h Print data hashing informations\n\n"); + exit(EXIT_SUCCESS); +} + +void density_benchmark_format_decimal(uint64_t number) { + if (number < 1000) { + printf("%"PRIu64, number); + return; + } + density_benchmark_format_decimal(number / 1000); + printf(",%03"PRIu64, number % 1000); +} + +const char *density_benchmark_convert_state_to_text(DENSITY_STATE state) { + switch (state) { + case DENSITY_STATE_ERROR_DURING_PROCESSING: + return "Error during processing"; + case DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL: + return "Input buffer is too small"; + case DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL: + return "Output buffer is too small"; + case DENSITY_STATE_ERROR_INVALID_CONTEXT: + return "Invalid context"; + case DENSITY_STATE_ERROR_INVALID_ALGORITHM: + return "Invalid algorithm"; + default: + return "Unknown error"; + } +} + +int main(int argc, char *argv[]) { + density_benchmark_version(); + DENSITY_ALGORITHM start_mode = DENSITY_ALGORITHM_CHAMELEON; + DENSITY_ALGORITHM end_mode = DENSITY_ALGORITHM_LION; + bool compression_only = false; + bool fuzzer = false; + bool hash_info = false; + char *file_path = NULL; + + if (argc <= 1) + density_benchmark_client_usage(); + for (int count = 1; count < argc; count++) { + if (argv[count][0] == '-') { + switch (argv[count][1]) { + case '1': + start_mode = DENSITY_ALGORITHM_CHAMELEON; + end_mode = DENSITY_ALGORITHM_CHAMELEON; + break; + case '2': + start_mode = DENSITY_ALGORITHM_CHEETAH; + end_mode = DENSITY_ALGORITHM_CHEETAH; + break; + case '3': + start_mode = DENSITY_ALGORITHM_LION; + end_mode = DENSITY_ALGORITHM_LION; + break; + case 'c': + compression_only = true; + break; + case 'f': + fuzzer = true; + break; + case 'h': + hash_info = true; + break; + default: + density_benchmark_client_usage(); + } + } else + file_path = argv[argc - 1]; + } + + uint8_t *in; + uint8_t *out; + uint_fast64_t uncompressed_size; + uint_fast64_t memory_allocated; + if (fuzzer) { + srand((unsigned int) (time(NULL) * 14521937821257379531llu)); + uncompressed_size = (uint_fast64_t) (((uint64_t) (rand() * 100000000llu)) / RAND_MAX); + memory_allocated = density_compress_safe_size(uncompressed_size); + in = malloc(memory_allocated * sizeof(uint8_t)); + uint8_t value = (uint8_t) rand(); + for (unsigned int count = 0; count < uncompressed_size; count++) { + if (!(rand() & 0xf)) + value += (uint8_t)rand(); + in[count] = value; + } + out = malloc(memory_allocated * sizeof(uint8_t)); + } else { + // Open file and get infos + FILE *file = fopen(file_path, "rb"); + if (file == NULL) { + DENSITY_BENCHMARK_ERROR(printf("Error opening file %s.", file_path), false); + } + struct stat file_attributes; + stat(file_path, &file_attributes); + + // Allocate memory and copy file to memory + uncompressed_size = (uint_fast64_t) file_attributes.st_size; + memory_allocated = density_compress_safe_size(uncompressed_size); + in = malloc(memory_allocated * sizeof(uint8_t)); + size_t read = fread(in, sizeof(uint8_t), uncompressed_size, file); + if(uncompressed_size != read) { + DENSITY_BENCHMARK_ERROR(printf("Error reading file %s.", file_path), false); + } + fclose(file); + out = malloc(memory_allocated * sizeof(uint8_t)); + } + + printf("Allocated "); + density_benchmark_format_decimal(2 * memory_allocated); + printf(" bytes of in-memory work space\n"); + + uint64_t original_hash_1 = DENSITY_BENCHMARK_HASH_SEED_1; + uint64_t original_hash_2 = DENSITY_BENCHMARK_HASH_SEED_2; + spookyhash_128(in, uncompressed_size, &original_hash_1, &original_hash_2); + + printf("\n"); + for (DENSITY_ALGORITHM compression_mode = start_mode; compression_mode <= end_mode; compression_mode++) { + // Print algorithm info + switch (compression_mode) { + case DENSITY_ALGORITHM_CHAMELEON: + DENSITY_BENCHMARK_BLUE(DENSITY_BENCHMARK_BOLD(printf("Chameleon algorithm"))); + DENSITY_BENCHMARK_UNDERLINE(19); + break; + case DENSITY_ALGORITHM_CHEETAH: + DENSITY_BENCHMARK_BLUE(DENSITY_BENCHMARK_BOLD(printf("Cheetah algorithm"))); + DENSITY_BENCHMARK_UNDERLINE(17); + break; + case DENSITY_ALGORITHM_LION: + DENSITY_BENCHMARK_BLUE(DENSITY_BENCHMARK_BOLD(printf("Lion algorithm"))); + DENSITY_BENCHMARK_UNDERLINE(14); + break; + } + fflush(stdout); + + // Pre-heat + printf("\nUsing "); + if (fuzzer) { + DENSITY_BENCHMARK_BOLD(printf("generated data")); + } else { + printf("file "); + DENSITY_BENCHMARK_BOLD(printf("%s", file_path)); + } + printf(" copied in memory\n"); + if(hash_info) { + printf("Uncompressed data hash is "); + DENSITY_BENCHMARK_BOLD(printf("0x%" PRIx64 "%" PRIx64, original_hash_1, original_hash_2)); + printf("\n"); + } + + density_processing_result result = density_compress(in, uncompressed_size, out, memory_allocated, compression_mode); + if (result.state) { + DENSITY_BENCHMARK_ERROR(printf("During compress API returned error %i (%s).", result.state, density_benchmark_convert_state_to_text(result.state)), true); + } + const uint_fast64_t compressed_size = result.bytesWritten; + + uint64_t hash_1 = DENSITY_BENCHMARK_HASH_SEED_1; + uint64_t hash_2 = DENSITY_BENCHMARK_HASH_SEED_2; + if(hash_info) { + spookyhash_128(out, compressed_size, &hash_1, &hash_2); + printf("Compressed data hash is "); + DENSITY_BENCHMARK_BOLD(printf("0x%" PRIx64 "%" PRIx64, hash_1, hash_2)); + printf("\n"); + } + + if (!compression_only) { + memset(in, 0, memory_allocated); + result = density_decompress(out, compressed_size, in, memory_allocated); + if (result.state) { + DENSITY_BENCHMARK_ERROR(printf("During decompress API returned error %i (%s).", result.state, density_benchmark_convert_state_to_text(result.state)), true); + } + if (result.bytesWritten != uncompressed_size) { + DENSITY_BENCHMARK_ERROR(printf("Round-trip size differs from original size ("); + density_benchmark_format_decimal(result.bytesWritten); + printf(" bytes against "); + density_benchmark_format_decimal(uncompressed_size); + printf(" bytes).");, true); + } + + hash_1 = DENSITY_BENCHMARK_HASH_SEED_1; + hash_2 = DENSITY_BENCHMARK_HASH_SEED_2; + spookyhash_128(in, uncompressed_size, &hash_1, &hash_2); + + if(hash_info) { + printf("Round-trip data hash is "); + DENSITY_BENCHMARK_BOLD(printf("0x%" PRIx64 "%" PRIx64, hash_1, hash_2)); + printf("\n"); + } + + if(hash_1 != original_hash_1 || hash_2 != original_hash_2) { + DENSITY_BENCHMARK_ERROR(printf("Uncompressed and round-trip data hashes do not match ("); + printf("0x%" PRIx64 "%" PRIx64, hash_1, hash_2); + printf(" vs. "); + printf("0x%" PRIx64 "%" PRIx64, original_hash_1, original_hash_2); + printf(").");, true); + } + + printf("Uncompressed and round-trip data hashes match. "); + } + printf("Starting main bench.\n"); + if (compression_only) + printf("Compressing "); + else + printf("Round-tripping "); + density_benchmark_format_decimal(uncompressed_size); + printf(" bytes to "); + density_benchmark_format_decimal(compressed_size); + printf(" bytes (compression ratio "); + DENSITY_BENCHMARK_BOLD(printf("%.2lf%%", (100.0 * compressed_size) / uncompressed_size)); + printf(" or "); + DENSITY_BENCHMARK_BOLD(printf("%.3fx", (1.0 * uncompressed_size) / compressed_size)); + if (compression_only) + printf(")\n"); + else + printf(") and back\n"); + fflush(stdout); + + // Main benchmark + unsigned int iterations = 0; + double compress_time_high = 0.0; + double compress_time_low = 60.0; + double decompress_time_high = 0.0; + double decompress_time_low = 60.0; + double total_compress_time = 0.0; + double total_decompress_time = 0.0; + double total_time = 0.0; + double decompress_speed = 0.0; + double decompress_speed_low = 0.0; + double decompress_speed_high = 0.0; + double compress_time_elapsed = 0.0; + double decompress_time_elapsed = 0.0; + cputime_chronometer chrono; + + while (total_time <= 10.0) { + ++iterations; + + cputime_chronometer_start(&chrono); + density_compress(in, uncompressed_size, out, memory_allocated, compression_mode); + compress_time_elapsed = cputime_chronometer_stop(&chrono); + + if (!compression_only) { + cputime_chronometer_start(&chrono); + density_decompress(out, compressed_size, in, memory_allocated); + decompress_time_elapsed = cputime_chronometer_stop(&chrono); + } + + total_compress_time += compress_time_elapsed; + + if (compress_time_elapsed < compress_time_low) + compress_time_low = compress_time_elapsed; + if (compress_time_elapsed > compress_time_high) + compress_time_high = compress_time_elapsed; + + double compress_speed = ((1.0 * uncompressed_size * iterations) / (total_compress_time * 1000.0 * 1000.0)); + double compress_speed_low = ((1.0 * uncompressed_size) / (compress_time_high * 1000.0 * 1000.0)); + double compress_speed_high = ((1.0 * uncompressed_size) / (compress_time_low * 1000.0 * 1000.0)); + + total_time += compress_time_elapsed; + + if (!compression_only) { + total_decompress_time += decompress_time_elapsed; + + if (decompress_time_elapsed < decompress_time_low) + decompress_time_low = decompress_time_elapsed; + if (decompress_time_elapsed > decompress_time_high) + decompress_time_high = decompress_time_elapsed; + + decompress_speed = ((1.0 * uncompressed_size * iterations) / (total_decompress_time * 1000.0 * 1000.0)); + decompress_speed_low = ((1.0 * uncompressed_size) / (decompress_time_high * 1000.0 * 1000.0)); + decompress_speed_high = ((1.0 * uncompressed_size) / (decompress_time_low * 1000.0 * 1000.0)); + + total_time += decompress_time_elapsed; + } + + DENSITY_BENCHMARK_BLUE(printf("\rCompress speed "); + DENSITY_BENCHMARK_BOLD(printf("%.0lf MB/s", compress_speed))); + printf(" (min %.0lf MB/s, max %.0lf MB/s, best %.4lfs) ", compress_speed_low, compress_speed_high, compress_time_low); + + if (!compression_only) { + printf("<=> "); + DENSITY_BENCHMARK_BLUE(printf("Decompress speed "); + DENSITY_BENCHMARK_BOLD(printf("%.0lf MB/s", decompress_speed))); + printf(" (min %.0lf MB/s, max %.0lf MB/s, best %.4lfs) ", decompress_speed_low, decompress_speed_high, decompress_time_low); + } + fflush(stdout); + } + printf("\nRun time %.3lfs (%i iterations)\n\n", total_time, iterations); + } + + free(in); + free(out); + + printf("Allocated memory released.\n\n"); + + return EXIT_SUCCESS; +} diff --git a/contrib/density/benchmark/src/benchmark.h b/contrib/density/benchmark/src/benchmark.h new file mode 100644 index 00000000000..749e1ccb1d1 --- /dev/null +++ b/contrib/density/benchmark/src/benchmark.h @@ -0,0 +1,129 @@ +/* + * Density benchmark + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program. If not, see . + * + * 6/04/15 0:11 + */ + +#ifndef DENSITY_BENCHMARK_H +#define DENSITY_BENCHMARK_H + +#define _CRT_SECURE_NO_DEPRECATE + +#include +#include +#include +#include +#include +#include +#include +#include +#include "../../src/density_api.h" +#include "../libs/cputime/src/cputime_api.h" +#include "../libs/spookyhash/src/spookyhash_api.h" + +#if defined(_WIN64) || defined(_WIN32) +#else +#define DENSITY_BENCHMARK_ALLOW_ANSI_ESCAPE_SEQUENCES +#endif + +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ +#define DENSITY_BENCHMARK_ENDIAN_STRING "Little" +#elif __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ +#define DENSITY_BENCHMARK_ENDIAN_STRING "Big" +#endif + +#if defined(__clang__) +#define DENSITY_BENCHMARK_COMPILER "Clang %d.%d.%d" +#define DENSITY_BENCHMARK_COMPILER_VERSION __clang_major__, __clang_minor__, __clang_patchlevel__ +#elif defined(__GNUC__) +#define DENSITY_BENCHMARK_COMPILER "GCC %d.%d.%d" +#define DENSITY_BENCHMARK_COMPILER_VERSION __GNUC__, __GNUC_MINOR__, __GNUC_PATCHLEVEL__ +#elif defined(_MSC_VER) +#define DENSITY_BENCHMARK_COMPILER "MSVC" +#define DENSITY_BENCHMARK_COMPILER_VERSION "" +#elif defined(__INTEL_COMPILER) +#define DENSITY_BENCHMARK_COMPILER "ICC" +#define DENSITY_BENCHMARK_COMPILER_VERSION "" +#else +#define DENSITY_BENCHMARK_COMPILER "an unknown compiler" +#define DENSITY_BENCHMARK_COMPILER_VERSION "" +#endif + +#if defined(_WIN64) || defined(_WIN32) +#define DENSITY_BENCHMARK_PLATFORM_STRING "Microsoft Windows" +#elif defined(__APPLE__) +#include "TargetConditionals.h" +#if TARGET_IPHONE_SIMULATOR +#define DENSITY_BENCHMARK_PLATFORM_STRING "iOS Simulator" +#elif TARGET_OS_IPHONE +#define DENSITY_BENCHMARK_PLATFORM_STRING "iOS" +#elif TARGET_OS_MAC +#define DENSITY_BENCHMARK_PLATFORM_STRING "MacOS" +#else +#define DENSITY_BENCHMARK_PLATFORM_STRING "an unknown Apple platform" +#endif +#elif defined(__FreeBSD__) +#define DENSITY_BENCHMARK_PLATFORM_STRING "FreeBSD" +#elif defined(__linux__) +#define DENSITY_BENCHMARK_PLATFORM_STRING "GNU/Linux" +#elif defined(__unix__) +#define DENSITY_BENCHMARK_PLATFORM_STRING "Unix" +#elif defined(__posix__) +#define DENSITY_BENCHMARK_PLATFORM_STRING "Posix" +#else +#define DENSITY_BENCHMARK_PLATFORM_STRING "an unknown platform" +#endif + +#define DENSITY_ESCAPE_CHARACTER ((char)27) + +#ifdef DENSITY_BENCHMARK_ALLOW_ANSI_ESCAPE_SEQUENCES +#define DENSITY_BENCHMARK_BOLD(op) printf("%c[1m", DENSITY_ESCAPE_CHARACTER);\ + op;\ + printf("%c[0m", DENSITY_ESCAPE_CHARACTER); + +#define DENSITY_BENCHMARK_BLUE(op) printf("%c[0;34m", DENSITY_ESCAPE_CHARACTER);\ + op;\ + printf("%c[0m", DENSITY_ESCAPE_CHARACTER); + +#define DENSITY_BENCHMARK_RED(op) printf("%c[0;31m", DENSITY_ESCAPE_CHARACTER);\ + op;\ + printf("%c[0m", DENSITY_ESCAPE_CHARACTER); +#else +#define DENSITY_BENCHMARK_BOLD(op) op; +#define DENSITY_BENCHMARK_BLUE(op) op; +#define DENSITY_BENCHMARK_RED(op) op; +#endif + +#define DENSITY_BENCHMARK_UNDERLINE(n) printf("\n");\ + for(int i = 0; i < n; i++) printf("="); + +#define DENSITY_BENCHMARK_ERROR(op, issue) DENSITY_BENCHMARK_RED(DENSITY_BENCHMARK_BOLD(printf("\nAn error has occured !\n")));\ + op;\ + printf("\n");\ + if(issue) {\ + printf("Please open an issue at , with your platform information and any relevant file.\n");\ + DENSITY_BENCHMARK_BOLD(printf("Thank you !\n"));\ + }\ + fflush(stdout);\ + exit(EXIT_FAILURE); + +#endif + +#define DENSITY_BENCHMARK_HASH_SEED_1 0x0123456789abcdefllu +#define DENSITY_BENCHMARK_HASH_SEED_2 0xfedcba9876543210llu diff --git a/contrib/density/msvc/Density.sln b/contrib/density/msvc/Density.sln new file mode 100644 index 00000000000..70c25ab3413 --- /dev/null +++ b/contrib/density/msvc/Density.sln @@ -0,0 +1,21 @@ + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio 14 +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "density", "density.vcxproj", "{65C51F09-D1A4-9EA4-DABC-297B461B0506}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "benchmark", "benchmark.vcxproj", "{7000C5C1-DC6A-7938-25A9-2ADE9152578D}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Release|x64 = Release|x64 + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {65C51F09-D1A4-9EA4-DABC-297B461B0506}.Release|x64.ActiveCfg = Release|x64 + {65C51F09-D1A4-9EA4-DABC-297B461B0506}.Release|x64.Build.0 = Release|x64 + {7000C5C1-DC6A-7938-25A9-2ADE9152578D}.Release|x64.ActiveCfg = Release|x64 + {7000C5C1-DC6A-7938-25A9-2ADE9152578D}.Release|x64.Build.0 = Release|x64 + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection +EndGlobal diff --git a/contrib/density/msvc/benchmark.vcxproj b/contrib/density/msvc/benchmark.vcxproj new file mode 100644 index 00000000000..ef32bbb5f3c --- /dev/null +++ b/contrib/density/msvc/benchmark.vcxproj @@ -0,0 +1,78 @@ + + + + + Release + x64 + + + + {7000C5C1-DC6A-7938-25A9-2ADE9152578D} + true + Win32Proj + benchmark + + + + Application + false + Unicode + v140 + true + + + + + + + + + + false + bin\Release\ + obj\Release\benchmark\ + benchmark + .exe + + + + NotUsing + Level4 + MaxSpeed + true + true + false + true + true + + + Console + true + true + + + + + + + + + + + + + + + + + + + + + {65C51F09-D1A4-9EA4-DABC-297B461B0506} + + + + + + \ No newline at end of file diff --git a/contrib/density/msvc/benchmark.vcxproj.filters b/contrib/density/msvc/benchmark.vcxproj.filters new file mode 100644 index 00000000000..26379e28724 --- /dev/null +++ b/contrib/density/msvc/benchmark.vcxproj.filters @@ -0,0 +1,63 @@ + + + + + {2F149A7C-1B4B-9B0D-C437-8110B04D170F} + + + {95C9EAC4-812C-7A69-2AB2-B21F16F445EC} + + + {2C4ACB69-1843-EABB-4175-CF402DCDC9C7} + + + {C78AF53E-3316-6303-3C27-E6F7A831BF03} + + + {5E54D780-CAB5-B48E-5323-FB40BF83EB4D} + + + {2DAB880B-99B4-887C-2230-9F7C8E38947C} + + + + + libs\cputime\src + + + libs\cputime\src + + + libs\spookyhash\src + + + libs\spookyhash\src + + + libs\spookyhash\src + + + libs\spookyhash\src + + + src + + + + + libs\cputime\src + + + libs\spookyhash\src + + + libs\spookyhash\src + + + libs\spookyhash\src + + + src + + + \ No newline at end of file diff --git a/contrib/density/msvc/density.vcxproj b/contrib/density/msvc/density.vcxproj new file mode 100644 index 00000000000..f1ad77fbc1b --- /dev/null +++ b/contrib/density/msvc/density.vcxproj @@ -0,0 +1,93 @@ + + + + + Release + x64 + + + + {65C51F09-D1A4-9EA4-DABC-297B461B0506} + true + Win32Proj + density + + + + DynamicLibrary + false + Unicode + v140 + true + + + + + + + + + + false + bin\Release\ + obj\Release\density\ + density + .dll + + + + NotUsing + Level4 + MaxSpeed + true + true + false + true + true + + + Windows + true + true + bin\Release\density.lib + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/contrib/density/msvc/density.vcxproj.filters b/contrib/density/msvc/density.vcxproj.filters new file mode 100644 index 00000000000..202f2522bbe --- /dev/null +++ b/contrib/density/msvc/density.vcxproj.filters @@ -0,0 +1,135 @@ + + + + + {DF10C562-CBC8-06B0-34D0-DF0B20A7A1A0} + + + {5AB95D49-4648-E712-EF66-FB0DDBD4F7B8} + + + {1252540B-7E14-8895-C7C4-162233C66073} + + + {3F6054BC-AB43-63FC-B446-913820A9294D} + + + {C09C0CB6-AC80-CD0B-15E1-C75E01E4B78D} + + + {3845B705-A47C-FF9C-AD86-63C0193D2696} + + + {E597066D-51B0-ED96-1A5D-7D3086348230} + + + {40FA2C44-AC85-9A08-B596-1DFD21A1F608} + + + {B8122E82-A4A1-B74B-4DC0-CB46392EC8F1} + + + {6585C1C9-5195-6D9A-BA4E-D2B8A67D5C33} + + + {B6B426C8-2221-E2C2-EB14-7A205740042B} + + + {B2DFE593-1EBF-642F-27D7-EF059335CB90} + + + {367A73B3-A2E4-272A-EB22-D9CF57CC057F} + + + + + algorithms + + + algorithms\chameleon + + + algorithms\chameleon\core + + + algorithms\chameleon\core + + + algorithms\chameleon\dictionary + + + algorithms\cheetah + + + algorithms\cheetah\core + + + algorithms\cheetah\core + + + algorithms\cheetah\dictionary + + + algorithms + + + algorithms\lion\core + + + algorithms\lion\core + + + algorithms\lion\dictionary + + + algorithms\lion\forms + + + algorithms\lion + + + buffers + + + + + structure + + + + + algorithms + + + algorithms\chameleon\core + + + algorithms\chameleon\core + + + algorithms\cheetah\core + + + algorithms\cheetah\core + + + algorithms + + + algorithms\lion\core + + + algorithms\lion\core + + + algorithms\lion\forms + + + buffers + + + + structure + + + \ No newline at end of file diff --git a/contrib/density/src/algorithms/algorithms.c b/contrib/density/src/algorithms/algorithms.c new file mode 100644 index 00000000000..f36e94cd08b --- /dev/null +++ b/contrib/density/src/algorithms/algorithms.c @@ -0,0 +1,43 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 14/10/15 02:06 + */ + +#include "algorithms.h" + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_algorithms_prepare_state(density_algorithm_state *const DENSITY_RESTRICT state, void *const DENSITY_RESTRICT dictionary) { + state->dictionary = dictionary; + state->copy_penalty = 0; + state->copy_penalty_start = 1; + state->previous_incompressible = false; + state->counter = 0; +} diff --git a/contrib/density/src/algorithms/algorithms.h b/contrib/density/src/algorithms/algorithms.h new file mode 100644 index 00000000000..b5e31377004 --- /dev/null +++ b/contrib/density/src/algorithms/algorithms.h @@ -0,0 +1,78 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 3/02/15 19:51 + */ + +#ifndef DENSITY_ALGORITHMS_H +#define DENSITY_ALGORITHMS_H + +#include "../globals.h" + +typedef enum { + DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED = 0, + DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING, + DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL, + DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL +} density_algorithm_exit_status; + +typedef struct { + void *dictionary; + uint_fast8_t copy_penalty; + uint_fast8_t copy_penalty_start; + bool previous_incompressible; + uint_fast64_t counter; +} density_algorithm_state; + +#define DENSITY_ALGORITHM_COPY(work_block_size)\ + DENSITY_MEMCPY(*out, *in, work_block_size);\ + *in += work_block_size;\ + *out += work_block_size; + +#define DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START\ + if(!(--state->copy_penalty))\ + state->copy_penalty_start++; + +#define DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START\ + if (state->copy_penalty_start & ~0x1)\ + state->copy_penalty_start >>= 1; + +#define DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY(span, work_block_size)\ + if (DENSITY_UNLIKELY(span & ~(work_block_size - 1))) {\ + if (state->previous_incompressible)\ + state->copy_penalty = state->copy_penalty_start;\ + state->previous_incompressible = true;\ + } else\ + state->previous_incompressible = false; + +DENSITY_WINDOWS_EXPORT void density_algorithms_prepare_state(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, void *const DENSITY_RESTRICT_DECLARE); + +#endif diff --git a/contrib/density/src/algorithms/chameleon/chameleon.h b/contrib/density/src/algorithms/chameleon/chameleon.h new file mode 100644 index 00000000000..2054f34cfd8 --- /dev/null +++ b/contrib/density/src/algorithms/chameleon/chameleon.h @@ -0,0 +1,70 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/10/13 11:57 + * + * ------------------- + * Chameleon algorithm + * ------------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Hash based superfast kernel + */ + +#ifndef DENSITY_CHAMELEON_H +#define DENSITY_CHAMELEON_H + +#include "../../globals.h" + +#define DENSITY_CHAMELEON_HASH_BITS 16 +#define DENSITY_CHAMELEON_HASH_MULTIPLIER (uint32_t)0x9D6EF916lu + +#define DENSITY_CHAMELEON_HASH_ALGORITHM(value32) (uint16_t)((value32 * DENSITY_CHAMELEON_HASH_MULTIPLIER) >> (32 - DENSITY_CHAMELEON_HASH_BITS)) + +typedef enum { + DENSITY_CHAMELEON_SIGNATURE_FLAG_CHUNK = 0x0, + DENSITY_CHAMELEON_SIGNATURE_FLAG_MAP = 0x1, +} DENSITY_CHAMELEON_SIGNATURE_FLAG; + +typedef uint64_t density_chameleon_signature; + +#define DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE (density_bitsizeof(density_chameleon_signature) * sizeof(uint32_t)) // Uncompressed chunks +#define DENSITY_CHAMELEON_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE (density_bitsizeof(density_chameleon_signature) * sizeof(uint32_t)) + +#define DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE (sizeof(density_chameleon_signature) + DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE) +#define DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE (DENSITY_CHAMELEON_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE) + +#define DENSITY_CHAMELEON_WORK_BLOCK_SIZE 256 + +#endif diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_decode.c b/contrib/density/src/algorithms/chameleon/core/chameleon_decode.c new file mode 100644 index 00000000000..240aaba8598 --- /dev/null +++ b/contrib/density/src/algorithms/chameleon/core/chameleon_decode.c @@ -0,0 +1,254 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 23/06/15 22:11 + * + * ------------------- + * Chameleon algorithm + * ------------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Hash based superfast kernel + */ + +#include "chameleon_decode.h" + +DENSITY_FORCE_INLINE void density_chameleon_decode_process_compressed(const uint16_t hash, uint8_t **DENSITY_RESTRICT out, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { + DENSITY_MEMCPY(*out, &dictionary->entries[hash].as_uint32_t, sizeof(uint32_t)); +} + +DENSITY_FORCE_INLINE void density_chameleon_decode_process_uncompressed(const uint32_t chunk, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { + const uint16_t hash = DENSITY_CHAMELEON_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(chunk)); + (&dictionary->entries[hash])->as_uint32_t = chunk; // Does not ensure dictionary content consistency between endiannesses +} + +DENSITY_FORCE_INLINE void density_chameleon_decode_kernel(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_bool compressed, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { + if (compressed) { + uint16_t hash; + DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); + density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16(hash), out, dictionary); + *in += sizeof(uint16_t); + } else { + uint32_t unit; + DENSITY_MEMCPY(&unit, *in, sizeof(uint32_t)); + density_chameleon_decode_process_uncompressed(unit, dictionary); + DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); + *in += sizeof(uint32_t); + } + *out += sizeof(uint32_t); +} + +DENSITY_FORCE_INLINE void density_chameleon_decode_kernel_dual(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_chameleon_signature signature, const uint_fast8_t shift, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { + uint32_t var_32; + uint64_t var_64; + + switch((signature >> shift) & 0x3) { + case 0x0: + DENSITY_MEMCPY(&var_64, *in, sizeof(uint32_t) + sizeof(uint32_t)); +#ifdef DENSITY_LITTLE_ENDIAN + density_chameleon_decode_process_uncompressed((uint32_t)(var_64 & 0xffffffff), dictionary); +#endif + density_chameleon_decode_process_uncompressed((uint32_t)(var_64 >> density_bitsizeof(uint32_t)), dictionary); +#ifdef DENSITY_BIG_ENDIAN + density_chameleon_decode_process_uncompressed((uint32_t)(var_64 & 0xffffffff), dictionary); +#endif + DENSITY_MEMCPY(*out, &var_64, sizeof(uint32_t) + sizeof(uint32_t)); + *in += (sizeof(uint32_t) + sizeof(uint32_t)); + *out += sizeof(uint64_t); + break; + case 0x1: + DENSITY_MEMCPY(&var_64, *in, sizeof(uint16_t) + sizeof(uint32_t)); +#ifdef DENSITY_LITTLE_ENDIAN + density_chameleon_decode_process_compressed((uint16_t)(var_64 & 0xffff), out, dictionary); + var_32 = (uint32_t)((var_64 >> density_bitsizeof(uint16_t)) & 0xffffffff); + density_chameleon_decode_process_uncompressed(var_32, dictionary); + DENSITY_MEMCPY(*out + sizeof(uint32_t), &var_32, sizeof(uint32_t)); + *out += sizeof(uint64_t); +#elif defined(DENSITY_BIG_ENDIAN) + density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)((var_64 >> (density_bitsizeof(uint16_t) + density_bitsizeof(uint32_t))) & 0xffff)), out, dictionary); + var_32 = (uint32_t)((var_64 >> density_bitsizeof(uint16_t)) & 0xffffffff); + density_chameleon_decode_process_uncompressed(var_32, dictionary); + DENSITY_MEMCPY(*out + sizeof(uint32_t), &var_32, sizeof(uint32_t)); + *out += sizeof(uint64_t); +#else +#error +#endif + *in += (sizeof(uint16_t) + sizeof(uint32_t)); + break; + case 0x2: + DENSITY_MEMCPY(&var_64, *in, sizeof(uint32_t) + sizeof(uint16_t)); +#ifdef DENSITY_LITTLE_ENDIAN + var_32 = (uint32_t)(var_64 & 0xffffffff); + density_chameleon_decode_process_uncompressed(var_32, dictionary); + DENSITY_MEMCPY(*out, &var_32, sizeof(uint32_t)); + *out += sizeof(uint32_t); + density_chameleon_decode_process_compressed((uint16_t)((var_64 >> density_bitsizeof(uint32_t)) & 0xffff), out, dictionary); + *out += sizeof(uint32_t); +#elif defined(DENSITY_BIG_ENDIAN) + var_32 = (uint32_t)((var_64 >> density_bitsizeof(uint32_t)) & 0xffffffff); + density_chameleon_decode_process_uncompressed(var_32, dictionary); + DENSITY_MEMCPY(*out, &var_32, sizeof(uint32_t)); + *out += sizeof(uint32_t); + density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)((var_64 >> density_bitsizeof(uint16_t)) & 0xffff)), out, dictionary); + *out += sizeof(uint32_t); +#else +#error +#endif + *in += (sizeof(uint32_t) + sizeof(uint16_t)); + break; + case 0x3: + DENSITY_MEMCPY(&var_32, *in, sizeof(uint16_t) + sizeof(uint16_t)); +#ifdef DENSITY_LITTLE_ENDIAN + density_chameleon_decode_process_compressed((uint16_t)(var_32 & 0xffff), out, dictionary); + *out += sizeof(uint32_t); +#endif + density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)(var_32 >> density_bitsizeof(uint16_t))), out, dictionary); + *out += sizeof(uint32_t); +#ifdef DENSITY_BIG_ENDIAN + density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)(var_32 & 0xffff)), out, dictionary); + *out += sizeof(uint32_t); +#endif + *in += (sizeof(uint16_t) + sizeof(uint16_t)); + break; + } +} + +DENSITY_FORCE_INLINE bool density_chameleon_decode_test_compressed(const density_chameleon_signature signature, const uint_fast8_t shift) { + return (density_bool const) ((signature >> shift) & DENSITY_CHAMELEON_SIGNATURE_FLAG_MAP); +} + +DENSITY_FORCE_INLINE void density_chameleon_decode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_chameleon_signature signature, const uint_fast8_t shift, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { + density_chameleon_decode_kernel(in, out, density_chameleon_decode_test_compressed(signature, shift), dictionary); +} + +DENSITY_FORCE_INLINE void density_chameleon_decode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_chameleon_signature signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { + uint_fast8_t count_a = 0; + uint_fast8_t count_b = 0; + +#if defined(__clang__) || defined(_MSC_VER) + do { + DENSITY_UNROLL_2(density_chameleon_decode_kernel_dual(in, out, signature, count_a, dictionary); count_a+= 2); + } while (++count_b & 0xf); +#else + do { + DENSITY_UNROLL_2(density_chameleon_decode_4(in, out, signature, count_a ++, dictionary)); + } while (++count_b & 0x1f); +#endif +} + +DENSITY_FORCE_INLINE void density_chameleon_decode_read_signature(const uint8_t **DENSITY_RESTRICT in, density_chameleon_signature *DENSITY_RESTRICT signature) { +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature, *in, sizeof(density_chameleon_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + density_chameleon_signature endian_signature; + DENSITY_MEMCPY(&endian_signature, *in, sizeof(density_chameleon_signature)); + *signature = DENSITY_LITTLE_ENDIAN_64(endian_signature); +#else +#error +#endif + *in += sizeof(density_chameleon_signature); +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_chameleon_decode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { + if (out_size < DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + density_chameleon_signature signature; + uint_fast8_t shift; + uint_fast64_t remaining; + + const uint8_t *start = *in; + + if (in_size < DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE) { + goto read_signature; + } + + const uint8_t *in_limit = *in + in_size - DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE; + uint8_t *out_limit = *out + out_size - DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE; + + while (DENSITY_LIKELY(*in <= in_limit && *out <= out_limit)) { + if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { + DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; + } + state->counter++; + if (DENSITY_UNLIKELY(state->copy_penalty)) { + DENSITY_ALGORITHM_COPY(DENSITY_CHAMELEON_WORK_BLOCK_SIZE); + DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; + } else { + const uint8_t *in_start = *in; + density_chameleon_decode_read_signature(in, &signature); + density_chameleon_decode_256(in, out, signature, (density_chameleon_dictionary *const) state->dictionary); + DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*in - in_start), DENSITY_CHAMELEON_WORK_BLOCK_SIZE); + } + } + + if (*out > out_limit) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + read_signature: + if (in_size - (*in - start) < sizeof(density_chameleon_signature)) + return DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL; + shift = 0; + density_chameleon_decode_read_signature(in, &signature); + read_and_decode_4: + switch (in_size - (*in - start)) { + case 0: + case 1: + if (density_chameleon_decode_test_compressed(signature, shift)) + return DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; + else // End marker + goto process_remaining_bytes; + case 2: + case 3: + if (density_chameleon_decode_test_compressed(signature, shift++)) + density_chameleon_decode_kernel(in, out, true, (density_chameleon_dictionary *const) state->dictionary); + else // End marker + goto process_remaining_bytes; + break; + default: + density_chameleon_decode_4(in, out, signature, shift++, (density_chameleon_dictionary *const) state->dictionary); + break; + } + + if (DENSITY_UNLIKELY(shift == density_bitsizeof(density_chameleon_signature))) + goto read_signature; + else + goto read_and_decode_4; + + process_remaining_bytes: + remaining = in_size - (*in - start); + DENSITY_ALGORITHM_COPY(remaining); + + return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; +} diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_decode.h b/contrib/density/src/algorithms/chameleon/core/chameleon_decode.h new file mode 100644 index 00000000000..cbf06e26a44 --- /dev/null +++ b/contrib/density/src/algorithms/chameleon/core/chameleon_decode.h @@ -0,0 +1,53 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 23/06/15 21:49 + * + * ------------------- + * Chameleon algorithm + * ------------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Hash based superfast kernel + */ + +#ifndef DENSITY_CHAMELEON_DECODE_H +#define DENSITY_CHAMELEON_DECODE_H + +#include "../dictionary/chameleon_dictionary.h" +#include "../../algorithms.h" + +DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_chameleon_decode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_encode.c b/contrib/density/src/algorithms/chameleon/core/chameleon_encode.c new file mode 100644 index 00000000000..2090cb99173 --- /dev/null +++ b/contrib/density/src/algorithms/chameleon/core/chameleon_encode.c @@ -0,0 +1,179 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 23/06/15 22:02 + * + * ------------------- + * Chameleon algorithm + * ------------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Hash based superfast kernel + */ + +#include "chameleon_encode.h" + +DENSITY_FORCE_INLINE void density_chameleon_encode_prepare_signature(uint8_t **DENSITY_RESTRICT out, density_chameleon_signature **DENSITY_RESTRICT signature_pointer, density_chameleon_signature *const DENSITY_RESTRICT signature) { + *signature = 0; + *signature_pointer = (density_chameleon_signature *) *out; + *out += sizeof(density_chameleon_signature); +} + +DENSITY_FORCE_INLINE void density_chameleon_encode_kernel(uint8_t **DENSITY_RESTRICT out, const uint16_t hash, const uint_fast8_t shift, density_chameleon_signature *const DENSITY_RESTRICT signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { + density_chameleon_dictionary_entry *const found = &dictionary->entries[hash]; + + switch (*unit ^ found->as_uint32_t) { + case 0: + *signature |= ((uint64_t) DENSITY_CHAMELEON_SIGNATURE_FLAG_MAP << shift); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + break; + default: + found->as_uint32_t = *unit; // Does not ensure dictionary content consistency between endiannesses + DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); + *out += sizeof(uint32_t); + break; + } +} + +DENSITY_FORCE_INLINE void density_chameleon_encode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const uint_fast8_t shift, density_chameleon_signature *const DENSITY_RESTRICT signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { + DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); + density_chameleon_encode_kernel(out, DENSITY_CHAMELEON_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)), shift, signature, dictionary, unit); + *in += sizeof(uint32_t); +} + +DENSITY_FORCE_INLINE void density_chameleon_encode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, density_chameleon_signature *const DENSITY_RESTRICT signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { + uint_fast8_t count = 0; + +#ifdef __clang__ + for (uint_fast8_t count_b = 0; count_b < 32; count_b++) { + DENSITY_UNROLL_2(density_chameleon_encode_4(in, out, count++, signature, dictionary, unit)); + } +#else + for (uint_fast8_t count_b = 0; count_b < 16; count_b++) { + DENSITY_UNROLL_4(density_chameleon_encode_4(in, out, count++, signature, dictionary, unit)); + } +#endif +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_chameleon_encode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { + if (out_size < DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + density_chameleon_signature signature; + density_chameleon_signature *signature_pointer; + uint32_t unit; + + uint8_t *out_limit = *out + out_size - DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE; + uint_fast64_t limit_256 = (in_size >> 8); + + while (DENSITY_LIKELY(limit_256-- && *out <= out_limit)) { + if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { + DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; + } + state->counter++; + if (DENSITY_UNLIKELY(state->copy_penalty)) { + DENSITY_ALGORITHM_COPY(DENSITY_CHAMELEON_WORK_BLOCK_SIZE); + DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; + } else { + const uint8_t *out_start = *out; + density_chameleon_encode_prepare_signature(out, &signature_pointer, &signature); + DENSITY_PREFETCH(*in + DENSITY_CHAMELEON_WORK_BLOCK_SIZE); + density_chameleon_encode_256(in, out, &signature, (density_chameleon_dictionary *const) state->dictionary, &unit); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_chameleon_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_chameleon_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_chameleon_signature)); +#else +#error +#endif + DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*out - out_start), DENSITY_CHAMELEON_WORK_BLOCK_SIZE); + } + } + + if (*out > out_limit) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + uint_fast64_t remaining; + + switch (in_size & 0xff) { + case 0: + case 1: + case 2: + case 3: + density_chameleon_encode_prepare_signature(out, &signature_pointer, &signature); + signature = ((uint64_t) DENSITY_CHAMELEON_SIGNATURE_FLAG_CHUNK); // End marker +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_chameleon_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_chameleon_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_chameleon_signature)); +#else +#error +#endif + goto process_remaining_bytes; + default: + break; + } + + const uint_fast64_t limit_4 = (in_size & 0xff) >> 2; + density_chameleon_encode_prepare_signature(out, &signature_pointer, &signature); + for (uint_fast8_t shift = 0; shift != limit_4; shift++) + density_chameleon_encode_4(in, out, shift, &signature, (density_chameleon_dictionary *const) state->dictionary, &unit); + + signature |= ((uint64_t) DENSITY_CHAMELEON_SIGNATURE_FLAG_CHUNK << limit_4); // End marker +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_chameleon_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_chameleon_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_chameleon_signature)); +#else +#error +#endif + + process_remaining_bytes: + remaining = in_size & 0x3; + if (remaining) + DENSITY_ALGORITHM_COPY(remaining); + + return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; +} diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_encode.h b/contrib/density/src/algorithms/chameleon/core/chameleon_encode.h new file mode 100644 index 00000000000..f71fc23fbc0 --- /dev/null +++ b/contrib/density/src/algorithms/chameleon/core/chameleon_encode.h @@ -0,0 +1,53 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 23/06/15 21:51 + * + * ------------------- + * Chameleon algorithm + * ------------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Hash based superfast kernel + */ + +#ifndef DENSITY_CHAMELEON_ENCODE_H +#define DENSITY_CHAMELEON_ENCODE_H + +#include "../dictionary/chameleon_dictionary.h" +#include "../../algorithms.h" + +DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_chameleon_encode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h b/contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h new file mode 100644 index 00000000000..1f584a5936e --- /dev/null +++ b/contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h @@ -0,0 +1,63 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/10/13 12:05 + * + * ------------------- + * Chameleon algorithm + * ------------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Hash based superfast kernel + */ + +#ifndef DENSITY_CHAMELEON_DICTIONARY_H +#define DENSITY_CHAMELEON_DICTIONARY_H + +#include "../chameleon.h" + +#include + +#pragma pack(push) +#pragma pack(4) +typedef struct { + uint32_t as_uint32_t; +} density_chameleon_dictionary_entry; + +typedef struct { + density_chameleon_dictionary_entry entries[1 << DENSITY_CHAMELEON_HASH_BITS]; +} density_chameleon_dictionary; +#pragma pack(pop) + +#endif diff --git a/contrib/density/src/algorithms/cheetah/cheetah.h b/contrib/density/src/algorithms/cheetah/cheetah.h new file mode 100644 index 00000000000..0d47d0ab502 --- /dev/null +++ b/contrib/density/src/algorithms/cheetah/cheetah.h @@ -0,0 +1,73 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 06/12/13 20:10 + * + * ----------------- + * Cheetah algorithm + * ----------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * Piotr Tarsa (https://github.com/tarsa) + * + * Description + * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup + */ + +#ifndef DENSITY_CHEETAH_H +#define DENSITY_CHEETAH_H + +#include "../../globals.h" + +#define DENSITY_CHEETAH_HASH_BITS 16 +#define DENSITY_CHEETAH_HASH_MULTIPLIER (uint32_t)0x9D6EF916lu + +#define DENSITY_CHEETAH_HASH_ALGORITHM(value32) (uint16_t)(value32 * DENSITY_CHEETAH_HASH_MULTIPLIER >> (32 - DENSITY_CHEETAH_HASH_BITS)) + +typedef enum { + DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED = 0x0, + DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A = 0x1, + DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B = 0x2, + DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK = 0x3, +} DENSITY_CHEETAH_SIGNATURE_FLAG; + +typedef uint64_t density_cheetah_signature; + +#define DENSITY_CHEETAH_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE ((density_bitsizeof(density_cheetah_signature) >> 1) * sizeof(uint32_t)) // Uncompressed chunks +#define DENSITY_CHEETAH_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE ((density_bitsizeof(density_cheetah_signature) >> 1) * sizeof(uint32_t)) + +#define DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE (sizeof(density_cheetah_signature) + DENSITY_CHEETAH_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE) +#define DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE (DENSITY_CHEETAH_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE) + +#define DENSITY_CHEETAH_WORK_BLOCK_SIZE 128 + +#endif \ No newline at end of file diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_decode.c b/contrib/density/src/algorithms/cheetah/core/cheetah_decode.c new file mode 100644 index 00000000000..4928938b41e --- /dev/null +++ b/contrib/density/src/algorithms/cheetah/core/cheetah_decode.c @@ -0,0 +1,266 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/06/15 0:32 + * + * ----------------- + * Cheetah algorithm + * ----------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * Piotr Tarsa (https://github.com/tarsa) + * + * Description + * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup + */ + +#include "cheetah_decode.h" + +DENSITY_FORCE_INLINE void density_cheetah_decode_process_predicted(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { + const uint32_t unit = dictionary->prediction_entries[*last_hash].next_chunk_prediction; + DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); + *last_hash = DENSITY_CHEETAH_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(unit)); +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_process_compressed_a(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, const uint16_t hash) { + DENSITY_PREFETCH(&dictionary->prediction_entries[hash]); + const uint32_t unit = dictionary->entries[hash].chunk_a; + DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); + dictionary->prediction_entries[*last_hash].next_chunk_prediction = unit; + *last_hash = hash; +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_process_compressed_b(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, const uint16_t hash) { + DENSITY_PREFETCH(&dictionary->prediction_entries[hash]); + density_cheetah_dictionary_entry *const entry = &dictionary->entries[hash]; + const uint32_t unit = entry->chunk_b; + entry->chunk_b = entry->chunk_a; + entry->chunk_a = unit; // Does not ensure dictionary content consistency between endiannesses + DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); + dictionary->prediction_entries[*last_hash].next_chunk_prediction = unit; + *last_hash = hash; +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_process_uncompressed(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, const uint32_t unit) { + const uint16_t hash = DENSITY_CHEETAH_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(unit)); + DENSITY_PREFETCH(&dictionary->prediction_entries[hash]); + density_cheetah_dictionary_entry *const entry = &dictionary->entries[hash]; + entry->chunk_b = entry->chunk_a; + entry->chunk_a = unit; // Does not ensure dictionary content consistency between endiannesses + DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); + dictionary->prediction_entries[*last_hash].next_chunk_prediction = unit; // Does not ensure dictionary content consistency between endiannesses + *last_hash = hash; +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_kernel_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint8_t flag, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { + uint16_t hash; + uint32_t unit; + + switch (flag) { + case DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED: + density_cheetah_decode_process_predicted(out, last_hash, dictionary); + break; + case DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A: + DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); + density_cheetah_decode_process_compressed_a(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash)); + *in += sizeof(uint16_t); + break; + case DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B: + DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); + density_cheetah_decode_process_compressed_b(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash)); + *in += sizeof(uint16_t); + break; + default: // DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK + DENSITY_MEMCPY(&unit, *in, sizeof(uint32_t)); + density_cheetah_decode_process_uncompressed(out, last_hash, dictionary, unit); + *in += sizeof(uint32_t); + break; + } + + *out += sizeof(uint32_t); +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_kernel_16(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint8_t flags, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { + uint16_t hash; + uint32_t unit; + + switch (flags) { + DENSITY_CASE_GENERATOR_4_4_COMBINED(\ + density_cheetah_decode_process_predicted(out, last_hash, dictionary);, \ + DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED, \ + DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); \ + density_cheetah_decode_process_compressed_a(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash));\ + *in += sizeof(uint16_t);, \ + DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A, \ + DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); \ + density_cheetah_decode_process_compressed_b(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash));\ + *in += sizeof(uint16_t);, \ + DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B, \ + DENSITY_MEMCPY(&unit, *in, sizeof(uint32_t)); \ + density_cheetah_decode_process_uncompressed(out, last_hash, dictionary, unit);\ + *in += sizeof(uint32_t);, \ + DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK, \ + *out += sizeof(uint32_t);, \ + 2\ + ); + default: + break; + } + + *out += sizeof(uint32_t); +} + +DENSITY_FORCE_INLINE uint8_t density_cheetah_decode_read_flag(const density_cheetah_signature signature, const uint_fast8_t shift) { + return (uint8_t const) ((signature >> shift) & 0x3); +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const density_cheetah_signature signature, const uint_fast8_t shift, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { + density_cheetah_decode_kernel_4(in, out, last_hash, density_cheetah_decode_read_flag(signature, shift), dictionary); +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_16(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const density_cheetah_signature signature, const uint_fast8_t shift, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { + density_cheetah_decode_kernel_16(in, out, last_hash, (uint8_t const) ((signature >> shift) & 0xff), dictionary); +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_128(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const density_cheetah_signature signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { +#ifdef __clang__ + uint_fast8_t count = 0; + for (uint_fast8_t count_b = 0; count_b < 8; count_b ++) { + density_cheetah_decode_16(in, out, last_hash, signature, count, dictionary); + count += 8; + } +#else + for (uint_fast8_t count_b = 0; count_b < density_bitsizeof(density_cheetah_signature); count_b += 8) + density_cheetah_decode_16(in, out, last_hash, signature, count_b, dictionary); +#endif +} + +DENSITY_FORCE_INLINE void density_cheetah_decode_read_signature(const uint8_t **DENSITY_RESTRICT in, density_cheetah_signature *DENSITY_RESTRICT signature) { +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature, *in, sizeof(density_cheetah_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + density_cheetah_signature endian_signature; + DENSITY_MEMCPY(&endian_signature, *in, sizeof(density_cheetah_signature)); + *signature = DENSITY_LITTLE_ENDIAN_64(endian_signature); +#else +#error +#endif + *in += sizeof(density_cheetah_signature); +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_cheetah_decode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { + if (out_size < DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + density_cheetah_signature signature; + uint_fast8_t shift; + uint_fast64_t remaining; + uint_fast16_t last_hash = 0; + uint8_t flag; + + const uint8_t *start = *in; + + if (in_size < DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE) { + goto read_signature; + } + + const uint8_t *in_limit = *in + in_size - DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE; + uint8_t *out_limit = *out + out_size - DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE; + + while (DENSITY_LIKELY(*in <= in_limit && *out <= out_limit)) { + if (DENSITY_UNLIKELY(!(state->counter & 0x1f))) { + DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; + } + state->counter++; + if (DENSITY_UNLIKELY(state->copy_penalty)) { + DENSITY_ALGORITHM_COPY(DENSITY_CHEETAH_WORK_BLOCK_SIZE); + DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; + } else { + const uint8_t *in_start = *in; + density_cheetah_decode_read_signature(in, &signature); + density_cheetah_decode_128(in, out, &last_hash, signature, (density_cheetah_dictionary *const) state->dictionary); + DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*in - in_start), DENSITY_CHEETAH_WORK_BLOCK_SIZE); + } + } + + if (*out > out_limit) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + read_signature: + if (in_size - (*in - start) < sizeof(density_cheetah_signature)) + return DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL; + shift = 0; + density_cheetah_decode_read_signature(in, &signature); + read_and_decode_4: + switch (in_size - (*in - start)) { + case 0: + case 1: + switch (density_cheetah_decode_read_flag(signature, shift)) { + case DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK: + goto process_remaining_bytes; // End marker + case DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED: + density_cheetah_decode_kernel_4(in, out, &last_hash, DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED, (density_cheetah_dictionary *const) state->dictionary); + shift += 2; + break; + default: + return DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; + } + break; + case 2: + case 3: + flag = density_cheetah_decode_read_flag(signature, shift); + switch (flag) { + case DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK: + goto process_remaining_bytes; // End marker + default: + density_cheetah_decode_kernel_4(in, out, &last_hash, flag, (density_cheetah_dictionary *const) state->dictionary); + shift += 2; + break; + } + break; + default: + density_cheetah_decode_4(in, out, &last_hash, signature, shift, (density_cheetah_dictionary *const) state->dictionary); + shift += 2; + break; + } + + if (DENSITY_UNLIKELY(shift == density_bitsizeof(density_cheetah_signature))) + goto read_signature; + else + goto read_and_decode_4; + + process_remaining_bytes: + remaining = in_size - (*in - start); + DENSITY_ALGORITHM_COPY(remaining); + + return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; +} diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_decode.h b/contrib/density/src/algorithms/cheetah/core/cheetah_decode.h new file mode 100644 index 00000000000..278368e7938 --- /dev/null +++ b/contrib/density/src/algorithms/cheetah/core/cheetah_decode.h @@ -0,0 +1,54 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/06/15 0:31 + * + * ----------------- + * Cheetah algorithm + * ----------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * Piotr Tarsa (https://github.com/tarsa) + * + * Description + * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup + */ + +#ifndef DENSITY_CHEETAH_DECODE_H +#define DENSITY_CHEETAH_DECODE_H + +#include "../dictionary/cheetah_dictionary.h" +#include "../../algorithms.h" + +DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_cheetah_decode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_encode.c b/contrib/density/src/algorithms/cheetah/core/cheetah_encode.c new file mode 100644 index 00000000000..6cec16f4bf0 --- /dev/null +++ b/contrib/density/src/algorithms/cheetah/core/cheetah_encode.c @@ -0,0 +1,202 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 23/06/15 23:29 + * + * ----------------- + * Cheetah algorithm + * ----------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * Piotr Tarsa (https://github.com/tarsa) + * + * Description + * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup + */ + +#include "cheetah_encode.h" + +DENSITY_FORCE_INLINE void density_cheetah_encode_prepare_signature(uint8_t **DENSITY_RESTRICT out, density_cheetah_signature **DENSITY_RESTRICT signature_pointer, density_cheetah_signature *const DENSITY_RESTRICT signature) { + *signature = 0; + *signature_pointer = (density_cheetah_signature *) *out; + *out += sizeof(density_cheetah_signature); +} + +DENSITY_FORCE_INLINE void density_cheetah_encode_kernel(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint_fast16_t hash, const uint_fast8_t shift, density_cheetah_signature *const DENSITY_RESTRICT signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { + uint32_t *predictedChunk = (uint32_t*) &dictionary->prediction_entries[*last_hash]; + + if (*predictedChunk ^ *unit) { + density_cheetah_dictionary_entry *found = &dictionary->entries[hash]; + uint32_t *found_a = &found->chunk_a; + if (*found_a ^ *unit) { + uint32_t *found_b = &found->chunk_b; + if (*found_b ^ *unit) { + *signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK << shift); + DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); + *out += sizeof(uint32_t); + } else { + *signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B << shift); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + } + *found_b = *found_a; + *found_a = *unit; // Does not ensure dictionary content consistency between endiannesses + } else { + *signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A << shift); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + } + *predictedChunk = *unit; // Does not ensure dictionary content consistency between endiannesses + } + *last_hash = hash; +} + +DENSITY_FORCE_INLINE void density_cheetah_encode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint_fast8_t shift, density_cheetah_signature *const DENSITY_RESTRICT signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { + DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); + *in += sizeof(uint32_t); + density_cheetah_encode_kernel(out, last_hash, DENSITY_CHEETAH_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)), shift, signature, dictionary, unit); +} + +DENSITY_FORCE_INLINE void density_cheetah_encode_128(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_signature *const DENSITY_RESTRICT signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { + uint_fast8_t count = 0; + +#ifdef __clang__ + for(; count < density_bitsizeof(density_cheetah_signature); count += 2) { + density_cheetah_encode_4(in, out, last_hash, count, signature, dictionary, unit); + } +#else + for (uint_fast8_t count_b = 0; count_b < 16; count_b++) { + DENSITY_UNROLL_2(\ + density_cheetah_encode_4(in, out, last_hash, count, signature, dictionary, unit);\ + count += 2); + } +#endif +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_cheetah_encode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { + if (out_size < DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + density_cheetah_signature signature; + density_cheetah_signature *signature_pointer; + uint_fast16_t last_hash = 0; + uint32_t unit; + + uint8_t *out_limit = *out + out_size - DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE; + uint_fast64_t limit_128 = (in_size >> 7); + + while (DENSITY_LIKELY(limit_128-- && *out <= out_limit)) { + if (DENSITY_UNLIKELY(!(state->counter & 0x1f))) { + DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; + } + state->counter++; + if (DENSITY_UNLIKELY(state->copy_penalty)) { + DENSITY_ALGORITHM_COPY(DENSITY_CHEETAH_WORK_BLOCK_SIZE); + DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; + } else { + const uint8_t *out_start = *out; + density_cheetah_encode_prepare_signature(out, &signature_pointer, &signature); + DENSITY_PREFETCH(*in + DENSITY_CHEETAH_WORK_BLOCK_SIZE); + density_cheetah_encode_128(in, out, &last_hash, &signature, (density_cheetah_dictionary *const) state->dictionary, &unit); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_cheetah_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_cheetah_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_cheetah_signature)); +#else +#error +#endif + DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*out - out_start), DENSITY_CHEETAH_WORK_BLOCK_SIZE); + } + } + + if (*out > out_limit) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + uint_fast64_t remaining; + + switch (in_size & 0x7f) { + case 0: + case 1: + case 2: + case 3: + density_cheetah_encode_prepare_signature(out, &signature_pointer, &signature); + signature = (uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK; // End marker +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_cheetah_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_cheetah_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_cheetah_signature)); +#else +#error +#endif + goto process_remaining_bytes; + default: + break; + } + + const uint_fast64_t limit_4 = ((in_size & 0x7f) >> 2) << 1; // 4-byte units times number of signature flag bits + density_cheetah_encode_prepare_signature(out, &signature_pointer, &signature); + for (uint_fast8_t shift = 0; shift != limit_4; shift += 2) + density_cheetah_encode_4(in, out, &last_hash, shift, &signature, (density_cheetah_dictionary *const) state->dictionary, &unit); + + signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK << limit_4); // End marker +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_cheetah_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_cheetah_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_cheetah_signature)); +#else +#error +#endif + + process_remaining_bytes: + remaining = in_size & 0x3; + if (remaining) + DENSITY_ALGORITHM_COPY(remaining); + + return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; +} diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_encode.h b/contrib/density/src/algorithms/cheetah/core/cheetah_encode.h new file mode 100644 index 00000000000..9f67e5bad9d --- /dev/null +++ b/contrib/density/src/algorithms/cheetah/core/cheetah_encode.h @@ -0,0 +1,54 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 23/06/15 23:29 + * + * ----------------- + * Cheetah algorithm + * ----------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * Piotr Tarsa (https://github.com/tarsa) + * + * Description + * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup + */ + +#ifndef DENSITY_CHEETAH_ENCODE_H +#define DENSITY_CHEETAH_ENCODE_H + +#include "../dictionary/cheetah_dictionary.h" +#include "../../algorithms.h" + +DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_cheetah_encode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h b/contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h new file mode 100644 index 00000000000..5223d843574 --- /dev/null +++ b/contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h @@ -0,0 +1,70 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 06/12/13 20:20 + * + * ----------------- + * Cheetah algorithm + * ----------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * Piotr Tarsa (https://github.com/tarsa) + * + * Description + * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup + */ + +#ifndef DENSITY_CHEETAH_DICTIONARY_H +#define DENSITY_CHEETAH_DICTIONARY_H + +#include "../cheetah.h" + +#include + +#pragma pack(push) +#pragma pack(4) +typedef struct { + uint32_t chunk_a; + uint32_t chunk_b; +} density_cheetah_dictionary_entry; + +typedef struct { + uint32_t next_chunk_prediction; +} density_cheetah_dictionary_prediction_entry; + +typedef struct { + density_cheetah_dictionary_entry entries[1 << DENSITY_CHEETAH_HASH_BITS]; + density_cheetah_dictionary_prediction_entry prediction_entries[1 << DENSITY_CHEETAH_HASH_BITS]; +} density_cheetah_dictionary; +#pragma pack(pop) + +#endif diff --git a/contrib/density/src/algorithms/dictionaries.c b/contrib/density/src/algorithms/dictionaries.c new file mode 100644 index 00000000000..1e8d5466014 --- /dev/null +++ b/contrib/density/src/algorithms/dictionaries.c @@ -0,0 +1,48 @@ +/* + * Density + * + * Copyright (c) 2018, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 13/01/18 17:31 + */ + +#include "dictionaries.h" + +DENSITY_WINDOWS_EXPORT size_t density_get_dictionary_size(DENSITY_ALGORITHM algorithm) { + switch(algorithm) { + case DENSITY_ALGORITHM_CHAMELEON: + return sizeof(density_chameleon_dictionary); + case DENSITY_ALGORITHM_CHEETAH: + return sizeof(density_cheetah_dictionary); + case DENSITY_ALGORITHM_LION: + return sizeof(density_lion_dictionary); + default: + return 0; + } +} diff --git a/contrib/density/src/algorithms/dictionaries.h b/contrib/density/src/algorithms/dictionaries.h new file mode 100644 index 00000000000..291c361963e --- /dev/null +++ b/contrib/density/src/algorithms/dictionaries.h @@ -0,0 +1,45 @@ +/* + * Density + * + * Copyright (c) 2018, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 13/01/18 17:26 + */ + +#ifndef DENSITY_DICTIONARIES_H +#define DENSITY_DICTIONARIES_H + +#include "../globals.h" +#include "../algorithms/chameleon/dictionary/chameleon_dictionary.h" +#include "../algorithms/cheetah/dictionary/cheetah_dictionary.h" +#include "../algorithms/lion/dictionary/lion_dictionary.h" + +DENSITY_WINDOWS_EXPORT size_t density_get_dictionary_size(DENSITY_ALGORITHM); + +#endif diff --git a/contrib/density/src/algorithms/lion/core/lion_decode.c b/contrib/density/src/algorithms/lion/core/lion_decode.c new file mode 100644 index 00000000000..c84262c0d50 --- /dev/null +++ b/contrib/density/src/algorithms/lion/core/lion_decode.c @@ -0,0 +1,327 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/06/15 20:55 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#include "lion_decode.h" + +DENSITY_FORCE_INLINE void density_lion_decode_read_signature(const uint8_t **DENSITY_RESTRICT in, uint_fast64_t *const DENSITY_RESTRICT signature) { +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature, *in, sizeof(density_lion_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + density_lion_signature endian_signature; + DENSITY_MEMCPY(&endian_signature, *in, sizeof(density_lion_signature)); + *signature = DENSITY_LITTLE_ENDIAN_64(endian_signature); +#else +#error +#endif + *in += sizeof(density_lion_signature); +} + +DENSITY_FORCE_INLINE void density_lion_decode_update_predictions_model(density_lion_dictionary_chunk_prediction_entry *const DENSITY_RESTRICT predictions, const uint32_t chunk) { + DENSITY_MEMMOVE((uint32_t *) predictions + 1, predictions, 2 * sizeof(uint32_t)); + *(uint32_t *) predictions = chunk; // Move chunk to the top of the predictions list, does not ensure dictionary content consistency between endiannesses +} + +DENSITY_FORCE_INLINE void density_lion_decode_update_dictionary_model(density_lion_dictionary_chunk_entry *const DENSITY_RESTRICT entry, const uint32_t chunk) { + DENSITY_MEMMOVE((uint32_t *) entry + 1, entry, 3 * sizeof(uint32_t)); + *(uint32_t *) entry = chunk; // Does not ensure dictionary content consistency between endiannesses +} + +DENSITY_FORCE_INLINE void density_lion_decode_read_hash(const uint8_t **DENSITY_RESTRICT in, uint16_t *DENSITY_RESTRICT const hash) { +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(hash, *in, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + uint16_t endian_hash; + DENSITY_MEMCPY(&endian_hash, *in, sizeof(uint16_t)); + *hash = DENSITY_LITTLE_ENDIAN_16(endian_hash); +#else +#error +#endif + *in += sizeof(uint16_t); +} + +DENSITY_FORCE_INLINE void density_lion_decode_prediction_generic(uint8_t **DENSITY_RESTRICT out, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + *hash = DENSITY_LION_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)); + DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); + *out += sizeof(uint32_t); +} + +DENSITY_FORCE_INLINE void density_lion_decode_dictionary_generic(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT const unit) { + DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); + *out += sizeof(uint32_t); + density_lion_dictionary_chunk_prediction_entry *prediction = &(dictionary->predictions[*last_hash]); + density_lion_decode_update_predictions_model(prediction, *unit); +} + +void density_lion_decode_prediction_a(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + (void)in; + *unit = dictionary->predictions[*last_hash].next_chunk_a; + density_lion_decode_prediction_generic(out, hash, unit); + + *last_hash = *hash; +} + +void density_lion_decode_prediction_b(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + (void)in; + density_lion_dictionary_chunk_prediction_entry *const prediction = &dictionary->predictions[*last_hash]; + *unit = prediction->next_chunk_b; + density_lion_decode_update_predictions_model(prediction, *unit); + density_lion_decode_prediction_generic(out, hash, unit); + + *last_hash = *hash; +} + +void density_lion_decode_prediction_c(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + (void)in; + density_lion_dictionary_chunk_prediction_entry *const prediction = &dictionary->predictions[*last_hash]; + *unit = prediction->next_chunk_c; + density_lion_decode_update_predictions_model(prediction, *unit); + density_lion_decode_prediction_generic(out, hash, unit); + + *last_hash = *hash; +} + +void density_lion_decode_dictionary_a(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + density_lion_decode_read_hash(in, hash); + DENSITY_PREFETCH(&dictionary->predictions[*hash]); + *unit = dictionary->chunks[*hash].chunk_a; + density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); + + *last_hash = *hash; +} + +void density_lion_decode_dictionary_b(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + density_lion_decode_read_hash(in, hash); + DENSITY_PREFETCH(&dictionary->predictions[*hash]); + density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; + *unit = entry->chunk_b; + density_lion_decode_update_dictionary_model(entry, *unit); + density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); + + *last_hash = *hash; +} + +void density_lion_decode_dictionary_c(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + density_lion_decode_read_hash(in, hash); + DENSITY_PREFETCH(&dictionary->predictions[*hash]); + density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; + *unit = entry->chunk_c; + density_lion_decode_update_dictionary_model(entry, *unit); + density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); + + *last_hash = *hash; +} + +void density_lion_decode_dictionary_d(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + density_lion_decode_read_hash(in, hash); + DENSITY_PREFETCH(&dictionary->predictions[*hash]); + density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; + *unit = entry->chunk_d; + density_lion_decode_update_dictionary_model(entry, *unit); + density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); + + *last_hash = *hash; +} + +void density_lion_decode_plain(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { + DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); + *in += sizeof(uint32_t); + *hash = DENSITY_LION_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)); + density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; + density_lion_decode_update_dictionary_model(entry, *unit); + DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); + *out += sizeof(uint32_t); + density_lion_dictionary_chunk_prediction_entry *prediction = &(dictionary->predictions[*last_hash]); + density_lion_decode_update_predictions_model(prediction, *unit); + + *last_hash = *hash; +} + +DENSITY_FORCE_INLINE void density_lion_decode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, const DENSITY_LION_FORM form) { + uint16_t hash; + uint32_t unit; + + data->attachments[form](in, out, last_hash, dictionary, &hash, &unit); +} + +DENSITY_FORCE_INLINE DENSITY_LION_FORM density_lion_decode_read_form(const uint8_t **DENSITY_RESTRICT in, uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_form_data *const form_data) { + const uint_fast8_t trailing_zeroes = DENSITY_CTZ(0x80 | (*signature >> *shift)); + if (DENSITY_LIKELY(!trailing_zeroes)) { + *shift = (uint_fast8_t)((*shift + 1) & 0x3f); + return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool); + } else if (DENSITY_LIKELY(trailing_zeroes <= 6)) { + *shift = (uint_fast8_t)((*shift + (trailing_zeroes + 1)) & 0x3f); + return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool + trailing_zeroes); + } else { + if (DENSITY_LIKELY(*shift <= (density_bitsizeof(density_lion_signature) - 7))) { + *shift = (uint_fast8_t)((*shift + 7) & 0x3f); + return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool + 7); + } else { + density_lion_decode_read_signature(in, signature); + const uint_fast8_t primary_trailing_zeroes = (uint_fast8_t)(density_bitsizeof(density_lion_signature) - *shift); + const uint_fast8_t ctz_barrier_shift = (uint_fast8_t)(7 - primary_trailing_zeroes); + const uint_fast8_t secondary_trailing_zeroes = DENSITY_CTZ(((uint64_t)1 << ctz_barrier_shift) | *signature); + if (DENSITY_LIKELY(secondary_trailing_zeroes != ctz_barrier_shift)) + *shift = (uint_fast8_t)(secondary_trailing_zeroes + 1); + else + *shift = secondary_trailing_zeroes; + return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool + primary_trailing_zeroes + secondary_trailing_zeroes); + } + } +} + +DENSITY_FORCE_INLINE void density_lion_decode_process_form(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const form_data, uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift) { + if (DENSITY_UNLIKELY(!*shift)) + density_lion_decode_read_signature(in, signature); + + switch ((*signature >> *shift) & 0x1) { + case 0: + density_lion_decode_4(in, out, last_hash, dictionary, form_data, density_lion_decode_read_form(in, signature, shift, form_data)); + break; + default: + density_lion_decode_4(in, out, last_hash, dictionary, form_data, density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool)); + *shift = (uint_fast8_t)((*shift + 1) & 0x3f); + break; + } +} + +DENSITY_FORCE_INLINE void density_lion_decode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const form_data, uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift) { +#ifdef __clang__ + for (uint_fast8_t count = 0; count < (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG >> 2); count++) { + DENSITY_UNROLL_4(density_lion_decode_process_form(in, out, last_hash, dictionary, form_data, signature, shift)); + } +#else + for (uint_fast8_t count = 0; count < (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG >> 2); count++) { + DENSITY_UNROLL_4(density_lion_decode_process_form(in, out, last_hash, dictionary, form_data, signature, shift)); + } +#endif +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_lion_decode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { + if (out_size < DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + density_lion_signature signature = 0; + density_lion_form_data data; + density_lion_form_model_init(&data); + void (*attachments[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const) = {(void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_prediction_a, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_prediction_b, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_prediction_c, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_a, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_b, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_c, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_d, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_plain}; + density_lion_form_model_attach(&data, attachments); + uint_fast8_t shift = 0; + uint_fast64_t remaining; + uint_fast16_t last_hash = 0; + DENSITY_LION_FORM form; + + const uint8_t *start = *in; + + if (in_size < DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE) { + goto read_and_decode_4; + } + + const uint8_t *in_limit = *in + in_size - DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE; + uint8_t *out_limit = *out + out_size - DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE; + + while (DENSITY_LIKELY(*in <= in_limit && *out <= out_limit)) { + if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { + DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; + } + state->counter++; + if (DENSITY_UNLIKELY(state->copy_penalty)) { + DENSITY_ALGORITHM_COPY(DENSITY_LION_WORK_BLOCK_SIZE); + DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; + } else { + const uint8_t *in_start = *in; + density_lion_decode_256(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, &signature, &shift); + DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*in - in_start), DENSITY_LION_WORK_BLOCK_SIZE); + } + } + + if (*out > out_limit) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + read_and_decode_4: + if (DENSITY_UNLIKELY(!shift)) { + if (in_size - (*in - start) < sizeof(density_lion_signature)) + return DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL; + + density_lion_decode_read_signature(in, &signature); + } + form = density_lion_decode_read_form(in, &signature, &shift, &data); + switch (in_size - (*in - start)) { + case 0: + case 1: + switch (form) { + case DENSITY_LION_FORM_PLAIN: + goto process_remaining_bytes; // End marker + case DENSITY_LION_FORM_PREDICTIONS_A: + case DENSITY_LION_FORM_PREDICTIONS_B: + case DENSITY_LION_FORM_PREDICTIONS_C: + density_lion_decode_4(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, form); + break; + default: + return DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; // Not enough bytes to read a hash + } + break; + case 2: + case 3: + switch (form) { + case DENSITY_LION_FORM_PLAIN: + goto process_remaining_bytes; // End marker + default: + density_lion_decode_4(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, form); + break; + } + break; + default: + density_lion_decode_4(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, form); + break; + } + goto read_and_decode_4; + + process_remaining_bytes: + remaining = in_size - (*in - start); + DENSITY_MEMCPY(*out, *in, remaining); + *in += remaining; + *out += remaining; + + return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; +} diff --git a/contrib/density/src/algorithms/lion/core/lion_decode.h b/contrib/density/src/algorithms/lion/core/lion_decode.h new file mode 100644 index 00000000000..9972f0be388 --- /dev/null +++ b/contrib/density/src/algorithms/lion/core/lion_decode.h @@ -0,0 +1,54 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/06/15 20:55 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#ifndef DENSITY_LION_DECODE_H +#define DENSITY_LION_DECODE_H + +#include "../dictionary/lion_dictionary.h" +#include "../forms/lion_form_model.h" +#include "../../algorithms.h" + +DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_lion_decode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/algorithms/lion/core/lion_encode.c b/contrib/density/src/algorithms/lion/core/lion_encode.c new file mode 100644 index 00000000000..7cff2f3dda7 --- /dev/null +++ b/contrib/density/src/algorithms/lion/core/lion_encode.c @@ -0,0 +1,298 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/06/15 18:57 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#include "lion_encode.h" + +DENSITY_FORCE_INLINE void density_lion_encode_prepare_signature(uint8_t **DENSITY_RESTRICT out, uint_fast64_t **DENSITY_RESTRICT signature_pointer, uint_fast64_t *const DENSITY_RESTRICT signature) { + *signature = 0; + *signature_pointer = (density_lion_signature *) *out; + *out += sizeof(density_lion_signature); +} + +DENSITY_FORCE_INLINE void density_lion_encode_push_to_proximity_signature(uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const uint64_t content, const uint_fast8_t bits) { + *signature |= (content << *shift); + *shift += bits; +} + +DENSITY_FORCE_INLINE void density_lion_encode_push_to_signature(uint8_t **DENSITY_RESTRICT out, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const uint64_t content, const uint_fast8_t bits) { + if (DENSITY_LIKELY(*shift)) { + density_lion_encode_push_to_proximity_signature(signature, shift, content, bits); + + if (DENSITY_UNLIKELY(*shift >= density_bitsizeof(density_lion_signature))) { +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*signature_pointer, signature, sizeof(density_lion_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(*signature); + DENSITY_MEMCPY(*signature_pointer, &endian_signature, sizeof(density_lion_signature)); +#else +#error +#endif + + const uint_fast8_t remainder = (uint_fast8_t)(*shift & 0x3f); + *shift = 0; + if (remainder) { + density_lion_encode_prepare_signature(out, signature_pointer, signature); + density_lion_encode_push_to_proximity_signature(signature, shift, content >> (bits - remainder), remainder); + } + } + } else { + density_lion_encode_prepare_signature(out, signature_pointer, signature); + density_lion_encode_push_to_proximity_signature(signature, shift, content, bits); + } +} + +DENSITY_FORCE_INLINE void density_lion_encode_push_zero_to_signature(uint8_t **DENSITY_RESTRICT out, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const uint_fast8_t bits) { + if (DENSITY_LIKELY(*shift)) { + *shift += bits; + + if (DENSITY_UNLIKELY(*shift >= density_bitsizeof(density_lion_signature))) { +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*signature_pointer, signature, sizeof(density_lion_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(*signature); + DENSITY_MEMCPY(*signature_pointer, &endian_signature, sizeof(density_lion_signature)); +#else +#error +#endif + + const uint_fast8_t remainder = (uint_fast8_t)(*shift & 0x3f); + if (remainder) { + density_lion_encode_prepare_signature(out, signature_pointer, signature); + *shift = remainder; + } else + *shift = 0; + } + } else { + density_lion_encode_prepare_signature(out, signature_pointer, signature); + *shift = bits; + } +} + +DENSITY_FORCE_INLINE void density_lion_encode_push_code_to_signature(uint8_t **DENSITY_RESTRICT out, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const density_lion_entropy_code code) { + density_lion_encode_push_to_signature(out, signature_pointer, signature, shift, code.value, code.bitLength); +} + +DENSITY_FORCE_INLINE void density_lion_encode_kernel_4(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, const uint16_t hash, density_lion_form_data *const data, const uint32_t unit) { + density_lion_dictionary_chunk_prediction_entry *const predictions = &dictionary->predictions[*last_hash]; + DENSITY_PREFETCH(&dictionary->predictions[hash]); + + if (*(uint32_t *) predictions ^ unit) { + if (*((uint32_t *) predictions + 1) ^ unit) { + if (*((uint32_t *) predictions + 2) ^ unit) { + density_lion_dictionary_chunk_entry *const in_dictionary = &dictionary->chunks[hash]; + if (*(uint32_t *) in_dictionary ^ unit) { + if (*((uint32_t *) in_dictionary + 1) ^ unit) { + if (*((uint32_t *) in_dictionary + 2) ^ unit) { + if (*((uint32_t *) in_dictionary + 3) ^ unit) { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PLAIN)); + DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); + *out += sizeof(uint32_t); + } else { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_D)); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + } + } else { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_C)); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + } + } else { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_B)); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + } + DENSITY_MEMMOVE((uint32_t *) in_dictionary + 1, in_dictionary, 3 * sizeof(uint32_t)); + *(uint32_t *) in_dictionary = unit; // Does not ensure dictionary content consistency between endiannesses + } else { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_A)); +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); +#elif defined(DENSITY_BIG_ENDIAN) + const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); + DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); +#else +#error +#endif + *out += sizeof(uint16_t); + } + } else { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PREDICTIONS_C)); + } + } else { + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PREDICTIONS_B)); + } + DENSITY_MEMMOVE((uint32_t *) predictions + 1, predictions, 2 * sizeof(uint32_t)); + *(uint32_t *) predictions = unit; // Does not ensure dictionary content consistency between endiannesses + } else + density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PREDICTIONS_A)); + *last_hash = hash; +} + +DENSITY_FORCE_INLINE void density_lion_encode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { + DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); + density_lion_encode_kernel_4(out, last_hash, signature_pointer, signature, shift, dictionary, DENSITY_LION_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)), data, *unit); + *in += sizeof(uint32_t); +} + +DENSITY_FORCE_INLINE void density_lion_encode_generic(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, const uint_fast8_t chunks_per_process_unit, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { +#ifdef __clang__ + for (uint_fast8_t count = 0; count < (chunks_per_process_unit >> 2); count++) { + DENSITY_UNROLL_4(density_lion_encode_4(in, out, last_hash, signature_pointer, signature, shift, dictionary, data, unit)); + } +#else + for (uint_fast8_t count = 0; count < (chunks_per_process_unit >> 1); count++) { + DENSITY_UNROLL_2(density_lion_encode_4(in, out, last_hash, signature_pointer, signature, shift, dictionary, data, unit)); + } +#endif +} + +DENSITY_FORCE_INLINE void density_lion_encode_32(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { + density_lion_encode_generic(in, out, last_hash, signature_pointer, signature, shift, dictionary, DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_SMALL, data, unit); +} + +DENSITY_FORCE_INLINE void density_lion_encode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { + density_lion_encode_generic(in, out, last_hash, signature_pointer, signature, shift, dictionary, DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG, data, unit); +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_lion_encode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { + if (out_size < DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + density_lion_signature signature = 0; + density_lion_signature *signature_pointer = NULL; + uint_fast8_t shift = 0; + density_lion_form_data data; + density_lion_form_model_init(&data); + uint_fast16_t last_hash = 0; + uint32_t unit; + + uint8_t *out_limit = *out + out_size - DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE; + uint_fast64_t limit_256 = (in_size >> 8); + + while (DENSITY_LIKELY(limit_256-- && *out <= out_limit)) { + if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { + DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; + } + state->counter++; + if (DENSITY_UNLIKELY(state->copy_penalty)) { + DENSITY_ALGORITHM_COPY(DENSITY_LION_WORK_BLOCK_SIZE); + DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; + } else { + const uint8_t *out_start = *out; + DENSITY_PREFETCH(*in + DENSITY_LION_WORK_BLOCK_SIZE); + density_lion_encode_256(in, out, &last_hash, &signature_pointer, &signature, &shift, (density_lion_dictionary *const) state->dictionary, &data, &unit); + DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*out - out_start), DENSITY_LION_WORK_BLOCK_SIZE); + } + } + + if (*out > out_limit) + return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; + + uint_fast64_t remaining; + + switch (in_size & 0xff) { + case 0: + case 1: + case 2: + case 3: + density_lion_encode_push_code_to_signature(out, &signature_pointer, &signature, &shift, density_lion_form_model_get_encoding(&data, DENSITY_LION_FORM_PLAIN)); // End marker +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_lion_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_lion_signature)); +#else +#error +#endif + goto process_remaining_bytes; + default: + break; + } + + uint_fast64_t limit_4 = (in_size & 0xff) >> 2; + while (limit_4--) + density_lion_encode_4(in, out, &last_hash, &signature_pointer, &signature, &shift, (density_lion_dictionary *const) state->dictionary, &data, &unit); + + density_lion_encode_push_code_to_signature(out, &signature_pointer, &signature, &shift, density_lion_form_model_get_encoding(&data, DENSITY_LION_FORM_PLAIN)); // End marker +#ifdef DENSITY_LITTLE_ENDIAN + DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_lion_signature)); +#elif defined(DENSITY_BIG_ENDIAN) + const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); + DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_lion_signature)); +#else +#error +#endif + + process_remaining_bytes: + remaining = in_size & 0x3; + if (remaining) { + DENSITY_MEMCPY(*out, *in, remaining); + *in += remaining; + *out += remaining; + } + + return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; +} diff --git a/contrib/density/src/algorithms/lion/core/lion_encode.h b/contrib/density/src/algorithms/lion/core/lion_encode.h new file mode 100644 index 00000000000..71cd2cad4cf --- /dev/null +++ b/contrib/density/src/algorithms/lion/core/lion_encode.h @@ -0,0 +1,54 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 24/06/15 18:56 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#ifndef DENSITY_LION_ENCODE_H +#define DENSITY_LION_ENCODE_H + +#include "../dictionary/lion_dictionary.h" +#include "../forms/lion_form_model.h" +#include "../../algorithms.h" + +DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_lion_encode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h b/contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h new file mode 100644 index 00000000000..68f71da6373 --- /dev/null +++ b/contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h @@ -0,0 +1,73 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 12/02/15 23:09 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#ifndef DENSITY_LION_DICTIONARY_H +#define DENSITY_LION_DICTIONARY_H + +#include "../lion.h" + +#pragma pack(push) +#pragma pack(4) + +typedef struct { + uint32_t chunk_a; + uint32_t chunk_b; + uint32_t chunk_c; + uint32_t chunk_d; + uint32_t chunk_e; +} density_lion_dictionary_chunk_entry; + +typedef struct { + uint32_t next_chunk_a; + uint32_t next_chunk_b; + uint32_t next_chunk_c; +} density_lion_dictionary_chunk_prediction_entry; + +typedef struct { + density_lion_dictionary_chunk_entry chunks[1 << DENSITY_LION_CHUNK_HASH_BITS]; + density_lion_dictionary_chunk_prediction_entry predictions[1 << DENSITY_LION_CHUNK_HASH_BITS]; +} density_lion_dictionary; +#pragma pack(pop) + +#endif diff --git a/contrib/density/src/algorithms/lion/forms/lion_form_model.c b/contrib/density/src/algorithms/lion/forms/lion_form_model.c new file mode 100644 index 00000000000..21dfc23e3c9 --- /dev/null +++ b/contrib/density/src/algorithms/lion/forms/lion_form_model.c @@ -0,0 +1,153 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 9/03/15 11:19 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#include "lion_form_model.h" + +const density_lion_entropy_code density_lion_form_entropy_codes[DENSITY_LION_NUMBER_OF_FORMS] = DENSITY_LION_FORM_MODEL_ENTROPY_CODES; + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_lion_form_model_init(density_lion_form_data *const data) { + density_lion_form_node *rank_0 = &data->formsPool[0]; + rank_0->form = DENSITY_LION_FORM_PLAIN; + rank_0->rank = 0; + rank_0->previousForm = NULL; + data->formsIndex[DENSITY_LION_FORM_PLAIN] = rank_0; + + density_lion_form_node *rank_1 = &data->formsPool[1]; + rank_1->form = DENSITY_LION_FORM_DICTIONARY_A; + rank_1->rank = 1; + rank_1->previousForm = rank_0; + data->formsIndex[DENSITY_LION_FORM_DICTIONARY_A] = rank_1; + + density_lion_form_node *rank_2 = &data->formsPool[2]; + rank_2->form = DENSITY_LION_FORM_DICTIONARY_B; + rank_2->rank = 2; + rank_2->previousForm = rank_1; + data->formsIndex[DENSITY_LION_FORM_DICTIONARY_B] = rank_2; + + density_lion_form_node *rank_3 = &data->formsPool[3]; + rank_3->form = DENSITY_LION_FORM_PREDICTIONS_A; + rank_3->rank = 3; + rank_3->previousForm = rank_2; + data->formsIndex[DENSITY_LION_FORM_PREDICTIONS_A] = rank_3; + + density_lion_form_node *rank_4 = &data->formsPool[4]; + rank_4->form = DENSITY_LION_FORM_PREDICTIONS_B; + rank_4->rank = 4; + rank_4->previousForm = rank_3; + data->formsIndex[DENSITY_LION_FORM_PREDICTIONS_B] = rank_4; + + density_lion_form_node *rank_5 = &data->formsPool[5]; + rank_5->form = DENSITY_LION_FORM_DICTIONARY_C; + rank_5->rank = 5; + rank_5->previousForm = rank_4; + data->formsIndex[DENSITY_LION_FORM_DICTIONARY_C] = rank_5; + + density_lion_form_node *rank_6 = &data->formsPool[6]; + rank_6->form = DENSITY_LION_FORM_PREDICTIONS_C; + rank_6->rank = 6; + rank_6->previousForm = rank_5; + data->formsIndex[DENSITY_LION_FORM_PREDICTIONS_C] = rank_6; + + density_lion_form_node *rank_7 = &data->formsPool[7]; + rank_7->form = DENSITY_LION_FORM_DICTIONARY_D; + rank_7->rank = 7; + rank_7->previousForm = rank_6; + data->formsIndex[DENSITY_LION_FORM_DICTIONARY_D] = rank_7; + + data->usages.usages_as_uint64_t = 0; +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_lion_form_model_attach(density_lion_form_data *const data, void (*attachments[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) { + for(uint_fast8_t count = 0; count < DENSITY_LION_NUMBER_OF_FORMS; count ++) + data->attachments[count] = attachments[count]; +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_lion_form_model_update(density_lion_form_data *const DENSITY_RESTRICT data, density_lion_form_node *const DENSITY_RESTRICT form, const uint8_t usage, density_lion_form_node *const DENSITY_RESTRICT previous_form, const uint8_t previous_usage) { + if (DENSITY_UNLIKELY(previous_usage < usage)) { // Relative stability is assumed + const DENSITY_LION_FORM form_value = form->form; + const DENSITY_LION_FORM previous_form_value = previous_form->form; + + previous_form->form = form_value; + form->form = previous_form_value; + + data->formsIndex[form_value] = previous_form; + data->formsIndex[previous_form_value] = form; + } +} + +DENSITY_FORCE_INLINE void density_lion_form_model_flatten(density_lion_form_data *const data, const uint8_t usage) { + if (DENSITY_UNLIKELY(usage & 0x80)) + data->usages.usages_as_uint64_t = (data->usages.usages_as_uint64_t >> 1) & 0x7f7f7f7f7f7f7f7fllu; // Flatten usage values +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE DENSITY_LION_FORM density_lion_form_model_increment_usage(density_lion_form_data *const data, density_lion_form_node *const DENSITY_RESTRICT form) { + const DENSITY_LION_FORM form_value = form->form; + const uint8_t usage = ++data->usages.usages_as_uint8_t[form_value]; + + density_lion_form_node *const previous_form = form->previousForm; + + if (previous_form) + density_lion_form_model_update(data, form, usage, previous_form, data->usages.usages_as_uint8_t[previous_form->form]); + else + density_lion_form_model_flatten(data, usage); + + return form_value; +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_lion_entropy_code density_lion_form_model_get_encoding(density_lion_form_data *const data, const DENSITY_LION_FORM form) { + const uint8_t usage = ++data->usages.usages_as_uint8_t[form]; + + density_lion_form_node *const form_found = data->formsIndex[form]; + density_lion_form_node *const previous_form = form_found->previousForm; + + if (previous_form) { + density_lion_form_model_update(data, form_found, usage, previous_form, data->usages.usages_as_uint8_t[previous_form->form]); + + return density_lion_form_entropy_codes[form_found->rank]; + } else { + density_lion_form_model_flatten(data, usage); + + return density_lion_form_entropy_codes[0]; + } +} diff --git a/contrib/density/src/algorithms/lion/forms/lion_form_model.h b/contrib/density/src/algorithms/lion/forms/lion_form_model.h new file mode 100644 index 00000000000..90164d94c78 --- /dev/null +++ b/contrib/density/src/algorithms/lion/forms/lion_form_model.h @@ -0,0 +1,96 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 9/03/15 12:04 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#ifndef DENSITY_LION_FORM_MODEL_H +#define DENSITY_LION_FORM_MODEL_H + +#include "../../../globals.h" +#include "../lion.h" + +#define DENSITY_LION_NUMBER_OF_FORMS 8 + +// Unary codes (reversed) except the last one +#define DENSITY_LION_FORM_MODEL_ENTROPY_CODES {\ + {DENSITY_BINARY_TO_UINT(1), 1},\ + {DENSITY_BINARY_TO_UINT(10), 2},\ + {DENSITY_BINARY_TO_UINT(100), 3},\ + {DENSITY_BINARY_TO_UINT(1000), 4},\ + {DENSITY_BINARY_TO_UINT(10000), 5},\ + {DENSITY_BINARY_TO_UINT(100000), 6},\ + {DENSITY_BINARY_TO_UINT(1000000), 7},\ + {DENSITY_BINARY_TO_UINT(0000000), 7},\ +} + +#pragma pack(push) +#pragma pack(4) +typedef struct { + void* previousForm; + DENSITY_LION_FORM form; + uint8_t rank; +} density_lion_form_node; + +typedef struct { + union { + uint8_t usages_as_uint8_t[DENSITY_LION_NUMBER_OF_FORMS]; + uint64_t usages_as_uint64_t; + } usages; + + void (*attachments[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const); + density_lion_form_node formsPool[DENSITY_LION_NUMBER_OF_FORMS]; + density_lion_form_node *formsIndex[DENSITY_LION_NUMBER_OF_FORMS]; + uint8_t nextAvailableForm; +} density_lion_form_data; +#pragma pack(pop) + +DENSITY_WINDOWS_EXPORT void density_lion_form_model_init(density_lion_form_data *const); + +DENSITY_WINDOWS_EXPORT void density_lion_form_model_attach(density_lion_form_data *const, void (*[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)); + +DENSITY_WINDOWS_EXPORT void density_lion_form_model_update(density_lion_form_data *const DENSITY_RESTRICT_DECLARE, density_lion_form_node *const DENSITY_RESTRICT_DECLARE, const uint8_t, density_lion_form_node *const DENSITY_RESTRICT_DECLARE, const uint8_t); + +DENSITY_WINDOWS_EXPORT DENSITY_LION_FORM density_lion_form_model_increment_usage(density_lion_form_data *const, density_lion_form_node *const DENSITY_RESTRICT_DECLARE); + +DENSITY_WINDOWS_EXPORT density_lion_entropy_code density_lion_form_model_get_encoding(density_lion_form_data *const, const DENSITY_LION_FORM); + +#endif diff --git a/contrib/density/src/algorithms/lion/lion.h b/contrib/density/src/algorithms/lion/lion.h new file mode 100644 index 00000000000..30b40f8d92b --- /dev/null +++ b/contrib/density/src/algorithms/lion/lion.h @@ -0,0 +1,94 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 5/02/15 20:57 + * + * -------------- + * Lion algorithm + * -------------- + * + * Author(s) + * Guillaume Voirin (https://github.com/gpnuma) + * + * Description + * Multiform compression algorithm + */ + +#ifndef DENSITY_LION_H +#define DENSITY_LION_H + +#include "../../globals.h" + +#define DENSITY_LION_HASH32_MULTIPLIER (uint32_t)0x9D6EF916lu +#define DENSITY_LION_CHUNK_HASH_BITS 16 + +#define DENSITY_LION_HASH_ALGORITHM(value32) (uint16_t)(value32 * DENSITY_LION_HASH32_MULTIPLIER >> (32 - DENSITY_LION_CHUNK_HASH_BITS)) + +typedef enum { + DENSITY_LION_FORM_PREDICTIONS_A = 0, + DENSITY_LION_FORM_PREDICTIONS_B, + DENSITY_LION_FORM_PREDICTIONS_C, + DENSITY_LION_FORM_DICTIONARY_A, + DENSITY_LION_FORM_DICTIONARY_B, + DENSITY_LION_FORM_DICTIONARY_C, + DENSITY_LION_FORM_DICTIONARY_D, + DENSITY_LION_FORM_PLAIN, +} DENSITY_LION_FORM; + +typedef enum { + DENSITY_LION_PREDICTIONS_SIGNATURE_FLAG_A = 0x0, + DENSITY_LION_PREDICTIONS_SIGNATURE_FLAG_B = 0x1, +} DENSITY_LION_PREDICTIONS_SIGNATURE_FLAG; + +#pragma pack(push) +#pragma pack(4) +typedef struct { + uint_fast8_t value; + uint_fast8_t bitLength; +} density_lion_entropy_code; +#pragma pack(pop) + +typedef uint64_t density_lion_signature; + +#define DENSITY_LION_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE (density_bitsizeof(density_lion_signature) * sizeof(uint32_t)) // Plain writes +#define DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE (sizeof(density_lion_signature) + DENSITY_LION_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE) + +#define DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE (density_bitsizeof(density_lion_signature) * sizeof(uint32_t)) // Smallest form size times work unit size + +#define DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_SMALL 8 +#define DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG 64 +#define DENSITY_LION_PROCESS_UNIT_SIZE_SMALL (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_SMALL * sizeof(uint32_t)) +#define DENSITY_LION_PROCESS_UNIT_SIZE_BIG (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG * sizeof(uint32_t)) + +#define DENSITY_LION_WORK_BLOCK_SIZE 256 +#define DENSITY_LION_COPY_PENALTY 2 + +#endif diff --git a/contrib/density/src/buffers/buffer.c b/contrib/density/src/buffers/buffer.c new file mode 100644 index 00000000000..d3e9943c599 --- /dev/null +++ b/contrib/density/src/buffers/buffer.c @@ -0,0 +1,220 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 3/02/15 19:53 + */ + +#include "buffer.h" + +DENSITY_WINDOWS_EXPORT uint_fast64_t density_compress_safe_size(const uint_fast64_t input_size) { + const uint_fast64_t slack = DENSITY_MAX_3(DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE, DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE, DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE); + + // Chameleon longest output + uint_fast64_t chameleon_longest_output_size = 0; + chameleon_longest_output_size += sizeof(density_header); + chameleon_longest_output_size += sizeof(density_chameleon_signature) * (1 + (input_size >> (5 + 3))); // Signature space (1 bit <=> 4 bytes) + chameleon_longest_output_size += sizeof(density_chameleon_signature); // Eventual supplementary signature for end marker + chameleon_longest_output_size += input_size; // Everything encoded as plain data + + // Cheetah longest output + uint_fast64_t cheetah_longest_output_size = 0; + cheetah_longest_output_size += sizeof(density_header); + cheetah_longest_output_size += sizeof(density_cheetah_signature) * (1 + (input_size >> (4 + 3))); // Signature space (2 bits <=> 4 bytes) + cheetah_longest_output_size += sizeof(density_cheetah_signature); // Eventual supplementary signature for end marker + cheetah_longest_output_size += input_size; // Everything encoded as plain data + + // Lion longest output + uint_fast64_t lion_longest_output_size = 0; + lion_longest_output_size += sizeof(density_header); + lion_longest_output_size += sizeof(density_lion_signature) * (1 + ((input_size * 7) >> (5 + 3))); // Signature space (7 bits <=> 4 bytes), although this size is technically impossible + lion_longest_output_size += sizeof(density_lion_signature); // Eventual supplementary signature for end marker + lion_longest_output_size += input_size; // Everything encoded as plain data + + return DENSITY_MAX_3(chameleon_longest_output_size, cheetah_longest_output_size, lion_longest_output_size) + slack; +} + +DENSITY_WINDOWS_EXPORT uint_fast64_t density_decompress_safe_size(const uint_fast64_t expected_decompressed_output_size) { + const uint_fast64_t slack = DENSITY_MAX_3(DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE, DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE, DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE); + + return expected_decompressed_output_size + slack; +} + +DENSITY_FORCE_INLINE DENSITY_STATE density_convert_algorithm_exit_status(const density_algorithm_exit_status status) { + switch (status) { + case DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED: + return DENSITY_STATE_OK; + case DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL: + return DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL; + case DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL: + return DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL; + default: + return DENSITY_STATE_ERROR_DURING_PROCESSING; + } +} + +DENSITY_FORCE_INLINE density_processing_result density_make_result(const DENSITY_STATE state, const uint_fast64_t read, const uint_fast64_t written, density_context *const context) { + density_processing_result result; + result.state = state; + result.bytesRead = read; + result.bytesWritten = written; + result.context = context; + return result; +} + +DENSITY_FORCE_INLINE density_context* density_allocate_context(const DENSITY_ALGORITHM algorithm, const bool custom_dictionary, void *(*mem_alloc)(size_t)) { + density_context* context = mem_alloc(sizeof(density_context)); + context->algorithm = algorithm; + context->dictionary_size = density_get_dictionary_size(context->algorithm); + context->dictionary_type = custom_dictionary; + if(!context->dictionary_type) { + context->dictionary = mem_alloc(context->dictionary_size); + DENSITY_MEMSET(context->dictionary, 0, context->dictionary_size); + } + return context; +} + +DENSITY_WINDOWS_EXPORT void density_free_context(density_context *const context, void (*mem_free)(void *)) { + if(mem_free == NULL) + mem_free = free; + if(!context->dictionary_type) + mem_free(context->dictionary); + mem_free(context); +} + +DENSITY_WINDOWS_EXPORT density_processing_result density_compress_prepare_context(const DENSITY_ALGORITHM algorithm, const bool custom_dictionary, void *(*mem_alloc)(size_t)) { + if(mem_alloc == NULL) + mem_alloc = malloc; + + return density_make_result(DENSITY_STATE_OK, 0, 0, density_allocate_context(algorithm, custom_dictionary, mem_alloc)); +} + +DENSITY_WINDOWS_EXPORT density_processing_result density_compress_with_context(const uint8_t * input_buffer, const uint_fast64_t input_size, uint8_t * output_buffer, const uint_fast64_t output_size, density_context *const context) { + if (output_size < sizeof(density_header)) + return density_make_result(DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL, 0, 0, context); + if(context == NULL) + return density_make_result(DENSITY_STATE_ERROR_INVALID_CONTEXT, 0, 0, context); + + // Variables setup + const uint8_t *in = input_buffer; + uint8_t *out = output_buffer; + density_algorithm_state state; + density_algorithm_exit_status status = DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; + + // Header + density_header_write(&out, context->algorithm); + + // Compression + density_algorithms_prepare_state(&state, context->dictionary); + switch (context->algorithm) { + case DENSITY_ALGORITHM_CHAMELEON: + status = density_chameleon_encode(&state, &in, input_size, &out, output_size); + break; + case DENSITY_ALGORITHM_CHEETAH: + status = density_cheetah_encode(&state, &in, input_size, &out, output_size); + break; + case DENSITY_ALGORITHM_LION: + status = density_lion_encode(&state, &in, input_size, &out, output_size); + break; + } + + // Result + return density_make_result(density_convert_algorithm_exit_status(status), in - input_buffer, out - output_buffer, context); +} + +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_prepare_context(const uint8_t *input_buffer, const uint_fast64_t input_size, const bool custom_dictionary, void *(*mem_alloc)(size_t)) { + if (input_size < sizeof(density_header)) + return density_make_result(DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL, 0, 0, NULL); + + // Variables setup + const uint8_t* in = input_buffer; + if(mem_alloc == NULL) + mem_alloc = malloc; + + // Read header + density_header main_header; + density_header_read(&in, &main_header); + + // Setup context + density_context *const context = density_allocate_context(main_header.algorithm, custom_dictionary, mem_alloc); + return density_make_result(DENSITY_STATE_OK, in - input_buffer, 0, context); +} + +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_with_context(const uint8_t * input_buffer, const uint_fast64_t input_size, uint8_t * output_buffer, const uint_fast64_t output_size, density_context *const context) { + if(context == NULL) + return density_make_result(DENSITY_STATE_ERROR_INVALID_CONTEXT, 0, 0, context); + + // Variables setup + const uint8_t *in = input_buffer; + uint8_t *out = output_buffer; + density_algorithm_state state; + density_algorithm_exit_status status = DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; + + // Decompression + density_algorithms_prepare_state(&state, context->dictionary); + switch (context->algorithm) { + case DENSITY_ALGORITHM_CHAMELEON: + status = density_chameleon_decode(&state, &in, input_size, &out, output_size); + break; + case DENSITY_ALGORITHM_CHEETAH: + status = density_cheetah_decode(&state, &in, input_size, &out, output_size); + break; + case DENSITY_ALGORITHM_LION: + status = density_lion_decode(&state, &in, input_size, &out, output_size); + break; + } + + // Result + return density_make_result(density_convert_algorithm_exit_status(status), in - input_buffer, out - output_buffer, context); +} + +DENSITY_WINDOWS_EXPORT density_processing_result density_compress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, const DENSITY_ALGORITHM algorithm) { + density_processing_result result = density_compress_prepare_context(algorithm, false, malloc); + if(result.state) { + density_free_context(result.context, free); + return result; + } + + result = density_compress_with_context(input_buffer, input_size, output_buffer, output_size, result.context); + density_free_context(result.context, free); + return result; +} + +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size) { + density_processing_result result = density_decompress_prepare_context(input_buffer, input_size, false, malloc); + if(result.state) { + density_free_context(result.context, free); + return result; + } + + result = density_decompress_with_context(input_buffer + result.bytesRead, input_size - result.bytesRead, output_buffer, output_size, result.context); + density_free_context(result.context, free); + return result; +} diff --git a/contrib/density/src/buffers/buffer.h b/contrib/density/src/buffers/buffer.h new file mode 100644 index 00000000000..f34ab5458d2 --- /dev/null +++ b/contrib/density/src/buffers/buffer.h @@ -0,0 +1,58 @@ +/* + * Density + * + * Copyright (c) 2015, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 3/02/15 19:51 + */ + +#ifndef DENSITY_BUFFER_H +#define DENSITY_BUFFER_H + +#include "../globals.h" +#include "../density_api.h" +#include "../structure/header.h" +#include "../algorithms/chameleon/core/chameleon_encode.h" +#include "../algorithms/chameleon/core/chameleon_decode.h" +#include "../algorithms/cheetah/core/cheetah_encode.h" +#include "../algorithms/cheetah/core/cheetah_decode.h" +#include "../algorithms/lion/core/lion_encode.h" +#include "../algorithms/lion/core/lion_decode.h" + +DENSITY_WINDOWS_EXPORT uint_fast64_t density_compress_safe_size(const uint_fast64_t); +DENSITY_WINDOWS_EXPORT uint_fast64_t density_decompress_safe_size(const uint_fast64_t); +DENSITY_WINDOWS_EXPORT void density_free_context(density_context *const, void (*)(void *)); +DENSITY_WINDOWS_EXPORT density_processing_result density_compress_prepare_context(const DENSITY_ALGORITHM, const bool, void *(*)(size_t)); +DENSITY_WINDOWS_EXPORT density_processing_result density_compress_with_context(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t, density_context *const); +DENSITY_WINDOWS_EXPORT density_processing_result density_compress(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t, const DENSITY_ALGORITHM); +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_prepare_context(const uint8_t *, const uint_fast64_t, const bool, void *(*)(size_t)); +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_with_context(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t, density_context *const); +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t); + +#endif diff --git a/contrib/density/src/density_api.h b/contrib/density/src/density_api.h new file mode 100644 index 00000000000..385e24e609a --- /dev/null +++ b/contrib/density/src/density_api.h @@ -0,0 +1,220 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 18/10/13 22:41 + */ + +#ifndef DENSITY_API_H +#define DENSITY_API_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include +#include +#include + +#if defined(_WIN64) || defined(_WIN32) +#define DENSITY_WINDOWS_EXPORT __declspec(dllexport) +#else +#define DENSITY_WINDOWS_EXPORT +#endif + + +/*********************************************************************************************************************** + * * + * API data structures * + * * + ***********************************************************************************************************************/ + +typedef uint8_t density_byte; +typedef bool density_bool; + +typedef enum { + DENSITY_ALGORITHM_CHAMELEON = 1, + DENSITY_ALGORITHM_CHEETAH = 2, + DENSITY_ALGORITHM_LION = 3, +} DENSITY_ALGORITHM; + +typedef enum { + DENSITY_STATE_OK = 0, // Everything went alright + DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL, // Input buffer size is too small + DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL, // Output buffer size is too small + DENSITY_STATE_ERROR_DURING_PROCESSING, // Error during processing + DENSITY_STATE_ERROR_INVALID_CONTEXT, // Invalid context + DENSITY_STATE_ERROR_INVALID_ALGORITHM, // Invalid algorithm +} DENSITY_STATE; + +typedef struct { + DENSITY_ALGORITHM algorithm; + bool dictionary_type; + size_t dictionary_size; + void* dictionary; +} density_context; + +typedef struct { + DENSITY_STATE state; + uint_fast64_t bytesRead; + uint_fast64_t bytesWritten; + density_context* context; +} density_processing_result; + + + +/*********************************************************************************************************************** + * * + * Density version information * + * * + ***********************************************************************************************************************/ + +/* + * Returns the major density version + */ +DENSITY_WINDOWS_EXPORT uint8_t density_version_major(void); + +/* + * Returns the minor density version + */ +DENSITY_WINDOWS_EXPORT uint8_t density_version_minor(void); + +/* + * Returns the density revision + */ +DENSITY_WINDOWS_EXPORT uint8_t density_version_revision(void); + + + +/*********************************************************************************************************************** + * * + * Density API functions * + * * + ***********************************************************************************************************************/ + +/* + * Return the required size of an algorithm's dictionary + * + * @param algorithm the algorithm to use this dictionary for + */ +DENSITY_WINDOWS_EXPORT size_t density_get_dictionary_size(DENSITY_ALGORITHM algorithm); + +/* + * Return an output buffer byte size which guarantees enough space for encoding input_size bytes + * + * @param input_size the size of the input data which is about to be compressed + */ +DENSITY_WINDOWS_EXPORT uint_fast64_t density_compress_safe_size(const uint_fast64_t input_size); + +/* + * Return an output buffer byte size which, if expected_decompressed_output_size is correct, will enable density to decompress properly + * + * @param expected_decompressed_output_size the expected (original) size of the decompressed data + */ +DENSITY_WINDOWS_EXPORT uint_fast64_t density_decompress_safe_size(const uint_fast64_t expected_decompressed_output_size); + +/* + * Releases a context from memory. + * + * @param context the context to free + * @param mem_free the memory freeing function. If set to NULL, free() is used + */ +DENSITY_WINDOWS_EXPORT void density_free_context(density_context *const context, void (*mem_free)(void *)); + +/* + * Allocate a context in memory using the provided function and optional dictionary + * + * @param algorithm the required algorithm + * @param custom_dictionary use an eventual custom dictionary ? If set to true the context's dictionary will have to be allocated + * @param mem_alloc the memory allocation function. If set to NULL, malloc() is used + */ +DENSITY_WINDOWS_EXPORT density_processing_result density_compress_prepare_context(const DENSITY_ALGORITHM algorithm, const bool custom_dictionary, void *(*mem_alloc)(size_t)); + +/* + * Compress an input_buffer of input_size bytes and store the result in output_buffer, using the provided context. + * Important note * this function could be unsafe memory-wise if not used properly. + * + * @param input_buffer a buffer of bytes + * @param input_size the size in bytes of input_buffer + * @param output_buffer a buffer of bytes + * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE + * @param context a pointer to a context structure + */ +DENSITY_WINDOWS_EXPORT density_processing_result density_compress_with_context(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, density_context *const context); + +/* + * Compress an input_buffer of input_size bytes and store the result in output_buffer. + * + * @param input_buffer a buffer of bytes + * @param input_size the size in bytes of input_buffer + * @param output_buffer a buffer of bytes + * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE + * @param algorithm the algorithm to use + */ +DENSITY_WINDOWS_EXPORT density_processing_result density_compress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, const DENSITY_ALGORITHM algorithm); + +/* + * Reads the compressed data's header and creates an adequate decompression context. + * + * @param input_buffer a buffer of bytes + * @param input_size the size in bytes of input_buffer + * @param custom_dictionary use a custom dictionary ? If set to true the context's dictionary will have to be allocated + * @param mem_alloc the memory allocation function. If set to NULL, malloc() is used + */ +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_prepare_context(const uint8_t *input_buffer, const uint_fast64_t input_size, const bool custom_dictionary, void *(*mem_alloc)(size_t)); + +/* + * Decompress an input_buffer of input_size bytes and store the result in output_buffer, using the provided dictionary. + * Important notes * You must know in advance the algorithm used for compression to provide the proper dictionary. + * * This function could be unsafe memory-wise if not used properly. + * + * @param input_buffer a buffer of bytes + * @param input_size the size in bytes of input_buffer + * @param output_buffer a buffer of bytes + * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE + * @param dictionaries a pointer to a dictionary + */ +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_with_context(const uint8_t * input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, density_context *const context); + +/* + * Decompress an input_buffer of input_size bytes and store the result in output_buffer. + * + * @param input_buffer a buffer of bytes + * @param input_size the size in bytes of input_buffer + * @param output_buffer a buffer of bytes + * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE + */ +DENSITY_WINDOWS_EXPORT density_processing_result density_decompress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size); + +#ifdef __cplusplus +} +#endif + +#endif diff --git a/contrib/density/src/globals.c b/contrib/density/src/globals.c new file mode 100644 index 00000000000..4c58cd6f067 --- /dev/null +++ b/contrib/density/src/globals.c @@ -0,0 +1,47 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 01/11/13 13:39 + */ + +#include "globals.h" + +DENSITY_WINDOWS_EXPORT uint8_t density_version_major() { + return DENSITY_MAJOR_VERSION; +} + +DENSITY_WINDOWS_EXPORT uint8_t density_version_minor() { + return DENSITY_MINOR_VERSION; +} + +DENSITY_WINDOWS_EXPORT uint8_t density_version_revision() { + return DENSITY_REVISION; +} diff --git a/contrib/density/src/globals.h b/contrib/density/src/globals.h new file mode 100644 index 00000000000..eb44a52a673 --- /dev/null +++ b/contrib/density/src/globals.h @@ -0,0 +1,232 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 11/10/13 02:01 + */ + +#ifndef DENSITY_GLOBALS_H +#define DENSITY_GLOBALS_H + +#include +#include +#include +#include + +#include "density_api.h" + +#if defined(__clang__) || defined(__GNUC__) +#define DENSITY_FORCE_INLINE inline __attribute__((always_inline)) +#define DENSITY_RESTRICT restrict +#define DENSITY_RESTRICT_DECLARE +#define DENSITY_MEMCPY __builtin_memcpy +#define DENSITY_MEMMOVE __builtin_memmove +#define DENSITY_MEMSET __builtin_memset +#define DENSITY_LIKELY(x) __builtin_expect(!!(x), 1) +#define DENSITY_UNLIKELY(x) __builtin_expect(!!(x), 0) +#define DENSITY_PREFETCH(x) __builtin_prefetch(x) +#define DENSITY_CTZ(x) __builtin_ctz(x) + +#if defined(__BYTE_ORDER__) +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ +#define DENSITY_LITTLE_ENDIAN +#elif __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ +#define DENSITY_BIG_ENDIAN +#else +#error Unsupported endianness +#endif +#else +#error Unkwnown endianness +#endif + +#elif defined(_MSC_VER) +#include +#include + +#define DENSITY_FORCE_INLINE __forceinline +#define DENSITY_RESTRICT __restrict +#define DENSITY_RESTRICT_DECLARE __restrict +#define DENSITY_MEMCPY memcpy +#define DENSITY_MEMMOVE memmove +#define DENSITY_MEMSET memset +#define DENSITY_LIKELY(x) (x) +#define DENSITY_UNLIKELY(x) (x) +#define DENSITY_PREFETCH(x) ((void)(x)) + +DENSITY_FORCE_INLINE uint_fast8_t density_msvc_ctz(uint64_t value) { + unsigned long trailing_zero = 0; + if (_BitScanForward(&trailing_zero, (unsigned long)value)) + return (uint_fast8_t)trailing_zero; + else + return 0; +} +#define DENSITY_CTZ(x) density_msvc_ctz(x) + +#define DENSITY_LITTLE_ENDIAN // Little endian by default on Windows + +#else +#error Unsupported compiler +#endif + +#ifdef DENSITY_LITTLE_ENDIAN +#define DENSITY_LITTLE_ENDIAN_64(b) ((uint64_t)b) +#define DENSITY_LITTLE_ENDIAN_32(b) ((uint32_t)b) +#define DENSITY_LITTLE_ENDIAN_16(b) ((uint16_t)b) +#elif defined(DENSITY_BIG_ENDIAN) +#if __GNUC__ * 100 + __GNUC_MINOR__ >= 403 +#define DENSITY_LITTLE_ENDIAN_64(b) __builtin_bswap64(b) +#define DENSITY_LITTLE_ENDIAN_32(b) __builtin_bswap32(b) +#define DENSITY_LITTLE_ENDIAN_16(b) __builtin_bswap16(b) +#else +#warning Using bulk byte swap routines. Expect performance issues. +#define DENSITY_LITTLE_ENDIAN_64(b) ((((b) & 0xFF00000000000000ull) >> 56) | (((b) & 0x00FF000000000000ull) >> 40) | (((b) & 0x0000FF0000000000ull) >> 24) | (((b) & 0x000000FF00000000ull) >> 8) | (((b) & 0x00000000FF000000ull) << 8) | (((b) & 0x0000000000FF0000ull) << 24ull) | (((b) & 0x000000000000FF00ull) << 40) | (((b) & 0x00000000000000FFull) << 56)) +#define DENSITY_LITTLE_ENDIAN_32(b) ((((b) & 0xFF000000) >> 24) | (((b) & 0x00FF0000) >> 8) | (((b) & 0x0000FF00) << 8) | (((b) & 0x000000FF) << 24)) +#define DENSITY_LITTLE_ENDIAN_16(b) ((((b) & 0xFF00) >> 8) | (((b) & 0x00FF) << 8)) +#endif +#else +#error Unsupported endianness +#endif + +#define DENSITY_MAX_2(a, b) (((a)>(b))?(a):(b)) +#define DENSITY_MAX_3(a, b, c) (DENSITY_MAX_2(DENSITY_MAX_2(a, b), c)) + +#define DENSITY_FORMAT(v) 0##v##llu + +#define DENSITY_ISOLATE(b, p) ((DENSITY_FORMAT(b) / p) & 0x1) + +#define DENSITY_BINARY_TO_UINT(b) ((DENSITY_ISOLATE(b, 1llu) ? 0x1 : 0)\ + + (DENSITY_ISOLATE(b, 8llu) ? 0x2 : 0)\ + + (DENSITY_ISOLATE(b, 64llu) ? 0x4 : 0)\ + + (DENSITY_ISOLATE(b, 512llu) ? 0x8 : 0)\ + + (DENSITY_ISOLATE(b, 4096llu) ? 0x10 : 0)\ + + (DENSITY_ISOLATE(b, 32768llu) ? 0x20 : 0)\ + + (DENSITY_ISOLATE(b, 262144llu) ? 0x40 : 0)\ + + (DENSITY_ISOLATE(b, 2097152llu) ? 0x80 : 0)\ + + (DENSITY_ISOLATE(b, 16777216llu) ? 0x100 : 0)\ + + (DENSITY_ISOLATE(b, 134217728llu) ? 0x200 : 0)\ + + (DENSITY_ISOLATE(b, 1073741824llu) ? 0x400 : 0)\ + + (DENSITY_ISOLATE(b, 8589934592llu) ? 0x800 : 0)\ + + (DENSITY_ISOLATE(b, 68719476736llu) ? 0x1000 : 0)\ + + (DENSITY_ISOLATE(b, 549755813888llu) ? 0x2000 : 0)\ + + (DENSITY_ISOLATE(b, 4398046511104llu) ? 0x4000 : 0)\ + + (DENSITY_ISOLATE(b, 35184372088832llu) ? 0x8000 : 0)\ + + (DENSITY_ISOLATE(b, 281474976710656llu) ? 0x10000 : 0)\ + + (DENSITY_ISOLATE(b, 2251799813685248llu) ? 0x20000 : 0)) + +#define DENSITY_UNROLL_2(op) op; op +#define DENSITY_UNROLL_4(op) DENSITY_UNROLL_2(op); DENSITY_UNROLL_2(op) +#define DENSITY_UNROLL_8(op) DENSITY_UNROLL_4(op); DENSITY_UNROLL_4(op) +#define DENSITY_UNROLL_16(op) DENSITY_UNROLL_8(op); DENSITY_UNROLL_8(op) +#define DENSITY_UNROLL_32(op) DENSITY_UNROLL_16(op); DENSITY_UNROLL_16(op) +#define DENSITY_UNROLL_64(op) DENSITY_UNROLL_32(op); DENSITY_UNROLL_32(op) + +#define DENSITY_CASE_GENERATOR_2(op_a, flag_a, op_b, flag_b, op_mid, shift)\ + case ((flag_b << shift) | flag_a):\ + op_a;\ + op_mid;\ + op_b;\ + break; + +#define DENSITY_CASE_GENERATOR_4(op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + case ((flag_d << (shift * 3)) | (flag_c << (shift * 2)) | (flag_b << shift) | flag_a):\ + op_a;\ + op_mid;\ + op_b;\ + op_mid;\ + op_c;\ + op_mid;\ + op_d;\ + break; + +#define DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_1, flag_1, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_a, flag_a, op_mid, shift);\ + DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_b, flag_b, op_mid, shift);\ + DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_c, flag_c, op_mid, shift);\ + DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_d, flag_d, op_mid, shift); + +#define DENSITY_CASE_GENERATOR_4_2_COMBINED(op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); + +#define DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_a, flag_a, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_b, flag_b, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_c, flag_c, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_d, flag_d, op_mid, shift); + +#define DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_2, flag_2, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); + +#define DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_1, flag_1, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); + +#define DENSITY_CASE_GENERATOR_4_4_COMBINED(op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ + DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ + DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); + +#define DENSITY_DICTIONARY_PREFERRED_RESET_CYCLE_SHIFT 6 +#define DENSITY_DICTIONARY_PREFERRED_RESET_CYCLE (1 << DENSITY_DICTIONARY_PREFERRED_RESET_CYCLE_SHIFT) + + +#define density_bitsizeof(x) (8 * sizeof(x)) + +#define DENSITY_SPOOKYHASH_SEED_1 (0xabc) +#define DENSITY_SPOOKYHASH_SEED_2 (0xdef) + +DENSITY_WINDOWS_EXPORT uint8_t density_version_major(); + +DENSITY_WINDOWS_EXPORT uint8_t density_version_minor(); + +DENSITY_WINDOWS_EXPORT uint8_t density_version_revision(); + + +/********************************************************************************************************************** + * * + * Global compile-time switches * + * * + **********************************************************************************************************************/ + +#define DENSITY_MAJOR_VERSION 0 +#define DENSITY_MINOR_VERSION 14 +#define DENSITY_REVISION 2 + + + +#endif diff --git a/contrib/density/src/structure/header.c b/contrib/density/src/structure/header.c new file mode 100644 index 00000000000..4306a5962cf --- /dev/null +++ b/contrib/density/src/structure/header.c @@ -0,0 +1,57 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 11/10/13 17:56 + */ + +#include "header.h" + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_header_read(const uint8_t **DENSITY_RESTRICT in, density_header *DENSITY_RESTRICT header) { + header->version[0] = *(*in); + header->version[1] = *(*in + 1); + header->version[2] = *(*in + 2); + header->algorithm = *(*in + 3); + + *in += sizeof(density_header); +} + +DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_header_write(uint8_t **DENSITY_RESTRICT out, const DENSITY_ALGORITHM algorithm) { + *(*out) = DENSITY_MAJOR_VERSION; + *(*out + 1) = DENSITY_MINOR_VERSION; + *(*out + 2) = DENSITY_REVISION; + *(*out + 3) = algorithm; + *(*out + 4) = 0; + *(*out + 5) = 0; + *(*out + 6) = 0; + *(*out + 7) = 0; + + *out += sizeof(density_header); +} diff --git a/contrib/density/src/structure/header.h b/contrib/density/src/structure/header.h new file mode 100644 index 00000000000..d4065b5600c --- /dev/null +++ b/contrib/density/src/structure/header.h @@ -0,0 +1,58 @@ +/* + * Density + * + * Copyright (c) 2013, Guillaume Voirin + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * 3. Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + * 11/10/13 02:06 + */ + +#ifndef DENSITY_FILE_HEADER_H +#define DENSITY_FILE_HEADER_H + +#include +#include + +#include "../globals.h" +#include "../density_api.h" + +#pragma pack(push) +#pragma pack(4) + +typedef struct { + density_byte version[3]; + density_byte algorithm; + density_byte reserved[4]; +} density_header; + +#pragma pack(pop) + +DENSITY_WINDOWS_EXPORT void density_header_read(const uint8_t ** DENSITY_RESTRICT_DECLARE, density_header * DENSITY_RESTRICT_DECLARE); +DENSITY_WINDOWS_EXPORT void density_header_write(uint8_t ** DENSITY_RESTRICT_DECLARE, const DENSITY_ALGORITHM); + +#endif diff --git a/contrib/lizard/.gitattributes b/contrib/lizard/.gitattributes new file mode 100644 index 00000000000..6212bd405b4 --- /dev/null +++ b/contrib/lizard/.gitattributes @@ -0,0 +1,21 @@ +# Set the default behavior +* text eol=lf + +# Explicitly declare source files +*.c text eol=lf +*.h text eol=lf + +# Denote files that should not be modified. +*.odt binary +*.png binary + +# Visual Studio +*.sln text eol=crlf +*.vcxproj* text eol=crlf +*.vcproj* text eol=crlf +*.suo binary +*.rc text eol=crlf + +# Windows +*.bat text eol=crlf +*.cmd text eol=crlf diff --git a/contrib/lizard/.gitignore b/contrib/lizard/.gitignore new file mode 100644 index 00000000000..dc30b0e5504 --- /dev/null +++ b/contrib/lizard/.gitignore @@ -0,0 +1,37 @@ +# Object files +*.o +*.ko + +# Libraries +*.lib +*.a + +# Shared objects (inc. Windows DLLs) +*.dll +*.so +*.so.* +*.dylib + +# Executables +*.exe +*.out +*.app + +# IDE / editors files +*.suo +*.user +.clang_complete + +# Directories +_lizardbench/ +_visual/ +_codelite/ +_backup/ +_codelite_lz4/ + +# Archives +*.zip +*.liz + +*.txt +*.bat \ No newline at end of file diff --git a/contrib/lizard/.travis.yml b/contrib/lizard/.travis.yml new file mode 100644 index 00000000000..29f75591a31 --- /dev/null +++ b/contrib/lizard/.travis.yml @@ -0,0 +1,268 @@ +language: c +matrix: + fast_finish: true + include: + - name: macOS + os: osx + env: Ubu=OS_X_Mavericks Cmd='make -C tests test-lizard CC=clang MOREFLAGS="-Werror -Wconversion -Wno-sign-conversion"' COMPILER=clang + + + # Ubuntu 18.04 + - name: Ubuntu 18.04, gcc-9, 64-bit and 32-bit tests + dist: bionic + script: + - CC=gcc-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-9; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + before_install: + - sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test + - sudo apt-get -q update + - sudo apt-get --no-install-suggests --no-install-recommends -yq install gcc-multilib gcc-9 gcc-9-multilib + + - name: Ubuntu 18.04, clang-9, 64-bit tests + dist: bionic + script: + - CC=clang-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - sourceline: 'deb https://apt.llvm.org/bionic/ llvm-toolchain-bionic-9 main' + key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' + packages: + - clang-9 + + + + # Ubuntu 16.04 gcc + - name: Ubuntu 16.04, gcc-9, 64-bit and 32-bit tests + dist: xenial + script: + - CC=gcc-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-9; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-multilib + - gcc-9 + - gcc-9-multilib + + - name: Ubuntu 16.04, gcc-8, 64-bit and 32-bit tests + dist: xenial + script: + - CC=gcc-8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-8; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-multilib + - gcc-8 + - gcc-8-multilib + + - name: Ubuntu 16.04, gcc-7, 64-bit and 32-bit tests + dist: xenial + script: + - CC=gcc-7; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-7; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-multilib + - gcc-7 + - gcc-7-multilib + + - name: Ubuntu 16.04, gcc-6, 64-bit and 32-bit tests + dist: xenial + script: + - CC=gcc-6; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-6; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-multilib + - gcc-6 + - gcc-6-multilib + + - name: Ubuntu 16.04, gcc-5, 64-bit and 32-bit tests + dist: xenial + script: + - CC=gcc-5; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-5; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + packages: + - gcc-multilib + - gcc-5 + - gcc-5-multilib + + - name: Ubuntu 16.04, gcc-4.8, 64-bit and 32-bit tests + dist: xenial + script: + - CC=gcc-4.8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + - CC=gcc-4.8; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror + addons: + apt: + packages: + - gcc-multilib + - gcc-4.8 + - gcc-4.8-multilib + + + + # Ubuntu 16.04 clang + - name: Ubuntu 16.04, clang-9, 64-bit tests + dist: xenial + script: + - CC=clang-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - sourceline: 'deb https://apt.llvm.org/xenial/ llvm-toolchain-xenial-9 main' + key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' + packages: + - clang-9 + + - name: Ubuntu 16.04, clang-8, 64-bit tests + dist: xenial + script: + - CC=clang-8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-xenial-8 + packages: + - clang-8 + + - name: Ubuntu 16.04, clang-7, 64-bit tests + dist: xenial + script: + - CC=clang-7; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-xenial-7 + packages: + - clang-7 + + - name: Ubuntu 16.04, clang-6.0, 64-bit tests + dist: xenial + script: + - CC=clang-6.0; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-xenial-6.0 + packages: + - clang-6.0 + + - name: Ubuntu 16.04, clang-5.0, 64-bit tests + dist: xenial + script: + - CC=clang-5.0; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-xenial-5.0 + packages: + - clang-5.0 + + - name: Ubuntu 16.04, clang-4.0, 64-bit tests + dist: xenial + script: + - CC=clang-4.0; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-xenial-4.0 + packages: + - clang-4.0 + + - name: Ubuntu 14.04, clang-3.8, 64-bit tests + dist: trusty + script: + - CC=clang-3.8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - llvm-toolchain-trusty-3.8 + packages: + - clang-3.8 + + + + - name: Ubuntu 16.04, misc tests + env: Cmd="make c_standards && make gpptest-native && make clean && make examples && make clean && make cmake && make clean && make travis-install && make clean && make clangtest-native" COMPILER=cc + dist: xenial + + - name: Ubuntu 14.04, arm and aarch64 + env: Cmd='make platformTest CC=arm-linux-gnueabi-gcc QEMU_SYS=qemu-arm-static && make platformTest CC=aarch64-linux-gnu-gcc QEMU_SYS=qemu-aarch64-static' COMPILER=arm-linux-gnueabi-gcc + dist: trusty + sudo: required + addons: + apt: + packages: + - qemu-system-arm + - qemu-user-static + - gcc-arm-linux-gnueabi + - libc6-dev-armel-cross + - gcc-aarch64-linux-gnu + - libc6-dev-arm64-cross + + - name: Ubuntu 14.04, powerpc and ppc64 + env: Cmd='make platformTest CC=powerpc-linux-gnu-gcc QEMU_SYS=qemu-ppc-static && make platformTest CC=powerpc-linux-gnu-gcc QEMU_SYS=qemu-ppc64-static MOREFLAGS=-m64' COMPILER=powerpc-linux-gnu-gcc + dist: trusty + sudo: required + addons: + apt: + packages: + - qemu-system-ppc + - qemu-user-static + - gcc-powerpc-linux-gnu + + - name: Ubuntu 14.04, valgrind + env: Cmd='make -C tests test-mem MOREFLAGS=-DLIZARD_RESET_MEM' COMPILER=cc + dist: trusty + sudo: required + addons: + apt: + packages: + - valgrind + + + #- env: Ubu=14.04 Cmd='make sanitize' COMPILER=clang + # dist: trusty + # sudo: required + # addons: + # apt: + # packages: + # - valgrind + + #- env: Ubu=14.04 Cmd='make staticAnalyze' COMPILER=clang + # dist: trusty + # sudo: required + # addons: + # apt: + # packages: + # - clang + + +script: + - echo Cmd=$Cmd + - $COMPILER -v + - sh -c "$Cmd" diff --git a/contrib/lizard/LICENSE b/contrib/lizard/LICENSE new file mode 100644 index 00000000000..df1edbca85e --- /dev/null +++ b/contrib/lizard/LICENSE @@ -0,0 +1,15 @@ +This repository uses 2 different licenses : +- all files in the `lib` directory use a BSD 2-Clause license +- all other files use a GPLv2 license, unless explicitly stated otherwise + +Relevant license is reminded at the top of each source file, +and with the presence of COPYING or LICENSE file. + +This model emphasizes the fact that +only files in the `lib` directory are designed to be included into 3rd party projects. + +Other files, such as those from `programs` or `examples` directory, +are not intended to be compiled outside of their context. +They can serve as source of inspiration, +but they should not be copy/pasted into 3rd party projects, +as this scenario is not supported. diff --git a/contrib/lizard/NEWS b/contrib/lizard/NEWS new file mode 100644 index 00000000000..05d424656e9 --- /dev/null +++ b/contrib/lizard/NEWS @@ -0,0 +1,41 @@ +Lizard v1.0 +- LZ5 v2.0 was renamed to Lizard v1.0 +- improved compression speed with many small files + +LZ5 v2.0 +- this version is optimized for decompression speed (instead of ratio for previous versions) +- LZ5 v2.0 contains 4 compression methods: + fastLZ4 : compression levels -10...-19 are designed to give better decompression speed than LZ4 i.e. over 2000 MB/s + LIZv1 : compression levels -20...-29 are designed to give better ratio than LZ4 keeping 75% decompression speed + fastLZ4 + Huffman : compression levels -30...-39 add Huffman coding to fastLZ4 + LIZv1 + Huffman : compression levels -40...-49 give the best ratio (comparable to zlib and low levels of zstd/brotli) at decompression speed of 1000 MB/s + +LZ5 v1.5 +- introduced compatibility with Visual C++ 2010 and newer +- attached Visual Studio 2010 project +- thoroughly tested with 21 Travis CI and 7 AppVeyor CI tests +- fixed bug with reusing a context in lizard_frame.c (LizardF_compressBegin and Lizard_compress_HC_continue) +- fixed rare bug in match finder (concerns levels 4 - 15) + +LZ5 v1.4.1 +- fixed bug with a backward match extension (level 11 and 12) + +LZ5 v1.4 +- improved: levels from 13 to 15 (maximum compression ratio) +- added a new parser: LizardHC_optimal_price_bt +- updated documentation: lizard_Block_format.md and lizard_Frame_format.md +- changed lizard.exe: the "-B" option with block size [1-7] = 64KB, 256KB, 1MB, 4MB, 16MB, 64MB, 256MB (default : 4 = 4MB) + +LZ5 v1.3.3 +- added: new levels from 11 to 18 (maximum compression ratio) +- added: a new parser: LizardHC_optimal_price +- fixed: buffer-overflow during decompression (thanks to m^2) + +LZ5 r132 +- improved compression ratio +- added: new parsers: LizardHC_fast, LizardHC_price_fast, LizardHC_lowest_price +- added: a special 1-byte codeword for the last occured offset +- added: support for 3-byte long matches (MINMATCH = 3) + +LZ5 r131 +The first release based on LZ4 r132 dev diff --git a/contrib/lizard/README.md b/contrib/lizard/README.md new file mode 100644 index 00000000000..50dd737fb8d --- /dev/null +++ b/contrib/lizard/README.md @@ -0,0 +1,95 @@ +Lizard - efficient compression with very fast decompression +-------------------------------------------------------- + +Lizard (formerly LZ5) is a lossless compression algorithm which contains 4 compression methods: +- fastLZ4 : compression levels -10...-19 are designed to give better decompression speed than [LZ4] i.e. over 2000 MB/s +- LIZv1 : compression levels -20...-29 are designed to give better ratio than [LZ4] keeping 75% decompression speed +- fastLZ4 + Huffman : compression levels -30...-39 add Huffman coding to fastLZ4 +- LIZv1 + Huffman : compression levels -40...-49 give the best ratio (comparable to [zlib] and low levels of [zstd]/[brotli]) at decompression speed of 1000 MB/s + +Lizard library is based on frequently used [LZ4] library by Yann Collet but the Lizard compression format is not compatible with LZ4. +Lizard library is provided as open-source software using BSD 2-Clause license. +The high compression/decompression speed is achieved without any SSE and AVX extensions. + + +|Branch |Status | +|------------|---------| +|lz5_v1.5 | [![Build Status][travis15Badge]][travisLink] [![Build status][Appveyor15Badge]][AppveyorLink] | +|lizard | [![Build Status][travis20Badge]][travisLink] [![Build status][Appveyor20Badge]][AppveyorLink] | + +[travis15Badge]: https://travis-ci.org/inikep/lizard.svg?branch=lz5_v1.5 "Continuous Integration test suite" +[travis20Badge]: https://travis-ci.org/inikep/lizard.svg?branch=lizard "Continuous Integration test suite" +[travisLink]: https://travis-ci.org/inikep/lizard +[Appveyor15Badge]: https://ci.appveyor.com/api/projects/status/cqw7emcuqge369p0/branch/lz5_v1.5?svg=true "Visual test suite" +[Appveyor20Badge]: https://ci.appveyor.com/api/projects/status/cqw7emcuqge369p0/branch/lizard?svg=true "Visual test suite" +[AppveyorLink]: https://ci.appveyor.com/project/inikep/lizard +[LZ4]: https://github.com/lz4/lz4 +[zlib]: https://github.com/madler/zlib +[zstd]: https://github.com/facebook/zstd +[brotli]: https://github.com/google/brotli + + +Benchmarks +------------------------- + +The following results are obtained with [lzbench](https://github.com/inikep/lzbench) and `-t16,16` +using 1 core of Intel Core i5-4300U, Windows 10 64-bit (MinGW-w64 compilation under gcc 6.2.0) +with [silesia.tar] which contains tarred files from [Silesia compression corpus](http://sun.aei.polsl.pl/~sdeor/index.php?page=silesia). + +| Compressor name | Compression| Decompress.| Compr. size | Ratio | +| --------------- | -----------| -----------| ----------- | ----- | +| memcpy | 7332 MB/s | 8719 MB/s | 211947520 |100.00 | +| lz4 1.7.3 | 440 MB/s | 2318 MB/s | 100880800 | 47.60 | +| lz4hc 1.7.3 -1 | 98 MB/s | 2121 MB/s | 87591763 | 41.33 | +| lz4hc 1.7.3 -4 | 55 MB/s | 2259 MB/s | 79807909 | 37.65 | +| lz4hc 1.7.3 -9 | 22 MB/s | 2315 MB/s | 77892285 | 36.75 | +| lz4hc 1.7.3 -12 | 17 MB/s | 2323 MB/s | 77849762 | 36.73 | +| lz4hc 1.7.3 -16 | 10 MB/s | 2323 MB/s | 77841782 | 36.73 | +| lizard 1.0 -10 | 346 MB/s | 2610 MB/s | 103402971 | 48.79 | +| lizard 1.0 -12 | 103 MB/s | 2458 MB/s | 86232422 | 40.69 | +| lizard 1.0 -15 | 50 MB/s | 2552 MB/s | 81187330 | 38.31 | +| lizard 1.0 -19 | 3.04 MB/s | 2497 MB/s | 77416400 | 36.53 | +| lizard 1.0 -21 | 157 MB/s | 1795 MB/s | 89239174 | 42.10 | +| lizard 1.0 -23 | 30 MB/s | 1778 MB/s | 81097176 | 38.26 | +| lizard 1.0 -26 | 6.63 MB/s | 1734 MB/s | 74503695 | 35.15 | +| lizard 1.0 -29 | 1.37 MB/s | 1634 MB/s | 68694227 | 32.41 | +| lizard 1.0 -30 | 246 MB/s | 909 MB/s | 85727429 | 40.45 | +| lizard 1.0 -32 | 94 MB/s | 1244 MB/s | 76929454 | 36.30 | +| lizard 1.0 -35 | 47 MB/s | 1435 MB/s | 73850400 | 34.84 | +| lizard 1.0 -39 | 2.94 MB/s | 1502 MB/s | 69807522 | 32.94 | +| lizard 1.0 -41 | 126 MB/s | 961 MB/s | 76100661 | 35.91 | +| lizard 1.0 -43 | 28 MB/s | 1101 MB/s | 70955653 | 33.48 | +| lizard 1.0 -46 | 6.25 MB/s | 1073 MB/s | 65413061 | 30.86 | +| lizard 1.0 -49 | 1.27 MB/s | 1064 MB/s | 60679215 | 28.63 | +| zlib 1.2.8 -1 | 66 MB/s | 244 MB/s | 77259029 | 36.45 | +| zlib 1.2.8 -6 | 20 MB/s | 263 MB/s | 68228431 | 32.19 | +| zlib 1.2.8 -9 | 8.37 MB/s | 266 MB/s | 67644548 | 31.92 | +| zstd 1.1.1 -1 | 235 MB/s | 645 MB/s | 73659468 | 34.75 | +| zstd 1.1.1 -2 | 181 MB/s | 600 MB/s | 70168955 | 33.11 | +| zstd 1.1.1 -5 | 88 MB/s | 565 MB/s | 65002208 | 30.67 | +| zstd 1.1.1 -8 | 31 MB/s | 619 MB/s | 61026497 | 28.79 | +| zstd 1.1.1 -11 | 16 MB/s | 613 MB/s | 59523167 | 28.08 | +| zstd 1.1.1 -15 | 4.97 MB/s | 639 MB/s | 58007773 | 27.37 | +| zstd 1.1.1 -18 | 2.87 MB/s | 583 MB/s | 55294241 | 26.09 | +| zstd 1.1.1 -22 | 1.44 MB/s | 505 MB/s | 52731930 | 24.88 | +| brotli 0.5.2 -0 | 217 MB/s | 244 MB/s | 78226979 | 36.91 | +| brotli 0.5.2 -2 | 96 MB/s | 283 MB/s | 68066621 | 32.11 | +| brotli 0.5.2 -5 | 24 MB/s | 312 MB/s | 60801716 | 28.69 | +| brotli 0.5.2 -8 | 5.56 MB/s | 324 MB/s | 57382470 | 27.07 | +| brotli 0.5.2 -11 | 0.39 MB/s | 266 MB/s | 51138054 | 24.13 | + +[silesia.tar]: https://drive.google.com/file/d/0BwX7dtyRLxThenZpYU9zLTZhR1k/view?usp=sharing + + +Documentation +------------------------- + +The raw Lizard block compression format is detailed within [lizard_Block_format]. + +To compress an arbitrarily long file or data stream, multiple blocks are required. +Organizing these blocks and providing a common header format to handle their content +is the purpose of the Frame format, defined into [lizard_Frame_format]. +Interoperable versions of Lizard must respect this frame format. + +[lizard_Block_format]: doc/lizard_Block_format.md +[lizard_Frame_format]: doc/lizard_Frame_format.md diff --git a/contrib/lizard/appveyor.yml b/contrib/lizard/appveyor.yml new file mode 100644 index 00000000000..d5863f7c06b --- /dev/null +++ b/contrib/lizard/appveyor.yml @@ -0,0 +1,143 @@ +version: 1.0.{build} +environment: + matrix: + - COMPILER: "visual" + CONFIGURATION: "Debug" + PLATFORM: "x64" + - COMPILER: "visual" + CONFIGURATION: "Debug" + PLATFORM: "Win32" + - COMPILER: "visual" + CONFIGURATION: "Release" + PLATFORM: "x64" + - COMPILER: "visual" + CONFIGURATION: "Release" + PLATFORM: "Win32" + - COMPILER: "gcc" + PLATFORM: "mingw64" + - COMPILER: "gcc" + PLATFORM: "mingw32" + - COMPILER: "gcc" + PLATFORM: "clang" + +install: + - ECHO Installing %COMPILER% %PLATFORM% %CONFIGURATION% + - MKDIR bin + - if [%COMPILER%]==[gcc] SET PATH_ORIGINAL=%PATH% + - if [%COMPILER%]==[gcc] ( + SET "PATH_MINGW32=c:\MinGW\bin;c:\MinGW\usr\bin" && + SET "PATH_MINGW64=c:\msys64\mingw64\bin;c:\msys64\usr\bin" && + COPY C:\MinGW\bin\mingw32-make.exe C:\MinGW\bin\make.exe && + COPY C:\MinGW\bin\gcc.exe C:\MinGW\bin\cc.exe + ) else ( + IF [%PLATFORM%]==[x64] (SET ADDITIONALPARAM=/p:LibraryPath="C:\Program Files\Microsoft SDKs\Windows\v7.1\lib\x64;c:\Program Files (x86)\Microsoft Visual Studio 10.0\VC\lib\amd64;C:\Program Files (x86)\Microsoft Visual Studio 10.0\;C:\Program Files (x86)\Microsoft Visual Studio 10.0\lib\amd64;") + ) + +build_script: + - if [%PLATFORM%]==[mingw32] SET PATH=%PATH_MINGW32%;%PATH_ORIGINAL% + - if [%PLATFORM%]==[mingw64] SET PATH=%PATH_MINGW64%;%PATH_ORIGINAL% + - if [%PLATFORM%]==[clang] SET PATH=%PATH_MINGW64%;%PATH_ORIGINAL% + - ECHO *** && + ECHO Building %COMPILER% %PLATFORM% %CONFIGURATION% && + ECHO *** + - if [%PLATFORM%]==[clang] (clang -v) + - if [%COMPILER%]==[gcc] (gcc -v) + - if [%COMPILER%]==[gcc] ( + echo ----- && + make -v && + echo ----- && + if not [%PLATFORM%]==[clang] ( + make -C programs lizard && make -C tests fullbench && make -C lib lib + ) ELSE ( + make -C programs lizard CC=clang MOREFLAGS="--target=x86_64-w64-mingw32 -Werror -Wconversion -Wno-sign-conversion" && + make -C tests fullbench CC=clang MOREFLAGS="--target=x86_64-w64-mingw32 -Werror -Wconversion -Wno-sign-conversion" && + make -C lib lib CC=clang MOREFLAGS="--target=x86_64-w64-mingw32 -Werror -Wconversion -Wno-sign-conversion" + ) + ) + - if [%COMPILER%]==[gccX] if not [%PLATFORM%]==[clang] ( + MKDIR bin\dll bin\static bin\example bin\include && + COPY tests\fullbench.c bin\example\ && + COPY lib\xxhash.c bin\example\ && + COPY lib\xxhash.h bin\example\ && + COPY lib\lizard.h bin\include\ && + COPY lib\lizardhc.h bin\include\ && + COPY lib\lizard_frame.h bin\include\ && + COPY lib\liblizard.a bin\static\liblizard_static.lib && + COPY lib\dll\liblizard.* bin\dll\ && + COPY lib\dll\example\Makefile bin\example\ && + COPY lib\dll\example\fullbench-dll.* bin\example\ && + COPY lib\dll\example\README.md bin\ && + COPY programs\lizard.exe bin\lizard.exe + ) + - if [%COMPILER%]==[gccX] if [%PLATFORM%]==[mingw64] ( + 7z.exe a bin\lizard_x64.zip NEWS .\bin\lizard.exe .\bin\README.md .\bin\example .\bin\dll .\bin\static .\bin\include && + appveyor PushArtifact bin\lizard_x64.zip + ) + - if [%COMPILER%]==[gccX] if [%PLATFORM%]==[mingw32] ( + 7z.exe a bin\lizard_x86.zip NEWS .\bin\lizard.exe .\bin\README.md .\bin\example .\bin\dll .\bin\static .\bin\include && + appveyor PushArtifact bin\lizard_x86.zip + ) + - if [%COMPILER%]==[gcc] (COPY tests\fullbench.exe programs\) + - if [%COMPILER%]==[visual] ( + ECHO *** && + ECHO *** Building Visual Studio 2010 %PLATFORM%\%CONFIGURATION% && + ECHO *** && + msbuild "visual\VS2010\lizard.sln" %ADDITIONALPARAM% /m /verbosity:minimal /property:PlatformToolset=v100 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && + ECHO *** && + ECHO *** Building Visual Studio 2012 %PLATFORM%\%CONFIGURATION% && + ECHO *** && + msbuild "visual\VS2010\lizard.sln" /m /verbosity:minimal /property:PlatformToolset=v110 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && + ECHO *** && + ECHO *** Building Visual Studio 2013 %PLATFORM%\%CONFIGURATION% && + ECHO *** && + msbuild "visual\VS2010\lizard.sln" /m /verbosity:minimal /property:PlatformToolset=v120 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && + ECHO *** && + ECHO *** Building Visual Studio 2015 %PLATFORM%\%CONFIGURATION% && + ECHO *** && + msbuild "visual\VS2010\lizard.sln" /m /verbosity:minimal /property:PlatformToolset=v140 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && + COPY visual\VS2010\bin\%PLATFORM%_%CONFIGURATION%\*.exe programs\ + ) + +test_script: + - ECHO *** && + ECHO Testing %COMPILER% %PLATFORM% %CONFIGURATION% && + ECHO *** + - if not [%COMPILER%]==[unknown] ( + CD programs && + lizard -h && + lizard -i1b lizard.exe && + lizard -i1b5 lizard.exe && + lizard -i1b10 lizard.exe && + lizard -i1b15 lizard.exe && + echo ------- lizard tested ------- && + fullbench.exe -i1 fullbench.exe + ) + +artifacts: + - path: bin\lizard_x64.zip + - path: bin\lizard_x86.zip + +deploy: +- provider: GitHub + artifact: bin\lizard_x64.zip + auth_token: + secure: LgJo8emYc3sFnlNWkGl4/VYK3nk/8+RagcsqDlAi3xeqNGNutnKjcftjg84uJoT4 + force_update: true + prerelease: true + on: + COMPILER: gcc + PLATFORM: "mingw64" + appveyor_repo_tag: true + branch: autobuild + +- provider: GitHub + artifact: bin\lizard_x86.zip + auth_token: + secure: LgJo8emYc3sFnlNWkGl4/VYK3nk/8+RagcsqDlAi3xeqNGNutnKjcftjg84uJoT4 + force_update: true + prerelease: true + on: + COMPILER: gcc + PLATFORM: "mingw32" + appveyor_repo_tag: true + branch: autobuild diff --git a/contrib/lizard/contrib/djgpp/LICENSE b/contrib/lizard/contrib/djgpp/LICENSE new file mode 100644 index 00000000000..fee0d3b3e23 --- /dev/null +++ b/contrib/lizard/contrib/djgpp/LICENSE @@ -0,0 +1,24 @@ +Copyright (c) 2014, lpsantil +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/contrib/lizard/contrib/djgpp/README.MD b/contrib/lizard/contrib/djgpp/README.MD new file mode 100644 index 00000000000..8dfc3a838c4 --- /dev/null +++ b/contrib/lizard/contrib/djgpp/README.MD @@ -0,0 +1,21 @@ +# lizard for DOS/djgpp +This file details on how to compile lizard.exe, and liblizard.a for use on DOS/djgpp using +Andrew Wu's build-djgpp cross compilers ([GH][0], [Binaries][1]) on OSX, Linux. + +## Setup +* Download a djgpp tarball [binaries][1] for your platform. +* Extract and install it (`tar jxvf djgpp-linux64-gcc492.tar.bz2`). Note the path. We'll assume `/home/user/djgpp`. +* Add the `bin` folder to your `PATH`. In bash, do `export PATH=/home/user/djgpp/bin:$PATH`. +* The `Makefile` in `contrib/djgpp/` sets up `CC`, `AR`, `LD` for you. So, `CC=i586-pc-msdosdjgpp-gcc`, `AR=i586-pc-msdosdjgpp-ar`, `LD=i586-pc-msdosdjgpp-gcc`. + +## Building Lizard for DOS +In the base dir of lizard and with `contrib/djgpp/Makefile`, try: +Try: +* `make -f contrib/djgpp/Makefile` +* `make -f contrib/djgpp/Makefile liblizard.a` +* `make -f contrib/djgpp/Makefile lizard.exe` +* `make -f contrib/djgpp/Makefile DESTDIR=/home/user/dos install`, however it doesn't make much sense on a \*nix. +* You can also do `make -f contrib/djgpp/Makefile uninstall` + +[0]: https://github.com/andrewwutw/build-djgpp +[1]: https://github.com/andrewwutw/build-djgpp/releases diff --git a/contrib/lizard/doc/lizard_Block_format.md b/contrib/lizard/doc/lizard_Block_format.md new file mode 100644 index 00000000000..45307a1fce5 --- /dev/null +++ b/contrib/lizard/doc/lizard_Block_format.md @@ -0,0 +1,181 @@ +Lizard v1.x Block Format Description +============================ +Last revised: 2016-10-08 +Authors : Yann Collet, Przemyslaw Skibinski + + +This specification is intended for developers +willing to produce Lizard-compatible compressed data blocks +using any programming language. + +Lizard is an LZ77-type compressor with a fixed, byte-oriented encoding. +There is no framing layer as it is assumed to be handled by other parts of the system (see [Lizard Frame format]). +This design is assumed to favor simplicity and speed. +It helps later on for optimizations, compactness, and features. + +This document describes only the block format, +not how the compressor nor decompressor actually work. +The correctness of the decompressor should not depend +on implementation details of the compressor, and vice versa. + +[Lizard Frame format]: lizard_Frame_format.md + + +Division into blocks +-------------------- + +The input data is divided into blocks of maximum size LIZARD_BLOCK_SIZE (which is 128 KB). The subsequent blocks use the same sliding window and are dependent on previous blocks. +Our impementation of Lizard compressor divides input data into blocks of size of LIZARD_BLOCK_SIZE except the last one which usually will be smaller. +The output data is a single byte 'Compression_Level' and one or more blocks in the format described below. + + +Block header format +----------------------- + +The block header is a single byte `Header_Byte` that is combination of following flags: + +| Name | Value | +| --------------------- | --- | +| LIZARD_FLAG_LITERALS | 1 | +| LIZARD_FLAG_FLAGS | 2 | +| LIZARD_FLAG_OFF16LEN | 4 | +| LIZARD_FLAG_OFF24LEN | 8 | +| LIZARD_FLAG_LEN | 16 | +| LIZARD_FLAG_UNCOMPRESSED | 128 | + +When `Header_Byte & LIZARD_FLAG_UNCOMPRESSED` is true then the block is followed by 3-byte `Uncompressed_length` and uncompressed data of given size. + + +Compressed block content +------------------------ + +When `Header_Byte & LIZARD_FLAG_UNCOMPRESSED` is false then compressed block contains of 5 streams: +- `Lengths_Stream` (compressed with Huffman if LIZARD_FLAG_LEN is set) +- `16-bit_Offsets_Stream` (compressed with Huffman if LIZARD_FLAG_OFF16LEN is set) +- `24-bit_Offsets_Stream` (compressed with Huffman if LIZARD_FLAG_OFF24LEN is set) +- `Tokens_Stream` (compressed with Huffman if LIZARD_FLAG_FLAGS is set) +- `Literals_Stream` (compressed with Huffman if LIZARD_FLAG_LITERALS is set) + + +Stream format +------------- +The single stream is either: +- if LIZARD_FLAG_XXX is not set: 3 byte `Stream_Length` followed by a given number bytes +- if LIZARD_FLAG_XXX is set: 3 byte `Original_Stream_Length`, 3 byte `Compressed_Stream_Length`, followed by a given number of Huffman compressed bytes + + +Lizard block decompression +----------------------- +At the beginning we have 5 streams and their sizes. +Decompressor should iterate through `Tokens_Stream`. Each token is 1-byte long and describes how to get data from other streams. +If token points a stream that is already empty it means that data is corrupted. + + +Lizard token decompression +----------------------- +The token is a one byte. Token decribes: +- how many literals should be copied from `Literals_Stream` +- if offset should be read from `16-bit_Offsets_Stream` or `24-bit_Offsets_Stream` +- how many bytes are part of a match and should be copied from a sliding window + +Lizard uses 4 types of tokens: +- [0_MMMM_LLL] - 3-bit literal length (0-7+), use offset from `16-bit_Offsets_Stream`, 4-bit match length (4-15+) +- [1_MMMM_LLL] - 3-bit literal length (0-7+), use last offset, 4-bit match length (0-15+) +- token 31 - no literal length, use offset from `24-bit_Offsets_Stream`, match length (47+) +- token 0-30 - no literal length, use offset from `24-bit_Offsets_Stream`, 31 match lengths (16-46) + +Lizard uses different output codewords and is not compatible with LZ4. LZ4 output codewords are 3 byte long (24-bit) and look as follows: +- LLLL_MMMM OOOOOOOO OOOOOOOO - 16-bit offset, 4-bit match length, 4-bit literal length + + +The format of `Lengths_Stream` +------------------------------ +`Lengths_Stream` contains lenghts in the the following format: +- when 'First_Byte' is < 254 then lenght is equal 'First_Byte' +- when 'First_Byte' is 254 then lenght is equal to value of 2-bytes after 'First_Byte' i.e. 0-65536 +- when 'First_Byte' is 255 then lenght is equal to value of 3-bytes after 'First_Byte' i.e. 0-16777215 + + +[0_MMMM_LLL] and [1_MMMM_LLL] tokens +--------------------------------------- +The length of literals to be copied from `Literals_Stream` depends on the literal length field (LLL) that uses 3 bits of the token. +Therefore each field ranges from 0 to 7. +If the value is 7, then the lenght is increased with a length taken from `Lengths_Stream`. + +Example 1 : A literal length of 48 will be represented as : + + - 7 : value for the 3-bits LLL field + - 41 : (=48-7) remaining length to reach 48 (in `Lengths_Stream`) + +Example 2 : A literal length of 280 for will be represented as : + + - 7 : value for the 3-bits LLL field + - 254 : informs that remaining length (=280-7) must be represented as 2-bytes (in `Lengths_Stream`) + - 273 : (=280-7) encoded as 2-bytes (in `Lengths_Stream`) + +Example 3 : A literal length of 7 for will be represented as : + + - 7 : value for the 3-bits LLL field + - 0 : (=7-7) yes, the zero must be output (in `Lengths_Stream`) + +After copying 0 or more literals from `Literals_Stream` we can prepare the match copy operation which depends on a offset and a match length. +The flag "0" informs that decoder should use the last encoded offset. +The flag "1" informs that the offset is a 2 bytes value (16-bit), in little endian format and should be taken from `16-bit_Offsets_Stream`. + +The match length depends on the match length field (MMMM) that uses 4 bits of the token. +Therefore each field ranges from 0 to 15. Values from 0-3 are forbidden with offset taken from `16-bit_Offsets_Stream`. +If the value is 15, then the lenght is increased with a length taken from `Lengths_Stream`. + +With the offset and the match length, +the decoder can now proceed to copy the data from the already decoded buffer. + + +Lizard block epilogue +------------------ +When all tokens are read from `Tokens_Stream` and interpreted all remaining streams should also be empty. +Otherwise, it means that the data is corrupted. The only exception is `Literals_Stream` that should have at least 16 remaining literals what +allows fast memory copy operations. The remaining literals up to the end of `Literals_Stream` should be appended to the output data. + + +Tokens 0-31 +----------- +The offset is a 3 bytes value (24-bit), in little endian format and should be taken from `24-bit_Offsets_Stream`. +The offset represents the position of the match to be copied from. +1 means "current position - 1 byte". +The maximum offset value is (1<<24)-1, 1<<24 cannot be coded. +Note that 0 is an invalid value, not used. + +The 'Token_Value' ranges from 0 to 31. +The match length is equal to 'Token_Value + 16 that is from 16 to 47. +If match length is 47, the lenght is increased with a length taken from `Lengths_Stream`. + + +Parsing restrictions +----------------------- +There are specific parsing rules to respect in order to remain compatible +with assumptions made by the decoder : + +1. The last 16 bytes are always literals what allows fast memory copy operations. +2. The last match must start at least 20 bytes before end of block. + Consequently, a block with less than 20 bytes cannot be compressed. + +These rules are in place to ensure that the decoder +will never read beyond the input buffer, nor write beyond the output buffer. + +Note that the last sequence is also incomplete, +and stops right after literals. + + +Additional notes +----------------------- +There is no assumption nor limits to the way the compressor +searches and selects matches within the source data block. +It could be a fast scan, a multi-probe, a full search using BST, +standard hash chains or MMC, well whatever. + +Advanced parsing strategies can also be implemented, such as lazy match, +or full optimal parsing. + +All these trade-off offer distinctive speed/memory/compression advantages. +Whatever the method used by the compressor, its result will be decodable +by any Lizard decoder if it follows the format specification described above. diff --git a/contrib/lizard/doc/lizard_Frame_format.md b/contrib/lizard/doc/lizard_Frame_format.md new file mode 100644 index 00000000000..ce24b0e23e2 --- /dev/null +++ b/contrib/lizard/doc/lizard_Frame_format.md @@ -0,0 +1,312 @@ +Lizard v1.x Frame Format Description +================================= + +###Notices + +Copyright (c) 2013-2015 Yann Collet + +Copyright (c) 2016 Przemyslaw Skibinski + +Permission is granted to copy and distribute this document +for any purpose and without charge, +including translations into other languages +and incorporation into compilations, +provided that the copyright notice and this notice are preserved, +and that any substantive changes or deletions from the original +are clearly marked. +Distribution of this document is unlimited. + +###Version + +1.0 (8-10-2016) + + +Introduction +------------ + +The purpose of this document is to define a lossless compressed data format, +that is independent of CPU type, operating system, +file system and character set, suitable for +File compression, Pipe and streaming compression +using the Lizard algorithm. + +The data can be produced or consumed, +even for an arbitrarily long sequentially presented input data stream, +using only an a priori bounded amount of intermediate storage, +and hence can be used in data communications. +The format uses the Lizard compression method, +and optional [xxHash-32 checksum method](https://github.com/Cyan4973/xxHash), +for detection of data corruption. + +The data format defined by this specification +does not attempt to allow random access to compressed data. + +This specification is intended for use by implementers of software +to compress data into Lizard format and/or decompress data from Lizard format. +The text of the specification assumes a basic background in programming +at the level of bits and other primitive data representations. + +Unless otherwise indicated below, +a compliant compressor must produce data sets +that conform to the specifications presented here. +It doesn’t need to support all options though. + +A compliant decompressor must be able to decompress +at least one working set of parameters +that conforms to the specifications presented here. +It may also ignore checksums. +Whenever it does not support a specific parameter within the compressed stream, +it must produce a non-ambiguous error code +and associated error message explaining which parameter is unsupported. + + +General Structure of Lizard Frame format +------------------------------------- + +| MagicNb | F. Descriptor | Block | (...) | EndMark | C. Checksum | +|:-------:|:-------------:| ----- | ----- | ------- | ----------- | +| 4 bytes | 3-11 bytes | | | 4 bytes | 0-4 bytes | + +__Magic Number__ + +4 Bytes, Little endian format. +Value : 0x184D2206 (it was 0x184D2204 for LZ4 and 0x184D2205 for LZ5 v1.x) + +__Frame Descriptor__ + +3 to 11 Bytes, to be detailed in the next part. +Most important part of the spec. + +__Data Blocks__ + +To be detailed later on. +That’s where compressed data is stored. + +__EndMark__ + +The flow of blocks ends when the last data block has a size of “0”. +The size is expressed as a 32-bits value. + +__Content Checksum__ + +Content Checksum verify that the full content has been decoded correctly. +The content checksum is the result +of [xxh32() hash function](https://github.com/Cyan4973/xxHash) +digesting the original (decoded) data as input, and a seed of zero. +Content checksum is only present when its associated flag +is set in the frame descriptor. +Content Checksum validates the result, +that all blocks were fully transmitted in the correct order and without error, +and also that the encoding/decoding process itself generated no distortion. +Its usage is recommended. + +__Frame Concatenation__ + +In some circumstances, it may be preferable to append multiple frames, +for example in order to add new data to an existing compressed file +without re-framing it. + +In such case, each frame has its own set of descriptor flags. +Each frame is considered independent. +The only relation between frames is their sequential order. + +The ability to decode multiple concatenated frames +within a single stream or file +is left outside of this specification. +As an example, the reference lizard command line utility behavior is +to decode all concatenated frames in their sequential order. + + +Frame Descriptor +---------------- + +| FLG | BD | (Content Size) | HC | +| ------- | ------- |:--------------:| ------- | +| 1 byte | 1 byte | 0 - 8 bytes | 1 byte | + +The descriptor uses a minimum of 3 bytes, +and up to 11 bytes depending on optional parameters. + +__FLG byte__ + +| BitNb | 7-6 | 5 | 4 | 3 | 2 | 1-0 | +| ------- | ------- | ------- | --------- | ------- | --------- | -------- | +|FieldName| Version | B.Indep | B.Checksum| C.Size | C.Checksum|*Reserved*| + + +__BD byte__ + +| BitNb | 7 | 6-5-4 | 3-2-1-0 | +| ------- | -------- | ------------ | -------- | +|FieldName|*Reserved*| Block MaxSize|*Reserved*| + +In the tables, bit 7 is highest bit, while bit 0 is lowest. + +__Version Number__ + +2-bits field, must be set to “01”. +Any other value cannot be decoded by this version of the specification. +Other version numbers will use different flag layouts. + +__Block Independence flag__ + +If this flag is set to “1”, blocks are independent. +If this flag is set to “0”, each block depends on previous ones +(up to Lizard window size, which is 16 MB). +In such case, it’s necessary to decode all blocks in sequence. + +Block dependency improves compression ratio, especially for small blocks. +On the other hand, it makes direct jumps or multi-threaded decoding impossible. + +__Block checksum flag__ + +If this flag is set, each data block will be followed by a 4-bytes checksum, +calculated by using the xxHash-32 algorithm on the raw (compressed) data block. +The intention is to detect data corruption (storage or transmission errors) +immediately, before decoding. +Block checksum usage is optional. + +__Content Size flag__ + +If this flag is set, the uncompressed size of data included within the frame +will be present as an 8 bytes unsigned little endian value, after the flags. +Content Size usage is optional. + +__Content checksum flag__ + +If this flag is set, a content checksum will be appended after the EndMark. + +Recommended value : “1” (content checksum is present) + +__Block Maximum Size__ + +This information is intended to help the decoder allocate memory. +Size here refers to the original (uncompressed) data size. +Block Maximum Size is one value among the following table : + +| 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | +| --- | ------ | ------ | ---- | ---- | ----- | ----- | ------ | +| N/A | 128 KB | 256 KB | 1 MB | 4 MB | 16 MB | 64 MB | 256 MB | + + +The decoder may refuse to allocate block sizes above a (system-specific) size. +Unused values may be used in a future revision of the spec. +A decoder conformant to the current version of the spec +is only able to decode blocksizes defined in this spec. + +__Reserved bits__ + +Value of reserved bits **must** be 0 (zero). +Reserved bit might be used in a future version of the specification, +typically enabling new optional features. +If this happens, a decoder respecting the current version of the specification +shall not be able to decode such a frame. + +__Content Size__ + +This is the original (uncompressed) size. +This information is optional, and only present if the associated flag is set. +Content size is provided using unsigned 8 Bytes, for a maximum of 16 HexaBytes. +Format is Little endian. +This value is informational, typically for display or memory allocation. +It can be skipped by a decoder, or used to validate content correctness. + +__Header Checksum__ + +One-byte checksum of combined descriptor fields, including optional ones. +The value is the second byte of xxh32() : ` (xxh32()>>8) & 0xFF ` +using zero as a seed, +and the full Frame Descriptor as an input +(including optional fields when they are present). +A wrong checksum indicates an error in the descriptor. +Header checksum is informational and can be skipped. + + +Data Blocks +----------- + +| Block Size | data | (Block Checksum) | +|:----------:| ------ |:----------------:| +| 4 bytes | | 0 - 4 bytes | + + +__Block Size__ + +This field uses 4-bytes, format is little-endian. + +The highest bit is “1” if data in the block is uncompressed. + +The highest bit is “0” if data in the block is compressed by Lizard. + +All other bits give the size, in bytes, of the following data block +(the size does not include the block checksum if present). + +Block Size shall never be larger than Block Maximum Size. +Such a thing could happen for incompressible source data. +In such case, such a data block shall be passed in uncompressed format. + +__Data__ + +Where the actual data to decode stands. +It might be compressed or not, depending on previous field indications. +Uncompressed size of Data can be any size, up to “block maximum size”. +Note that data block is not necessarily full : +an arbitrary “flush” may happen anytime. Any block can be “partially filled”. + +__Block checksum__ + +Only present if the associated flag is set. +This is a 4-bytes checksum value, in little endian format, +calculated by using the xxHash-32 algorithm on the raw (undecoded) data block, +and a seed of zero. +The intention is to detect data corruption (storage or transmission errors) +before decoding. + +Block checksum is cumulative with Content checksum. + + +Skippable Frames +---------------- + +| Magic Number | Frame Size | User Data | +|:------------:|:----------:| --------- | +| 4 bytes | 4 bytes | | + +Skippable frames allow the integration of user-defined data +into a flow of concatenated frames. +Its design is pretty straightforward, +with the sole objective to allow the decoder to quickly skip +over user-defined data and continue decoding. + +For the purpose of facilitating identification, +it is discouraged to start a flow of concatenated frames with a skippable frame. +If there is a need to start such a flow with some user data +encapsulated into a skippable frame, +it’s recommended to start with a zero-byte Lizard frame +followed by a skippable frame. +This will make it easier for file type identifiers. + + +__Magic Number__ + +4 Bytes, Little endian format. +Value : 0x184D2A5X, which means any value from 0x184D2A50 to 0x184D2A5F. +All 16 values are valid to identify a skippable frame. + +__Frame Size__ + +This is the size, in bytes, of the following User Data +(without including the magic number nor the size field itself). +4 Bytes, Little endian format, unsigned 32-bits. +This means User Data can’t be bigger than (2^32-1) Bytes. + +__User Data__ + +User Data can be anything. Data will just be skipped by the decoder. + + + +Version changes +--------------- + +1.0 : based on LZ4 Frame Format Description 1.5.1 (31/03/2015) diff --git a/contrib/lizard/examples/.gitignore b/contrib/lizard/examples/.gitignore new file mode 100644 index 00000000000..f6f8bbcca09 --- /dev/null +++ b/contrib/lizard/examples/.gitignore @@ -0,0 +1,8 @@ +/Makefile.liz* +/printVersion +/doubleBuffer +/ringBuffer +/ringBufferHC +/lineCompress +/frameCompress +/*.exe diff --git a/contrib/lizard/examples/HCStreaming_ringBuffer.c b/contrib/lizard/examples/HCStreaming_ringBuffer.c new file mode 100644 index 00000000000..3e4d75a5d78 --- /dev/null +++ b/contrib/lizard/examples/HCStreaming_ringBuffer.c @@ -0,0 +1,241 @@ +// Lizard streaming API example : ring buffer +// Based on previous work from Takayuki Matsuoka + + +/************************************** + * Compiler Options + **************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define _CRT_SECURE_NO_WARNINGS // for MSVC +# define snprintf sprintf_s +#endif + +#define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) +#ifdef __GNUC__ +# pragma GCC diagnostic ignored "-Wmissing-braces" /* GCC bug 53119 : doesn't accept { 0 } as initializer (https://gcc.gnu.org/bugzilla/show_bug.cgi?id=53119) */ +#endif + + +/************************************** + * Includes + **************************************/ +#include "lizard_compress.h" +#include "lizard_decompress.h" +#include "lizard_common.h" + +#include +#include +#include +#include + +enum { + MESSAGE_MAX_BYTES = 1024, + RING_BUFFER_BYTES = 1024 * 8 + MESSAGE_MAX_BYTES, + DEC_BUFFER_BYTES = RING_BUFFER_BYTES + MESSAGE_MAX_BYTES // Intentionally larger to test unsynchronized ring buffers +}; + + +size_t write_int32(FILE* fp, int32_t i) { + return fwrite(&i, sizeof(i), 1, fp); +} + +size_t write_bin(FILE* fp, const void* array, int arrayBytes) { + return fwrite(array, 1, arrayBytes, fp); +} + +size_t read_int32(FILE* fp, int32_t* i) { + return fread(i, sizeof(*i), 1, fp); +} + +size_t read_bin(FILE* fp, void* array, int arrayBytes) { + return fread(array, 1, arrayBytes, fp); +} + + +void test_compress(FILE* outFp, FILE* inpFp) +{ + Lizard_stream_t* lizardStream = Lizard_createStream(0); + static char inpBuf[RING_BUFFER_BYTES]; + int inpOffset = 0; + + if (!lizardStream) return; + + for(;;) + { + // Read random length ([1,MESSAGE_MAX_BYTES]) data to the ring buffer. + char* const inpPtr = &inpBuf[inpOffset]; + const int randomLength = (rand() % MESSAGE_MAX_BYTES) + 1; + const int inpBytes = (int) read_bin(inpFp, inpPtr, randomLength); + if (0 == inpBytes) break; + + { + char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; + const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, Lizard_compressBound(inpBytes)); + + if(cmpBytes <= 0) break; + write_int32(outFp, cmpBytes); + write_bin(outFp, cmpBuf, cmpBytes); + + inpOffset += inpBytes; + + // Wraparound the ringbuffer offset + if(inpOffset >= RING_BUFFER_BYTES - MESSAGE_MAX_BYTES) + inpOffset = 0; + } + } + + write_int32(outFp, 0); + Lizard_freeStream(lizardStream); +} + + +void test_decompress(FILE* outFp, FILE* inpFp) +{ + static char decBuf[DEC_BUFFER_BYTES]; + int decOffset = 0; + Lizard_streamDecode_t lizardStreamDecode_body = { 0 }; + Lizard_streamDecode_t* lizardStreamDecode = &lizardStreamDecode_body; + + for(;;) + { + int cmpBytes = 0; + char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; + + { + const size_t r0 = read_int32(inpFp, &cmpBytes); + size_t r1; + if(r0 != 1 || cmpBytes <= 0) + break; + + r1 = read_bin(inpFp, cmpBuf, cmpBytes); + if(r1 != (size_t) cmpBytes) + break; + } + + { + char* const decPtr = &decBuf[decOffset]; + const int decBytes = Lizard_decompress_safe_continue( + lizardStreamDecode, cmpBuf, decPtr, cmpBytes, MESSAGE_MAX_BYTES); + if(decBytes <= 0) + break; + + decOffset += decBytes; + write_bin(outFp, decPtr, decBytes); + + // Wraparound the ringbuffer offset + if(decOffset >= DEC_BUFFER_BYTES - MESSAGE_MAX_BYTES) + decOffset = 0; + } + } +} + + +// Compare 2 files content +// return 0 if identical +// return ByteNb>0 if different +size_t compare(FILE* f0, FILE* f1) +{ + size_t result = 1; + + for (;;) + { + char b0[65536]; + char b1[65536]; + const size_t r0 = fread(b0, 1, sizeof(b0), f0); + const size_t r1 = fread(b1, 1, sizeof(b1), f1); + + if ((r0==0) && (r1==0)) return 0; // success + + if (r0 != r1) + { + size_t smallest = r0; + if (r1 +#include +#include +#include + +enum { + BLOCK_BYTES = 1024 * 8, +// BLOCK_BYTES = 1024 * 64, +}; + + +size_t write_int(FILE* fp, int i) { + return fwrite(&i, sizeof(i), 1, fp); +} + +size_t write_bin(FILE* fp, const void* array, size_t arrayBytes) { + return fwrite(array, 1, arrayBytes, fp); +} + +size_t read_int(FILE* fp, int* i) { + return fread(i, sizeof(*i), 1, fp); +} + +size_t read_bin(FILE* fp, void* array, size_t arrayBytes) { + return fread(array, 1, arrayBytes, fp); +} + + +void test_compress(FILE* outFp, FILE* inpFp) +{ + Lizard_stream_t* lizardStream = Lizard_createStream_MinLevel(); + char inpBuf[2][BLOCK_BYTES]; + int inpBufIndex = 0; + + if (!lizardStream) return; + lizardStream = Lizard_resetStream_MinLevel(lizardStream); + if (!lizardStream) return; + + for(;;) { + char* const inpPtr = inpBuf[inpBufIndex]; + const int inpBytes = (int) read_bin(inpFp, inpPtr, BLOCK_BYTES); + if(0 == inpBytes) { + break; + } + + { + char cmpBuf[LIZARD_COMPRESSBOUND(BLOCK_BYTES)]; + const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, sizeof(cmpBuf)); + if(cmpBytes <= 0) { + break; + } + write_int(outFp, cmpBytes); + write_bin(outFp, cmpBuf, (size_t) cmpBytes); + } + + inpBufIndex = (inpBufIndex + 1) % 2; + } + + write_int(outFp, 0); + Lizard_freeStream(lizardStream); +} + + +void test_decompress(FILE* outFp, FILE* inpFp) +{ + Lizard_streamDecode_t lizardStreamDecode_body; + Lizard_streamDecode_t* lizardStreamDecode = &lizardStreamDecode_body; + + char decBuf[2][BLOCK_BYTES]; + int decBufIndex = 0; + + Lizard_setStreamDecode(lizardStreamDecode, NULL, 0); + + for(;;) { + char cmpBuf[LIZARD_COMPRESSBOUND(BLOCK_BYTES)]; + int cmpBytes = 0; + + { + const size_t readCount0 = read_int(inpFp, &cmpBytes); + if(readCount0 != 1 || cmpBytes <= 0) { + break; + } + + const size_t readCount1 = read_bin(inpFp, cmpBuf, (size_t) cmpBytes); + if(readCount1 != (size_t) cmpBytes) { + break; + } + } + + { + char* const decPtr = decBuf[decBufIndex]; + const int decBytes = Lizard_decompress_safe_continue( + lizardStreamDecode, cmpBuf, decPtr, cmpBytes, BLOCK_BYTES); + if(decBytes <= 0) { + break; + } + write_bin(outFp, decPtr, (size_t) decBytes); + } + + decBufIndex = (decBufIndex + 1) % 2; + } +} + + +int compare(FILE* fp0, FILE* fp1) +{ + int result = 0; + + while(0 == result) { + char b0[65536]; + char b1[65536]; + const size_t r0 = read_bin(fp0, b0, sizeof(b0)); + const size_t r1 = read_bin(fp1, b1, sizeof(b1)); + + result = (int) r0 - (int) r1; + + if(0 == r0 || 0 == r1) { + break; + } + if(0 == result) { + result = memcmp(b0, b1, r0); + } + } + + return result; +} + + +int main(int argc, char* argv[]) +{ + char inpFilename[256] = { 0 }; + char lizardFilename[256] = { 0 }; + char decFilename[256] = { 0 }; + + if(argc < 2) { + printf("Please specify input filename\n"); + return 0; + } + + snprintf(inpFilename, 256, "%s", argv[1]); + snprintf(lizardFilename, 256, "%s.lizs-%d", argv[1], BLOCK_BYTES); + snprintf(decFilename, 256, "%s.lizs-%d.dec", argv[1], BLOCK_BYTES); + + printf("inp = [%s]\n", inpFilename); + printf("lizard = [%s]\n", lizardFilename); + printf("dec = [%s]\n", decFilename); + + // compress + { + FILE* inpFp = fopen(inpFilename, "rb"); + FILE* outFp = fopen(lizardFilename, "wb"); + + printf("compress : %s -> %s\n", inpFilename, lizardFilename); + test_compress(outFp, inpFp); + printf("compress : done\n"); + + fclose(outFp); + fclose(inpFp); + } + + // decompress + { + FILE* inpFp = fopen(lizardFilename, "rb"); + FILE* outFp = fopen(decFilename, "wb"); + + printf("decompress : %s -> %s\n", lizardFilename, decFilename); + test_decompress(outFp, inpFp); + printf("decompress : done\n"); + + fclose(outFp); + fclose(inpFp); + } + + // verify + { + FILE* inpFp = fopen(inpFilename, "rb"); + FILE* decFp = fopen(decFilename, "rb"); + + printf("verify : %s <-> %s\n", inpFilename, decFilename); + const int cmp = compare(inpFp, decFp); + if(0 == cmp) { + printf("verify : OK\n"); + } else { + printf("verify : NG\n"); + } + + fclose(decFp); + fclose(inpFp); + } + + return 0; +} diff --git a/contrib/lizard/examples/blockStreaming_doubleBuffer.md b/contrib/lizard/examples/blockStreaming_doubleBuffer.md new file mode 100644 index 00000000000..c27afcfb98c --- /dev/null +++ b/contrib/lizard/examples/blockStreaming_doubleBuffer.md @@ -0,0 +1,100 @@ +# Lizard streaming API Example : Double Buffer +by *Takayuki Matsuoka* + +`blockStreaming_doubleBuffer.c` is Lizard Straming API example which implements double buffer (de)compression. + +Please note : + + - Firstly, read "Lizard Streaming API Basics". + - This is relatively advanced application example. + - Output file is not compatible with lizard_frame and platform dependent. + + +## What's the point of this example ? + + - Handle huge file in small amount of memory + - Always better compression ratio than Block API + - Uniform block size + + +## How the compression works + +First of all, allocate "Double Buffer" for input and Lizard compressed data buffer for output. +Double buffer has two pages, "first" page (Page#1) and "second" page (Page#2). + +``` + Double Buffer + + Page#1 Page#2 + +---------+---------+ + | Block#1 | | + +----+----+---------+ + | + v + {Out#1} + + + Prefix Dependency + +---------+ + | | + v | + +---------+----+----+ + | Block#1 | Block#2 | + +---------+----+----+ + | + v + {Out#2} + + + External Dictionary Mode + +---------+ + | | + | v + +----+----+---------+ + | Block#3 | Block#2 | + +----+----+---------+ + | + v + {Out#3} + + + Prefix Dependency + +---------+ + | | + v | + +---------+----+----+ + | Block#3 | Block#4 | + +---------+----+----+ + | + v + {Out#4} +``` + +Next, read first block to double buffer's first page. And compress it by `Lizard_compress_continue()`. +For the first time, Lizard doesn't know any previous dependencies, +so it just compress the line without dependencies and generates compressed block {Out#1} to Lizard compressed data buffer. +After that, write {Out#1} to the file. + +Next, read second block to double buffer's second page. And compress it. +In this time, Lizard can use dependency to Block#1 to improve compression ratio. +This dependency is called "Prefix mode". + +Next, read third block to double buffer's *first* page. And compress it. +Also this time, Lizard can use dependency to Block#2. +This dependency is called "External Dictonaly mode". + +Continue these procedure to the end of the file. + + +## How the decompression works + +Decompression will do reverse order. + + - Read first compressed block. + - Decompress it to the first page and write that page to the file. + - Read second compressed block. + - Decompress it to the second page and write that page to the file. + - Read third compressed block. + - Decompress it to the *first* page and write that page to the file. + +Continue these procedure to the end of the compressed file. diff --git a/contrib/lizard/examples/blockStreaming_lineByLine.c b/contrib/lizard/examples/blockStreaming_lineByLine.c new file mode 100644 index 00000000000..9d239533501 --- /dev/null +++ b/contrib/lizard/examples/blockStreaming_lineByLine.c @@ -0,0 +1,210 @@ +// Lizard streaming API example : line-by-line logfile compression +// Copyright : Takayuki Matsuoka + + +#ifdef _MSC_VER /* Visual Studio */ +# define _CRT_SECURE_NO_WARNINGS +# define snprintf sprintf_s +#endif +#include "lizard_common.h" +#include "lizard_decompress.h" +#include +#include +#include +#include + +static size_t write_uint16(FILE* fp, uint16_t i) +{ + return fwrite(&i, sizeof(i), 1, fp); +} + +static size_t write_bin(FILE* fp, const void* array, int arrayBytes) +{ + return fwrite(array, 1, arrayBytes, fp); +} + +static size_t read_uint16(FILE* fp, uint16_t* i) +{ + return fread(i, sizeof(*i), 1, fp); +} + +static size_t read_bin(FILE* fp, void* array, int arrayBytes) +{ + return fread(array, 1, arrayBytes, fp); +} + + +static void test_compress( + FILE* outFp, + FILE* inpFp, + size_t messageMaxBytes, + size_t ringBufferBytes) +{ + Lizard_stream_t* const lizardStream = Lizard_createStream_MinLevel(); + const size_t cmpBufBytes = LIZARD_COMPRESSBOUND(messageMaxBytes); + char* const cmpBuf = (char*) malloc(cmpBufBytes); + char* const inpBuf = (char*) malloc(ringBufferBytes); + int inpOffset = 0; + + for ( ; ; ) + { + char* const inpPtr = &inpBuf[inpOffset]; + +#if 0 + // Read random length data to the ring buffer. + const int randomLength = (rand() % messageMaxBytes) + 1; + const int inpBytes = (int) read_bin(inpFp, inpPtr, randomLength); + if (0 == inpBytes) break; +#else + // Read line to the ring buffer. + int inpBytes = 0; + if (!fgets(inpPtr, (int) messageMaxBytes, inpFp)) + break; + inpBytes = (int) strlen(inpPtr); +#endif + + { + const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, cmpBufBytes); + if (cmpBytes <= 0) break; + write_uint16(outFp, (uint16_t) cmpBytes); + write_bin(outFp, cmpBuf, cmpBytes); + + // Add and wraparound the ringbuffer offset + inpOffset += inpBytes; + if ((size_t)inpOffset >= ringBufferBytes - messageMaxBytes) inpOffset = 0; + } + } + write_uint16(outFp, 0); + + free(inpBuf); + free(cmpBuf); + Lizard_freeStream(lizardStream); +} + + +static void test_decompress( + FILE* outFp, + FILE* inpFp, + size_t messageMaxBytes, + size_t ringBufferBytes) +{ + Lizard_streamDecode_t* const lizardStreamDecode = Lizard_createStreamDecode(); + char* const cmpBuf = (char*) malloc(LIZARD_COMPRESSBOUND(messageMaxBytes)); + char* const decBuf = (char*) malloc(ringBufferBytes); + int decOffset = 0; + + for ( ; ; ) + { + uint16_t cmpBytes = 0; + + if (read_uint16(inpFp, &cmpBytes) != 1) break; + if (cmpBytes <= 0) break; + if (read_bin(inpFp, cmpBuf, cmpBytes) != cmpBytes) break; + + { + char* const decPtr = &decBuf[decOffset]; + const int decBytes = Lizard_decompress_safe_continue( + lizardStreamDecode, cmpBuf, decPtr, cmpBytes, (int) messageMaxBytes); + if (decBytes <= 0) break; + write_bin(outFp, decPtr, decBytes); + + // Add and wraparound the ringbuffer offset + decOffset += decBytes; + if ((size_t)decOffset >= ringBufferBytes - messageMaxBytes) decOffset = 0; + } + } + + free(decBuf); + free(cmpBuf); + Lizard_freeStreamDecode(lizardStreamDecode); +} + + +static int compare(FILE* f0, FILE* f1) +{ + int result = 0; + const size_t tempBufferBytes = 65536; + char* const b0 = (char*) malloc(tempBufferBytes); + char* const b1 = (char*) malloc(tempBufferBytes); + + while(0 == result) + { + const size_t r0 = fread(b0, 1, tempBufferBytes, f0); + const size_t r1 = fread(b1, 1, tempBufferBytes, f1); + + result = (int) r0 - (int) r1; + + if (0 == r0 || 0 == r1) break; + if (0 == result) result = memcmp(b0, b1, r0); + } + + free(b1); + free(b0); + return result; +} + + +int main(int argc, char* argv[]) +{ + enum { + MESSAGE_MAX_BYTES = 1024, + RING_BUFFER_BYTES = 1024 * 256 + MESSAGE_MAX_BYTES, + }; + + char inpFilename[256] = { 0 }; + char lizardFilename[256] = { 0 }; + char decFilename[256] = { 0 }; + + if (argc < 2) + { + printf("Please specify input filename\n"); + return 0; + } + + snprintf(inpFilename, 256, "%s", argv[1]); + snprintf(lizardFilename, 256, "%s.lizs", argv[1]); + snprintf(decFilename, 256, "%s.lizs.dec", argv[1]); + + printf("inp = [%s]\n", inpFilename); + printf("lizard = [%s]\n", lizardFilename); + printf("dec = [%s]\n", decFilename); + + // compress + { + FILE* inpFp = fopen(inpFilename, "rb"); + FILE* outFp = fopen(lizardFilename, "wb"); + + test_compress(outFp, inpFp, MESSAGE_MAX_BYTES, RING_BUFFER_BYTES); + + fclose(outFp); + fclose(inpFp); + } + + // decompress + { + FILE* inpFp = fopen(lizardFilename, "rb"); + FILE* outFp = fopen(decFilename, "wb"); + + test_decompress(outFp, inpFp, MESSAGE_MAX_BYTES, RING_BUFFER_BYTES); + + fclose(outFp); + fclose(inpFp); + } + + // verify + { + FILE* inpFp = fopen(inpFilename, "rb"); + FILE* decFp = fopen(decFilename, "rb"); + + const int cmp = compare(inpFp, decFp); + if (0 == cmp) + printf("Verify : OK\n"); + else + printf("Verify : NG\n"); + + fclose(decFp); + fclose(inpFp); + } + + return 0; +} diff --git a/contrib/lizard/examples/blockStreaming_lineByLine.md b/contrib/lizard/examples/blockStreaming_lineByLine.md new file mode 100644 index 00000000000..ac56b879c90 --- /dev/null +++ b/contrib/lizard/examples/blockStreaming_lineByLine.md @@ -0,0 +1,122 @@ +# Lizard streaming API Example : Line by Line Text Compression +by *Takayuki Matsuoka* + +`blockStreaming_lineByLine.c` is Lizard Straming API example which implements line by line incremental (de)compression. + +Please note the following restrictions : + + - Firstly, read "Lizard Streaming API Basics". + - This is relatively advanced application example. + - Output file is not compatible with lizard_frame and platform dependent. + + +## What's the point of this example ? + + - Line by line incremental (de)compression. + - Handle huge file in small amount of memory + - Generally better compression ratio than Block API + - Non-uniform block size + + +## How the compression works + +First of all, allocate "Ring Buffer" for input and Lizard compressed data buffer for output. + +``` +(1) + Ring Buffer + + +--------+ + | Line#1 | + +---+----+ + | + v + {Out#1} + + +(2) + Prefix Mode Dependency + +----+ + | | + v | + +--------+-+------+ + | Line#1 | Line#2 | + +--------+---+----+ + | + v + {Out#2} + + +(3) + Prefix Prefix + +----+ +----+ + | | | | + v | v | + +--------+-+------+-+------+ + | Line#1 | Line#2 | Line#3 | + +--------+--------+---+----+ + | + v + {Out#3} + + +(4) + External Dictionary Mode + +----+ +----+ + | | | | + v | v | + ------+--------+-+------+-+--------+ + | .... | Line#X | Line#X+1 | + ------+--------+--------+-----+----+ + ^ | + | v + | {Out#X+1} + | + Reset + + +(5) + Prefix + +-----+ + | | + v | + ------+--------+--------+----------+--+-------+ + | .... | Line#X | Line#X+1 | Line#X+2 | + ------+--------+--------+----------+-----+----+ + ^ | + | v + | {Out#X+2} + | + Reset +``` + +Next (see (1)), read first line to ringbuffer and compress it by `Lizard_compress_continue()`. +For the first time, Lizard doesn't know any previous dependencies, +so it just compress the line without dependencies and generates compressed line {Out#1} to Lizard compressed data buffer. +After that, write {Out#1} to the file and forward ringbuffer offset. + +Do the same things to second line (see (2)). +But in this time, Lizard can use dependency to Line#1 to improve compression ratio. +This dependency is called "Prefix mode". + +Eventually, we'll reach end of ringbuffer at Line#X (see (4)). +This time, we should reset ringbuffer offset. +After resetting, at Line#X+1 pointer is not adjacent, but Lizard still maintain its memory. +This is called "External Dictionary Mode". + +In Line#X+2 (see (5)), finally Lizard forget almost all memories but still remains Line#X+1. +This is the same situation as Line#2. + +Continue these procedure to the end of text file. + + +## How the decompression works + +Decompression will do reverse order. + + - Read compressed line from the file to buffer. + - Decompress it to the ringbuffer. + - Output decompressed plain text line to the file. + - Forward ringbuffer offset. If offset exceedes end of the ringbuffer, reset it. + +Continue these procedure to the end of the compressed file. diff --git a/contrib/lizard/examples/blockStreaming_ringBuffer.c b/contrib/lizard/examples/blockStreaming_ringBuffer.c new file mode 100644 index 00000000000..011cd8b665b --- /dev/null +++ b/contrib/lizard/examples/blockStreaming_ringBuffer.c @@ -0,0 +1,202 @@ +// Lizard streaming API example : ring buffer +// Based on sample code from Takayuki Matsuoka + + +/************************************** + * Compiler Options + **************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define _CRT_SECURE_NO_WARNINGS // for MSVC +# define snprintf sprintf_s +#endif +#ifdef __GNUC__ +# pragma GCC diagnostic ignored "-Wmissing-braces" /* GCC bug 53119 : doesn't accept { 0 } as initializer (https://gcc.gnu.org/bugzilla/show_bug.cgi?id=53119) */ +#endif + + +/************************************** + * Includes + **************************************/ +#include +#include +#include +#include +#include "lizard_common.h" +#include "lizard_decompress.h" + + +enum { + MESSAGE_MAX_BYTES = 1024, + RING_BUFFER_BYTES = 1024 * 8 + MESSAGE_MAX_BYTES, + DECODE_RING_BUFFER = RING_BUFFER_BYTES + MESSAGE_MAX_BYTES // Intentionally larger, to test unsynchronized ring buffers +}; + + +size_t write_int32(FILE* fp, int32_t i) { + return fwrite(&i, sizeof(i), 1, fp); +} + +size_t write_bin(FILE* fp, const void* array, int arrayBytes) { + return fwrite(array, 1, arrayBytes, fp); +} + +size_t read_int32(FILE* fp, int32_t* i) { + return fread(i, sizeof(*i), 1, fp); +} + +size_t read_bin(FILE* fp, void* array, int arrayBytes) { + return fread(array, 1, arrayBytes, fp); +} + + +void test_compress(FILE* outFp, FILE* inpFp) +{ + Lizard_stream_t* lizardStream = Lizard_createStream_MinLevel(); + if (!lizardStream) return; + + static char inpBuf[RING_BUFFER_BYTES]; + int inpOffset = 0; + + for(;;) { + // Read random length ([1,MESSAGE_MAX_BYTES]) data to the ring buffer. + char* const inpPtr = &inpBuf[inpOffset]; + const int randomLength = (rand() % MESSAGE_MAX_BYTES) + 1; + const int inpBytes = (int) read_bin(inpFp, inpPtr, randomLength); + if (0 == inpBytes) break; + + { + char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; + const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, Lizard_compressBound(inpBytes)); + if(cmpBytes <= 0) break; + write_int32(outFp, cmpBytes); + write_bin(outFp, cmpBuf, cmpBytes); + + inpOffset += inpBytes; + + // Wraparound the ringbuffer offset + if(inpOffset >= RING_BUFFER_BYTES - MESSAGE_MAX_BYTES) inpOffset = 0; + } + } + + write_int32(outFp, 0); + Lizard_freeStream(lizardStream); +} + + +void test_decompress(FILE* outFp, FILE* inpFp) +{ + static char decBuf[DECODE_RING_BUFFER]; + int decOffset = 0; + Lizard_streamDecode_t lizardStreamDecode_body = { 0 }; + Lizard_streamDecode_t* lizardStreamDecode = &lizardStreamDecode_body; + + for(;;) { + int cmpBytes = 0; + char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; + + { + const size_t r0 = read_int32(inpFp, &cmpBytes); + if(r0 != 1 || cmpBytes <= 0) break; + + const size_t r1 = read_bin(inpFp, cmpBuf, cmpBytes); + if(r1 != (size_t) cmpBytes) break; + } + + { + char* const decPtr = &decBuf[decOffset]; + const int decBytes = Lizard_decompress_safe_continue( + lizardStreamDecode, cmpBuf, decPtr, cmpBytes, MESSAGE_MAX_BYTES); + if(decBytes <= 0) break; + decOffset += decBytes; + write_bin(outFp, decPtr, decBytes); + + // Wraparound the ringbuffer offset + if(decOffset >= DECODE_RING_BUFFER - MESSAGE_MAX_BYTES) decOffset = 0; + } + } +} + + +int compare(FILE* f0, FILE* f1) +{ + int result = 0; + + while(0 == result) { + char b0[65536]; + char b1[65536]; + const size_t r0 = fread(b0, 1, sizeof(b0), f0); + const size_t r1 = fread(b1, 1, sizeof(b1), f1); + + result = (int) r0 - (int) r1; + + if(0 == r0 || 0 == r1) { + break; + } + if(0 == result) { + result = memcmp(b0, b1, r0); + } + } + + return result; +} + + +int main(int argc, char** argv) +{ + char inpFilename[256] = { 0 }; + char lizardFilename[256] = { 0 }; + char decFilename[256] = { 0 }; + + if(argc < 2) { + printf("Please specify input filename\n"); + return 0; + } + + snprintf(inpFilename, 256, "%s", argv[1]); + snprintf(lizardFilename, 256, "%s.lizs-%d", argv[1], 0); + snprintf(decFilename, 256, "%s.lizs-%d.dec", argv[1], 0); + + printf("inp = [%s]\n", inpFilename); + printf("lizard = [%s]\n", lizardFilename); + printf("dec = [%s]\n", decFilename); + + // compress + { + FILE* inpFp = fopen(inpFilename, "rb"); + FILE* outFp = fopen(lizardFilename, "wb"); + + test_compress(outFp, inpFp); + + fclose(outFp); + fclose(inpFp); + } + + // decompress + { + FILE* inpFp = fopen(lizardFilename, "rb"); + FILE* outFp = fopen(decFilename, "wb"); + + test_decompress(outFp, inpFp); + + fclose(outFp); + fclose(inpFp); + } + + // verify + { + FILE* inpFp = fopen(inpFilename, "rb"); + FILE* decFp = fopen(decFilename, "rb"); + + const int cmp = compare(inpFp, decFp); + if(0 == cmp) { + printf("Verify : OK\n"); + } else { + printf("Verify : NG\n"); + } + + fclose(decFp); + fclose(inpFp); + } + + return 0; +} diff --git a/contrib/lizard/examples/compress_functions.c b/contrib/lizard/examples/compress_functions.c new file mode 100644 index 00000000000..36bc40e691a --- /dev/null +++ b/contrib/lizard/examples/compress_functions.c @@ -0,0 +1,303 @@ +/* + * compress_functions.c + * Copyright : Kyle Harper + * License : Follows same licensing as the lizard_compress.c/lizard_compress.h program at any given time. Currently, BSD 2. + * Description: A program to demonstrate the various compression functions involved in when using Lizard_compress_MinLevel(). The idea + * is to show how each step in the call stack can be used directly, if desired. There is also some benchmarking for + * each function to demonstrate the (probably lack of) performance difference when jumping the stack. + * (If you're new to lizard, please read simple_buffer.c to understand the fundamentals) + * + * The call stack (before theoretical compiler optimizations) for Lizard_compress_MinLevel is as follows: + * Lizard_compress_MinLevel + * Lizard_compress_fast + * Lizard_compress_extState_MinLevel + * Lizard_compress_generic + * + * Lizard_compress_MinLevel() + * This is the recommended function for compressing data. It will serve as the baseline for comparison. + * Lizard_compress_fast() + * Despite its name, it's not a "fast" version of compression. It simply decides if HEAPMODE is set and either + * allocates memory on the heap for a struct or creates the struct directly on the stack. Stack access is generally + * faster but this function itself isn't giving that advantage, it's just some logic for compile time. + * Lizard_compress_extState_MinLevel() + * This simply accepts all the pointers and values collected thus far and adds logic to determine how + * Lizard_compress_generic should be invoked; specifically: can the source fit into a single pass as determined by + * Lizard_64Klimit. + * Lizard_compress_generic() + * As the name suggests, this is the generic function that ultimately does most of the heavy lifting. Calling this + * directly can help avoid some test cases and branching which might be useful in some implementation-specific + * situations, but you really need to know what you're doing AND what you're asking lizard to do! You also need a + * wrapper function because this function isn't exposed with lizard_compress.h. + * + * The call stack for decompression functions is shallow. There are 2 options: + * Lizard_decompress_safe || Lizard_decompress_fast + * Lizard_decompress_generic + * + * Lizard_decompress_safe + * This is the recommended function for decompressing data. It is considered safe because the caller specifies + * both the size of the compresssed buffer to read as well as the maximum size of the output (decompressed) buffer + * instead of just the latter. + * Lizard_decompress_generic + * This is the generic function that both of the Lizard_decompress_* functions above end up calling. Calling this + * directly is not advised, period. Furthermore, it is a static inline function in lizard_compress.c, so there isn't a symbol + * exposed for anyone using lizard_compress.h to utilize. + * + * Special Note About Decompression: + * Using the Lizard_decompress_safe() function protects against malicious (user) input. + */ + +/* Since lizard compiles with c99 and not gnu/std99 we need to enable POSIX linking for time.h structs and functions. */ +#if __STDC_VERSION__ >= 199901L +#define _XOPEN_SOURCE 600 +#else +#define _XOPEN_SOURCE 500 +#endif +#define _POSIX_C_SOURCE 199309L + +/* Includes, for Power! */ +#include "lizard_compress.h" +#include "lizard_decompress.h" +#include /* for printf() */ +#include /* for exit() */ +#include /* for atoi() memcmp() */ +#include /* for uint_types */ +#include /* for PRIu64 */ +#include /* for clock_gettime() */ +#include /* for setlocale() */ + +/* We need to know what one billion is for clock timing. */ +#define BILLION 1000000000L + +/* Create a crude set of test IDs so we can switch on them later (Can't switch() on a char[] or char*). */ +#define ID__LIZARD_COMPRESS_DEFAULT 1 +#define ID__LIZARD_COMPRESS_GENERIC 4 +#define ID__LIZARD_DECOMPRESS_SAFE 5 + + + +/* + * Easy show-error-and-bail function. + */ +void run_screaming(const char *message, const int code) { + printf("%s\n", message); + exit(code); + return; +} + + +/* + * Centralize the usage function to keep main cleaner. + */ +void usage(const char *message) { + printf("Usage: ./argPerformanceTesting \n"); + run_screaming(message, 1); + return; +} + + + +/* + * Runs the benchmark for Lizard_compress_* based on function_id. + */ +uint64_t bench( + const char *known_good_dst, + const int function_id, + const int iterations, + const char *src, + char *dst, + const size_t src_size, + const size_t max_dst_size, + const size_t comp_size + ) { + uint64_t time_taken = 0; + int rv = 0; + const int warm_up = 5000; + struct timespec start, end; + Lizard_stream_t* state = Lizard_createStream_MinLevel(); + if (!state) return; + + // Select the right function to perform the benchmark on. We perform 5000 initial loops to warm the cache and ensure that dst + // remains matching to known_good_dst between successive calls. + switch(function_id) { + case ID__LIZARD_COMPRESS_DEFAULT: + printf("Starting benchmark for function: Lizard_compress_MinLevel()\n"); + for(int junk=0; junk 1) + iterations = atoi(argv[1]); + if (iterations < 1) + usage("Argument 1 (iterations) must be > 0."); + + // First we will create 2 sources (char *) of 2000 bytes each. One normal text, the other highly-compressible text. + const char *src = "Lorem ipsum dolor sit amet, consectetur adipiscing elit. Sed luctus purus et risus vulputate, et mollis orci ullamcorper. Nulla facilisi. Fusce in ligula sed purus varius aliquet interdum vitae justo. Proin quis diam velit. Nulla varius iaculis auctor. Cras volutpat, justo eu dictum pulvinar, elit sem porttitor metus, et imperdiet metus sapien et ante. Nullam nisi nulla, ornare eu tristique eu, dignissim vitae diam. Nulla sagittis porta libero, a accumsan felis sagittis scelerisque. Integer laoreet eleifend congue. Etiam rhoncus leo vel dolor fermentum, quis luctus nisl iaculis. Praesent a erat sapien. Aliquam semper mi in lorem ultrices ultricies. Lorem ipsum dolor sit amet, consectetur adipiscing elit. In feugiat risus sed enim ultrices, at sodales nulla tristique. Maecenas eget pellentesque justo, sed pellentesque lectus. Fusce sagittis sit amet elit vel varius. Donec sed ligula nec ligula vulputate rutrum sed ut lectus. Etiam congue pharetra leo vitae cursus. Morbi enim ante, porttitor ut varius vel, tincidunt quis justo. Nunc iaculis, risus id ultrices semper, metus est efficitur ligula, vel posuere risus nunc eget purus. Ut lorem turpis, condimentum at sem sed, porta aliquam turpis. In ut sapien a nulla dictum tincidunt quis sit amet lorem. Fusce at est egestas, luctus neque eu, consectetur tortor. Phasellus eleifend ultricies nulla ac lobortis. Morbi maximus quam cursus vehicula iaculis. Maecenas cursus vel justo ut rutrum. Curabitur magna orci, dignissim eget dapibus vitae, finibus id lacus. Praesent rhoncus mattis augue vitae bibendum. Praesent porta mauris non ultrices fermentum. Quisque vulputate ipsum in sodales pulvinar. Aliquam nec mollis felis. Donec vitae augue pulvinar, congue nisl sed, pretium purus. Fusce lobortis mi ac neque scelerisque semper. Pellentesque vel est vitae magna aliquet aliquet. Nam non dolor. Nulla facilisi. Class aptent taciti sociosqu ad litora torquent per conubia nostra, per inceptos himenaeos. Morbi ac lacinia felis metus."; + const char *hc_src = "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"; + // Set and derive sizes. Since we're using strings, use strlen() + 1 for \0. + const size_t src_size = strlen(src) + 1; + const size_t max_dst_size = Lizard_compressBound(src_size); + int bytes_returned = 0; + // Now build allocations for the data we'll be playing with. + char *dst = calloc(1, max_dst_size); + char *known_good_dst = calloc(1, max_dst_size); + char *known_good_hc_dst = calloc(1, max_dst_size); + if (dst == NULL || known_good_dst == NULL || known_good_hc_dst == NULL) + run_screaming("Couldn't allocate memory for the destination buffers. Sad :(", 1); + + // Create known-good buffers to verify our tests with other functions will produce the same results. + bytes_returned = Lizard_compress_MinLevel(src, known_good_dst, src_size, max_dst_size); + if (bytes_returned < 1) + run_screaming("Couldn't create a known-good destination buffer for comparison... this is bad.", 1); + const size_t src_comp_size = bytes_returned; + bytes_returned = Lizard_compress_MinLevel(hc_src, known_good_hc_dst, src_size, max_dst_size); + if (bytes_returned < 1) + run_screaming("Couldn't create a known-good (highly compressible) destination buffer for comparison... this is bad.", 1); + const size_t hc_src_comp_size = bytes_returned; + + + /* Lizard_compress_MinLevel() */ + // This is the default function so we don't need to demonstrate how to use it. See basics.c if you need more basal information. + + /* Lizard_compress_extState_MinLevel() */ + // Using this function directly requires that we build an Lizard_stream_t struct ourselves. We do NOT have to reset it ourselves. + memset(dst, 0, max_dst_size); + Lizard_stream_t* state = Lizard_createStream_MinLevel(); + if (!state) return; + bytes_returned = Lizard_compress_extState_MinLevel(state, src, dst, src_size, max_dst_size, 1); + if (bytes_returned < 1) + run_screaming("Failed to compress src using Lizard_compress_extState_MinLevel. echo $? for return code.", bytes_returned); + if (memcmp(dst, known_good_dst, bytes_returned) != 0) + run_screaming("According to memcmp(), the value we got in dst from Lizard_compress_extState_MinLevel doesn't match the known-good value. This is bad.", 1); + + /* Lizard_compress_generic */ + // When you can exactly control the inputs and options of your Lizard needs, you can use Lizard_compress_generic and fixed (const) + // values for the enum types such as dictionary and limitations. Any other direct-use is probably a bad idea. + // + // That said, the Lizard_compress_generic() function is 'static inline' and does not have a prototype in lizard_compress.h to expose a symbol + // for it. In other words: we can't access it directly. I don't want to submit a PR that modifies lizard_compress.c/h. Yann and others can + // do that if they feel it's worth expanding this example. + // + // I will, however, leave a skeleton of what would be required to use it directly: + /* + memset(dst, 0, max_dst_size); + // Lizard_stream_t state: is already declared above. We can reuse it BUT we have to reset the stream ourselves between each call. + Lizard_resetStream_MinLevel((Lizard_stream_t *)state); + // Since src size is small we know the following enums will be used: notLimited (0), byU16 (2), noDict (0), noDictIssue (0). + bytes_returned = Lizard_compress_generic(state, src, dst, src_size, max_dst_size, notLimited, byU16, noDict, noDictIssue, 1); + if (bytes_returned < 1) + run_screaming("Failed to compress src using Lizard_compress_generic. echo $? for return code.", bytes_returned); + if (memcmp(dst, known_good_dst, bytes_returned) != 0) + run_screaming("According to memcmp(), the value we got in dst from Lizard_compress_generic doesn't match the known-good value. This is bad.", 1); + */ + Lizard_freeStream(state); + + + /* Benchmarking */ + /* Now we'll run a few rudimentary benchmarks with each function to demonstrate differences in speed based on the function used. + * Remember, we cannot call Lizard_compress_generic() directly (yet) so it's disabled. + */ + // Suite A - Normal Compressibility + char *dst_d = calloc(1, src_size); + memset(dst, 0, max_dst_size); + printf("\nStarting suite A: Normal compressible text.\n"); + uint64_t time_taken__default = bench(known_good_dst, ID__LIZARD_COMPRESS_DEFAULT, iterations, src, dst, src_size, max_dst_size, src_comp_size); + //uint64_t time_taken__generic = bench(known_good_dst, ID__LIZARD_COMPRESS_GENERIC, iterations, src, dst, src_size, max_dst_size, src_comp_size); + uint64_t time_taken__decomp_safe = bench(src, ID__LIZARD_DECOMPRESS_SAFE, iterations, known_good_dst, dst_d, src_size, max_dst_size, src_comp_size); + // Suite B - Highly Compressible + memset(dst, 0, max_dst_size); + printf("\nStarting suite B: Highly compressible text.\n"); + uint64_t time_taken_hc__default = bench(known_good_hc_dst, ID__LIZARD_COMPRESS_DEFAULT, iterations, hc_src, dst, src_size, max_dst_size, hc_src_comp_size); + //uint64_t time_taken_hc__generic = bench(known_good_hc_dst, ID__LIZARD_COMPRESS_GENERIC, iterations, hc_src, dst, src_size, max_dst_size, hc_src_comp_size); + uint64_t time_taken_hc__decomp_safe = bench(hc_src, ID__LIZARD_DECOMPRESS_SAFE, iterations, known_good_hc_dst, dst_d, src_size, max_dst_size, hc_src_comp_size); + + // Report and leave. + setlocale(LC_ALL, ""); + const char *format = "|%-14s|%-30s|%'14.9f|%'16d|%'14d|%'13.2f%%|\n"; + const char *header_format = "|%-14s|%-30s|%14s|%16s|%14s|%14s|\n"; + const char *separator = "+--------------+------------------------------+--------------+----------------+--------------+--------------+\n"; + printf("\n"); + printf("%s", separator); + printf(header_format, "Source", "Function Benchmarked", "Total Seconds", "Iterations/sec", "ns/Iteration", "% of default"); + printf("%s", separator); + printf(format, "Normal Text", "Lizard_compress_MinLevel()", (double)time_taken__default / BILLION, (int)(iterations / ((double)time_taken__default /BILLION)), time_taken__default / iterations, (double)time_taken__default * 100 / time_taken__default); + printf(format, "Normal Text", "Lizard_compress_fast()", (double)time_taken__fast / BILLION, (int)(iterations / ((double)time_taken__fast /BILLION)), time_taken__fast / iterations, (double)time_taken__fast * 100 / time_taken__default); + printf(format, "Normal Text", "Lizard_compress_extState_MinLevel()", (double)time_taken__fast_extstate / BILLION, (int)(iterations / ((double)time_taken__fast_extstate /BILLION)), time_taken__fast_extstate / iterations, (double)time_taken__fast_extstate * 100 / time_taken__default); + //printf(format, "Normal Text", "Lizard_compress_generic()", (double)time_taken__generic / BILLION, (int)(iterations / ((double)time_taken__generic /BILLION)), time_taken__generic / iterations, (double)time_taken__generic * 100 / time_taken__default); + printf(format, "Normal Text", "Lizard_decompress_safe()", (double)time_taken__decomp_safe / BILLION, (int)(iterations / ((double)time_taken__decomp_safe /BILLION)), time_taken__decomp_safe / iterations, (double)time_taken__decomp_safe * 100 / time_taken__default); + printf(header_format, "", "", "", "", "", ""); + printf(format, "Compressible", "Lizard_compress_MinLevel()", (double)time_taken_hc__default / BILLION, (int)(iterations / ((double)time_taken_hc__default /BILLION)), time_taken_hc__default / iterations, (double)time_taken_hc__default * 100 / time_taken_hc__default); + printf(format, "Compressible", "Lizard_compress_fast()", (double)time_taken_hc__fast / BILLION, (int)(iterations / ((double)time_taken_hc__fast /BILLION)), time_taken_hc__fast / iterations, (double)time_taken_hc__fast * 100 / time_taken_hc__default); + printf(format, "Compressible", "Lizard_compress_extState_MinLevel()", (double)time_taken_hc__fast_extstate / BILLION, (int)(iterations / ((double)time_taken_hc__fast_extstate /BILLION)), time_taken_hc__fast_extstate / iterations, (double)time_taken_hc__fast_extstate * 100 / time_taken_hc__default); + //printf(format, "Compressible", "Lizard_compress_generic()", (double)time_taken_hc__generic / BILLION, (int)(iterations / ((double)time_taken_hc__generic /BILLION)), time_taken_hc__generic / iterations, (double)time_taken_hc__generic * 100 / time_taken_hc__default); + printf(format, "Compressible", "Lizard_decompress_safe()", (double)time_taken_hc__decomp_safe / BILLION, (int)(iterations / ((double)time_taken_hc__decomp_safe /BILLION)), time_taken_hc__decomp_safe / iterations, (double)time_taken_hc__decomp_safe * 100 / time_taken_hc__default); + printf("%s", separator); + printf("\n"); + printf("All done, ran %d iterations per test.\n", iterations); + return 0; +} diff --git a/contrib/lizard/examples/frameCompress.c b/contrib/lizard/examples/frameCompress.c new file mode 100644 index 00000000000..37ea0aa4fa8 --- /dev/null +++ b/contrib/lizard/examples/frameCompress.c @@ -0,0 +1,169 @@ +// Lizardframe API example : compress a file +// Based on sample code from Zbigniew Jędrzejewski-Szmek + +#include +#include +#include +#include + +#include + +#define BUF_SIZE (16*1024) +#define LIZARD_HEADER_SIZE 19 +#define LIZARD_FOOTER_SIZE 4 + +static const LizardF_preferences_t lizard_preferences = { + { LizardF_max256KB, LizardF_blockLinked, LizardF_noContentChecksum, LizardF_frame, 0, { 0, 0 } }, + 0, /* compression level */ + 0, /* autoflush */ + { 0, 0, 0, 0 }, /* reserved, must be set to 0 */ +}; + +static int compress_file(FILE *in, FILE *out, size_t *size_in, size_t *size_out) { + LizardF_errorCode_t r; + LizardF_compressionContext_t ctx; + char *src, *buf = NULL; + size_t size, n, k, count_in = 0, count_out, offset = 0, frame_size; + + r = LizardF_createCompressionContext(&ctx, LIZARDF_VERSION); + if (LizardF_isError(r)) { + printf("Failed to create context: error %zu", r); + return 1; + } + r = 1; + + src = malloc(BUF_SIZE); + if (!src) { + printf("Not enough memory"); + goto cleanup; + } + + frame_size = LizardF_compressBound(BUF_SIZE, &lizard_preferences); + size = frame_size + LIZARD_HEADER_SIZE + LIZARD_FOOTER_SIZE; + buf = malloc(size); + if (!buf) { + printf("Not enough memory"); + goto cleanup; + } + + n = offset = count_out = LizardF_compressBegin(ctx, buf, size, &lizard_preferences); + if (LizardF_isError(n)) { + printf("Failed to start compression: error %zu", n); + goto cleanup; + } + + printf("Buffer size is %zu bytes, header size %zu bytes\n", size, n); + + for (;;) { + k = fread(src, 1, BUF_SIZE, in); + if (k == 0) + break; + count_in += k; + + n = LizardF_compressUpdate(ctx, buf + offset, size - offset, src, k, NULL); + if (LizardF_isError(n)) { + printf("Compression failed: error %zu", n); + goto cleanup; + } + + offset += n; + count_out += n; + if (size - offset < frame_size + LIZARD_FOOTER_SIZE) { + printf("Writing %zu bytes\n", offset); + + k = fwrite(buf, 1, offset, out); + if (k < offset) { + if (ferror(out)) + printf("Write failed"); + else + printf("Short write"); + goto cleanup; + } + + offset = 0; + } + } + + n = LizardF_compressEnd(ctx, buf + offset, size - offset, NULL); + if (LizardF_isError(n)) { + printf("Failed to end compression: error %zu", n); + goto cleanup; + } + + offset += n; + count_out += n; + printf("Writing %zu bytes\n", offset); + + k = fwrite(buf, 1, offset, out); + if (k < offset) { + if (ferror(out)) + printf("Write failed"); + else + printf("Short write"); + goto cleanup; + } + + *size_in = count_in; + *size_out = count_out; + r = 0; + cleanup: + if (ctx) + LizardF_freeCompressionContext(ctx); + free(src); + free(buf); + return r; +} + +static int compress(const char *input, const char *output) { + char *tmp = NULL; + FILE *in = NULL, *out = NULL; + size_t size_in = 0, size_out = 0; + int r = 1; + + if (!output) { + size_t len = strlen(input); + + output = tmp = malloc(len + 5); + if (!tmp) { + printf("Not enough memory"); + return 1; + } + strcpy(tmp, input); + strcpy(tmp + len, ".liz"); + } + + in = fopen(input, "rb"); + if (!in) { + fprintf(stderr, "Failed to open input file %s: %s\n", input, strerror(errno)); + goto cleanup; + } + + out = fopen(output, "wb"); + if (!out) { + fprintf(stderr, "Failed to open output file %s: %s\n", output, strerror(errno)); + goto cleanup; + } + + r = compress_file(in, out, &size_in, &size_out); + if (r == 0) + printf("%s: %zu → %zu bytes, %.1f%%\n", + input, size_in, size_out, + (double)size_out / size_in * 100); + cleanup: + if (in) + fclose(in); + if (out) + fclose(out); + free(tmp); + return r; +} + + +int main(int argc, char **argv) { + if (argc < 2 || argc > 3) { + fprintf(stderr, "Syntax: %s \n", argv[0]); + return EXIT_FAILURE; + } + + return compress(argv[1], argv[2]); +} diff --git a/contrib/lizard/examples/printVersion.c b/contrib/lizard/examples/printVersion.c new file mode 100644 index 00000000000..9818fdd98cf --- /dev/null +++ b/contrib/lizard/examples/printVersion.c @@ -0,0 +1,13 @@ +// Lizard trivial example : print Library version number +// Copyright : Takayuki Matsuoka & Yann Collet + + +#include +#include "lizard_compress.h" + +int main(int argc, char** argv) +{ + (void)argc; (void)argv; + printf("Hello World ! Lizard Library version = %d\n", Lizard_versionNumber()); + return 0; +} diff --git a/contrib/lizard/examples/simple_buffer.c b/contrib/lizard/examples/simple_buffer.c new file mode 100644 index 00000000000..154d3b2cb76 --- /dev/null +++ b/contrib/lizard/examples/simple_buffer.c @@ -0,0 +1,91 @@ +/* + * simple_buffer.c + * Copyright : Kyle Harper + * License : Follows same licensing as the lizard_compress.c/lizard_compress.h program at any given time. Currently, BSD 2. + * Description: Example program to demonstrate the basic usage of the compress/decompress functions within lizard_compress.c/lizard_compress.h. + * The functions you'll likely want are Lizard_compress_MinLevel and Lizard_decompress_safe. Both of these are documented in + * the lizard_compress.h header file; I recommend reading them. + */ + +/* Includes, for Power! */ +#include "lizard_compress.h" // This is all that is required to expose the prototypes for basic compression and decompression. +#include "lizard_decompress.h" +#include // For printf() +#include // For memcmp() +#include // For exit() + +/* + * Easy show-error-and-bail function. + */ +void run_screaming(const char *message, const int code) { + printf("%s\n", message); + exit(code); + return; +} + + +/* + * main + */ +int main(void) { + /* Introduction */ + // Below we will have a Compression and Decompression section to demonstrate. There are a few important notes before we start: + // 1) The return codes of Lizard_ functions are important. Read lizard_compress.h if you're unsure what a given code means. + // 2) Lizard uses char* pointers in all Lizard_ functions. This is baked into the API and probably not going to change. If your + // program uses pointers that are unsigned char*, void*, or otherwise different you may need to do some casting or set the + // right -W compiler flags to ignore those warnings (e.g.: -Wno-pointer-sign). + + /* Compression */ + // We'll store some text into a variable pointed to by *src to be compressed later. + const char *src = "Lorem ipsum dolor sit amet, consectetur adipiscing elit."; + // The compression function needs to know how many bytes of exist. Since we're using a string, we can use strlen() + 1 (for \0). + const size_t src_size = strlen(src) + 1; + // Lizard provides a function that will tell you the maximum size of compressed output based on input data via Lizard_compressBound(). + const size_t max_dst_size = Lizard_compressBound(src_size); + // We will use that size for our destination boundary when allocating space. + char *compressed_data = malloc(max_dst_size); + if (compressed_data == NULL) + run_screaming("Failed to allocate memory for *compressed_data.", 1); + // That's all the information and preparation Lizard needs to compress *src into *compressed_data. Invoke Lizard_compress_MinLevel now + // with our size values and pointers to our memory locations. Save the return value for error checking. + int return_value = 0; + return_value = Lizard_compress_MinLevel(src, compressed_data, src_size, max_dst_size); + // Check return_value to determine what happened. + if (return_value < 0) + run_screaming("A negative result from Lizard_compress_MinLevel indicates a failure trying to compress the data. See exit code (echo $?) for value returned.", return_value); + if (return_value == 0) + run_screaming("A result of 0 means compression worked, but was stopped because the destination buffer couldn't hold all the information.", 1); + if (return_value > 0) + printf("We successfully compressed some data!\n"); + // Not only does a positive return_value mean success, the value returned == the number of bytes required. You can use this to + // realloc() *compress_data to free up memory, if desired. We'll do so just to demonstrate the concept. + const size_t compressed_data_size = return_value; + compressed_data = (char *)realloc(compressed_data, compressed_data_size); + if (compressed_data == NULL) + run_screaming("Failed to re-alloc memory for compressed_data. Sad :(", 1); + + /* Decompression */ + // Now that we've successfully compressed the information from *src to *compressed_data, let's do the opposite! We'll create a + // *new_src location of size src_size since we know that value. + char *new_src = malloc(src_size); + if (new_src == NULL) + run_screaming("Failed to allocate memory for *new_src.", 1); + // The Lizard_decompress_safe function needs to know where the compressed data is, how many bytes long it is, where the new_src + // memory location is, and how large the new_src (uncompressed) output will be. Again, save the return_value. + return_value = Lizard_decompress_safe(compressed_data, new_src, compressed_data_size, src_size); + if (return_value < 0) + run_screaming("A negative result from Lizard_decompress_fast indicates a failure trying to decompress the data. See exit code (echo $?) for value returned.", return_value); + if (return_value == 0) + run_screaming("I'm not sure this function can ever return 0. Documentation in lizard_compress.h doesn't indicate so.", 1); + if (return_value > 0) + printf("We successfully decompressed some data!\n"); + // Not only does a positive return value mean success, the value returned == the number of bytes read from the compressed_data + // stream. I'm not sure there's ever a time you'll need to know this in most cases... + + /* Validation */ + // We should be able to compare our original *src with our *new_src and be byte-for-byte identical. + if (memcmp(src, new_src, src_size) != 0) + run_screaming("Validation failed. *src and *new_src are not identical.", 1); + printf("Validation done. The string we ended up with is:\n%s\n", new_src); + return 0; +} diff --git a/contrib/lizard/examples/streaming_api_basics.md b/contrib/lizard/examples/streaming_api_basics.md new file mode 100644 index 00000000000..85c06803455 --- /dev/null +++ b/contrib/lizard/examples/streaming_api_basics.md @@ -0,0 +1,87 @@ +# Lizard Streaming API Basics +by *Takayuki Matsuoka* +## Lizard API sets + +Lizard has the following API sets : + + - "Auto Framing" API (lizard_frame.h) : + This is most recommended API for usual application. + It guarantees interoperability with other Lizard framing format compliant tools/libraries + such as Lizard command line utility, node-lizard, etc. + - "Block" API : This is recommended for simple purpose. + It compress single raw memory block to Lizard memory block and vice versa. + - "Streaming" API : This is designed for complex thing. + For example, compress huge stream data in restricted memory environment. + +Basically, you should use "Auto Framing" API. +But if you want to write advanced application, it's time to use Block or Streaming APIs. + + +## What is difference between Block and Streaming API ? + +Block API (de)compresses single contiguous memory block. +In other words, Lizard library find redundancy from single contiguous memory block. +Streaming API does same thing but (de)compress multiple adjacent contiguous memory block. +So Lizard library could find more redundancy than Block API. + +The following figure shows difference between API and block sizes. +In these figures, original data is splitted to 4KiBytes contiguous chunks. + +``` +Original Data + +---------------+---------------+----+----+----+ + | 4KiB Chunk A | 4KiB Chunk B | C | D |... | + +---------------+---------------+----+----+----+ + +Example (1) : Block API, 4KiB Block + +---------------+---------------+----+----+----+ + | 4KiB Chunk A | 4KiB Chunk B | C | D |... | + +---------------+---------------+----+----+----+ + | Block #1 | Block #2 | #3 | #4 |... | + +---------------+---------------+----+----+----+ + + (No Dependency) + + +Example (2) : Block API, 8KiB Block + +---------------+---------------+----+----+----+ + | 4KiB Chunk A | 4KiB Chunk B | C | D |... | + +---------------+---------------+----+----+----+ + | Block #1 |Block #2 |... | + +--------------------+----------+-------+-+----+ + ^ | ^ | + | | | | + +--------------+ +----+ + Internal Dependency Internal Dependency + + +Example (3) : Streaming API, 4KiB Block + +---------------+---------------+-----+----+----+ + | 4KiB Chunk A | 4KiB Chunk B | C | D |... | + +---------------+---------------+-----+----+----+ + | Block #1 | Block #2 | #3 | #4 |... | + +---------------+----+----------+-+---+-+--+----+ + ^ | ^ | ^ | + | | | | | | + +--------------+ +--------+ +---+ + Dependency Dependency Dependency +``` + + - In example (1), there is no dependency. + All blocks are compressed independently. + - In example (2), naturally 8KiBytes block has internal dependency. + But still block #1 and #2 are compressed independently. + - In example (3), block #2 has dependency to #1, + also #3 has dependency to #2 and #1, #4 has #3, #2 and #1, and so on. + +Here, we can observe difference between example (2) and (3). +In (2), there's no dependency between chunk B and C, but (3) has dependency between B and C. +This dependency improves compression ratio. + + +## Restriction of Streaming API + +For the efficiency, Streaming API doesn't keep mirror copy of dependent (de)compressed memory. +This means users should keep these dependent (de)compressed memory explicitly. +Usually, "Dependent memory" is previous adjacent contiguous memory up to 64KiBytes. +Lizard will not access further memories. diff --git a/contrib/lizard/lib/.gitignore b/contrib/lizard/lib/.gitignore new file mode 100644 index 00000000000..6eedf69343a --- /dev/null +++ b/contrib/lizard/lib/.gitignore @@ -0,0 +1,3 @@ +# make install artefact +liblizard.pc +test/ diff --git a/contrib/lizard/lib/LICENSE b/contrib/lizard/lib/LICENSE new file mode 100644 index 00000000000..b335b0bafcf --- /dev/null +++ b/contrib/lizard/lib/LICENSE @@ -0,0 +1,25 @@ +Lizard Library +Copyright (C) 2011-2016, Yann Collet. +Copyright (C) 2016-2017, Przemyslaw Skibinski +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/lizard/lib/README.md b/contrib/lizard/lib/README.md new file mode 100644 index 00000000000..2775b984d63 --- /dev/null +++ b/contrib/lizard/lib/README.md @@ -0,0 +1,76 @@ +Lizard - Library Files +================================ + +The __lib__ directory contains several directories. +Depending on target use case, it's enough to include only files from relevant directories. + + +#### API + +Lizard stable API is exposed within [lizard_compress.h](lizard_compress.h) and [lizard_decompress.h](lizard_decompress.h), +at the root of `lib` directory. + + +#### Compatibility issues + +The raw Lizard block compression format is detailed within [lizard_Block_format]. +To compress an arbitrarily long file or data stream, multiple blocks are required. +Organizing these blocks and providing a common header format to handle their content +is the purpose of the Frame format, defined in [lizard_Frame_format]. +`lizard` command line utility produces files or streams compatible with the Frame format. +(_Advanced stuff_ : It's possible to hide xxhash symbols into a local namespace. +This is what `liblizard` does, to avoid symbol duplication +in case a user program would link to several libraries containing xxhash symbols.) + +[lizard_Block_format]: ../doc/lizard_Block_format.md +[lizard_Frame_format]: ../doc/lizard_Frame_format.md + + +#### Various Lizard builds + +Files `lizard_common.h`, `lizard_compress*`, `lizard_parser_*.h`, `lizard_decompress*`, and `entropy\mem.h` are required in all circumstances. + +To compile: +- Lizard_raw only with levels 10...29 : use the `-DLIZARD_NO_HUFFMAN` compiler flag +- Lizard_raw with levels 10...49 : include also all files from `entropy` directory +- Lizard_frame with levels 10...49 : `lizard_frame*` and all files from `entropy` and `xxhash` directories + + +#### Advanced API + +A more complex `lizard_frame_static.h` is also provided. +It contains definitions which are not guaranteed to remain stable within future versions. +It must be used with static linking ***only***. + + +#### Using MinGW+MSYS to create DLL + +DLL can be created using MinGW+MSYS with the `make liblizard` command. +This command creates `dll\liblizard.dll` and the import library `dll\liblizard.lib`. +The import library is only required with Visual C++. +The header files `lizard.h`, `lizardhc.h`, `lizard_frame.h` and the dynamic library +`dll\liblizard.dll` are required to compile a project using gcc/MinGW. +The dynamic library has to be added to linking options. +It means that if a project that uses Lizard consists of a single `test-dll.c` +file it should be compiled with `liblizard.lib`. For example: +``` + gcc $(CFLAGS) -Iinclude/ test-dll.c -o test-dll dll\liblizard.dll +``` +The compiled executable will require Lizard DLL which is available at `dll\liblizard.dll`. + + +#### Miscellaneous + +Other files present in the directory are not source code. There are : + + - LICENSE : contains the BSD license text + - Makefile : script to compile or install lizard library (static or dynamic) + - liblizard.pc.in : for pkg-config (make install) + - README.md : this file + + +#### License + +All source material within __lib__ directory are BSD 2-Clause licensed. +See [LICENSE](LICENSE) for details. +The license is also repeated at the top of each source file. diff --git a/contrib/lizard/lib/dll/liblizard.def b/contrib/lizard/lib/dll/liblizard.def new file mode 100644 index 00000000000..73ebc44570a --- /dev/null +++ b/contrib/lizard/lib/dll/liblizard.def @@ -0,0 +1,19 @@ +LIBRARY liblizard.dll +EXPORTS + Lizard_compress + Lizard_compressBound + Lizard_compress_continue + Lizard_compress_extState + Lizard_createStream + Lizard_createStreamDecode + Lizard_decompress_safe + Lizard_decompress_safe_continue + Lizard_decompress_safe_partial + Lizard_decompress_safe_usingDict + Lizard_freeStream + Lizard_freeStreamDecode + Lizard_loadDict + Lizard_resetStream + Lizard_saveDict + Lizard_setStreamDecode + Lizard_sizeofState diff --git a/contrib/lizard/lib/entropy/README.md b/contrib/lizard/lib/entropy/README.md new file mode 100644 index 00000000000..5b18f20b063 --- /dev/null +++ b/contrib/lizard/lib/entropy/README.md @@ -0,0 +1,38 @@ +New Generation Entropy library +============================== + +The __lib__ directory contains several files, but you don't necessarily want them all. +Here is a detailed list, to help you decide which one you need : + + +#### Compulsory files + +These files are required in all circumstances : +- __error_public.h__ : error list as enum +- __error_private.h__ : error management +- __mem.h__ : low level memory access routines +- __bitstream.h__ : generic read/write bitstream common to all entropy codecs +- __entropy_common.c__ : common functions needed for both compression and decompression + + +#### Finite State Entropy + +This is the base codec required by other ones. +It implements a tANS variant, similar to arithmetic in compression performance, but much faster. Compression and decompression can be compiled independently. +- __fse.h__ : exposes interfaces +- __fse_compress.c__ : implements compression codec +- __fse_decompress.c__ : implements decompression codec + + +#### FSE 16-bits symbols version + +This codec is able to encode alphabets of size > 256, using 2 bytes per symbol. It requires the base FSE codec to compile properly. Compression and decompression are merged in the same file. +- __fseU16.c__ implements the codec, while __fseU16.h__ exposes its interfaces. + + +#### Huffman codec + +This is the fast huffman codec. It requires the base FSE codec to compress its headers. Compression and decompression can be compiled independently. +- __huf.h__ : exposes interfaces. +- __huf_compress.c__ : implements compression codec +- __huf_decompress.c__ : implements decompression codec diff --git a/contrib/lizard/lib/entropy/bitstream.h b/contrib/lizard/lib/entropy/bitstream.h new file mode 100644 index 00000000000..849a9e6018a --- /dev/null +++ b/contrib/lizard/lib/entropy/bitstream.h @@ -0,0 +1,414 @@ +/* ****************************************************************** + bitstream + Part of FSE library + header file (to include) + Copyright (C) 2013-2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Source repository : https://github.com/Cyan4973/FiniteStateEntropy +****************************************************************** */ +#ifndef BITSTREAM_H_MODULE +#define BITSTREAM_H_MODULE + +#if defined (__cplusplus) +extern "C" { +#endif + + +/* +* This API consists of small unitary functions, which must be inlined for best performance. +* Since link-time-optimization is not available for all compilers, +* these functions are defined into a .h to be included. +*/ + +/*-**************************************** +* Dependencies +******************************************/ +#include "mem.h" /* unaligned access routines */ +#include "error_private.h" /* error codes and messages */ + + +/*========================================= +* Target specific +=========================================*/ +#if defined(__BMI__) && defined(__GNUC__) +# include /* support for bextr (experimental) */ +#endif + + +/*-****************************************** +* bitStream encoding API (write forward) +********************************************/ +/* bitStream can mix input from multiple sources. +* A critical property of these streams is that they encode and decode in **reverse** direction. +* So the first bit sequence you add will be the last to be read, like a LIFO stack. +*/ +typedef struct +{ + size_t bitContainer; + int bitPos; + char* startPtr; + char* ptr; + char* endPtr; +} BIT_CStream_t; + +MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* dstBuffer, size_t dstCapacity); +MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits); +MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC); +MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC); + +/* Start with initCStream, providing the size of buffer to write into. +* bitStream will never write outside of this buffer. +* `dstCapacity` must be >= sizeof(bitD->bitContainer), otherwise @return will be an error code. +* +* bits are first added to a local register. +* Local register is size_t, hence 64-bits on 64-bits systems, or 32-bits on 32-bits systems. +* Writing data into memory is an explicit operation, performed by the flushBits function. +* Hence keep track how many bits are potentially stored into local register to avoid register overflow. +* After a flushBits, a maximum of 7 bits might still be stored into local register. +* +* Avoid storing elements of more than 24 bits if you want compatibility with 32-bits bitstream readers. +* +* Last operation is to close the bitStream. +* The function returns the final size of CStream in bytes. +* If data couldn't fit into `dstBuffer`, it will return a 0 ( == not storable) +*/ + + +/*-******************************************** +* bitStream decoding API (read backward) +**********************************************/ +typedef struct +{ + size_t bitContainer; + unsigned bitsConsumed; + const char* ptr; + const char* start; +} BIT_DStream_t; + +typedef enum { BIT_DStream_unfinished = 0, + BIT_DStream_endOfBuffer = 1, + BIT_DStream_completed = 2, + BIT_DStream_overflow = 3 } BIT_DStream_status; /* result of BIT_reloadDStream() */ + /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */ + +MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize); +MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits); +MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD); +MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD); + + +/* Start by invoking BIT_initDStream(). +* A chunk of the bitStream is then stored into a local register. +* Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t). +* You can then retrieve bitFields stored into the local register, **in reverse order**. +* Local register is explicitly reloaded from memory by the BIT_reloadDStream() method. +* A reload guarantee a minimum of ((8*sizeof(bitD->bitContainer))-7) bits when its result is BIT_DStream_unfinished. +* Otherwise, it can be less than that, so proceed accordingly. +* Checking if DStream has reached its end can be performed with BIT_endOfDStream(). +*/ + + +/*-**************************************** +* unsafe API +******************************************/ +MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC, size_t value, unsigned nbBits); +/* faster, but works only if value is "clean", meaning all high bits above nbBits are 0 */ + +MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC); +/* unsafe version; does not check buffer overflow */ + +MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits); +/* faster, but works only if nbBits >= 1 */ + + + +/*-************************************************************** +* Internal functions +****************************************************************/ +MEM_STATIC unsigned BIT_highbit32 (register U32 val) +{ +# if defined(_MSC_VER) /* Visual */ + unsigned long r=0; + _BitScanReverse ( &r, val ); + return (unsigned) r; +# elif defined(__GNUC__) && (__GNUC__ >= 3) /* Use GCC Intrinsic */ + return 31 - __builtin_clz (val); +# else /* Software version */ + static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 }; + U32 v = val; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + return DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27]; +# endif +} + +/*===== Local Constants =====*/ +static const unsigned BIT_mask[] = { 0, 1, 3, 7, 0xF, 0x1F, 0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF, 0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0x1FFFF, 0x3FFFF, 0x7FFFF, 0xFFFFF, 0x1FFFFF, 0x3FFFFF, 0x7FFFFF, 0xFFFFFF, 0x1FFFFFF, 0x3FFFFFF }; /* up to 26 bits */ + + +/*-************************************************************** +* bitStream encoding +****************************************************************/ +/*! BIT_initCStream() : + * `dstCapacity` must be > sizeof(void*) + * @return : 0 if success, + otherwise an error code (can be tested using ERR_isError() ) */ +MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* startPtr, size_t dstCapacity) +{ + bitC->bitContainer = 0; + bitC->bitPos = 0; + bitC->startPtr = (char*)startPtr; + bitC->ptr = bitC->startPtr; + bitC->endPtr = bitC->startPtr + dstCapacity - sizeof(bitC->ptr); + if (dstCapacity <= sizeof(bitC->ptr)) return ERROR(dstSize_tooSmall); + return 0; +} + +/*! BIT_addBits() : + can add up to 26 bits into `bitC`. + Does not check for register overflow ! */ +MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits) +{ + bitC->bitContainer |= (value & BIT_mask[nbBits]) << bitC->bitPos; + bitC->bitPos += nbBits; +} + +/*! BIT_addBitsFast() : + * works only if `value` is _clean_, meaning all high bits above nbBits are 0 */ +MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC, size_t value, unsigned nbBits) +{ + bitC->bitContainer |= value << bitC->bitPos; + bitC->bitPos += nbBits; +} + +/*! BIT_flushBitsFast() : + * unsafe version; does not check buffer overflow */ +MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC) +{ + size_t const nbBytes = bitC->bitPos >> 3; + MEM_writeLEST(bitC->ptr, bitC->bitContainer); + bitC->ptr += nbBytes; + bitC->bitPos &= 7; + bitC->bitContainer >>= nbBytes*8; /* if bitPos >= sizeof(bitContainer)*8 --> undefined behavior */ +} + +/*! BIT_flushBits() : + * safe version; check for buffer overflow, and prevents it. + * note : does not signal buffer overflow. This will be revealed later on using BIT_closeCStream() */ +MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC) +{ + size_t const nbBytes = bitC->bitPos >> 3; + MEM_writeLEST(bitC->ptr, bitC->bitContainer); + bitC->ptr += nbBytes; + if (bitC->ptr > bitC->endPtr) bitC->ptr = bitC->endPtr; + bitC->bitPos &= 7; + bitC->bitContainer >>= nbBytes*8; /* if bitPos >= sizeof(bitContainer)*8 --> undefined behavior */ +} + +/*! BIT_closeCStream() : + * @return : size of CStream, in bytes, + or 0 if it could not fit into dstBuffer */ +MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC) +{ + BIT_addBitsFast(bitC, 1, 1); /* endMark */ + BIT_flushBits(bitC); + + if (bitC->ptr >= bitC->endPtr) return 0; /* doesn't fit within authorized budget : cancel */ + + return (bitC->ptr - bitC->startPtr) + (bitC->bitPos > 0); +} + + +/*-******************************************************** +* bitStream decoding +**********************************************************/ +/*! BIT_initDStream() : +* Initialize a BIT_DStream_t. +* `bitD` : a pointer to an already allocated BIT_DStream_t structure. +* `srcSize` must be the *exact* size of the bitStream, in bytes. +* @return : size of stream (== srcSize) or an errorCode if a problem is detected +*/ +MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize) +{ + if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); } + + if (srcSize >= sizeof(bitD->bitContainer)) { /* normal case */ + bitD->start = (const char*)srcBuffer; + bitD->ptr = (const char*)srcBuffer + srcSize - sizeof(bitD->bitContainer); + bitD->bitContainer = MEM_readLEST(bitD->ptr); + { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1]; + bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; /* ensures bitsConsumed is always set */ + if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ } + } else { + bitD->start = (const char*)srcBuffer; + bitD->ptr = bitD->start; + bitD->bitContainer = *(const BYTE*)(bitD->start); + switch(srcSize) + { + case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16); // fallthrough + case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24); // fallthrough + case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32); // fallthrough + case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24; // fallthrough + case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16; // fallthrough + case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) << 8; // fallthrough + default:; + } + { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1]; + bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; + if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ } + bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8; + } + + return srcSize; +} + +MEM_STATIC size_t BIT_getUpperBits(size_t bitContainer, U32 const start) +{ + return bitContainer >> start; +} + +MEM_STATIC size_t BIT_getMiddleBits(size_t bitContainer, U32 const start, U32 const nbBits) +{ +#if defined(__BMI__) && defined(__GNUC__) && __GNUC__*1000+__GNUC_MINOR__ >= 4008 /* experimental */ +# if defined(__x86_64__) + if (sizeof(bitContainer)==8) + return _bextr_u64(bitContainer, start, nbBits); + else +# endif + return _bextr_u32(bitContainer, start, nbBits); +#else + return (bitContainer >> start) & BIT_mask[nbBits]; +#endif +} + +MEM_STATIC size_t BIT_getLowerBits(size_t bitContainer, U32 const nbBits) +{ + return bitContainer & BIT_mask[nbBits]; +} + +/*! BIT_lookBits() : + * Provides next n bits from local register. + * local register is not modified. + * On 32-bits, maxNbBits==24. + * On 64-bits, maxNbBits==56. + * @return : value extracted + */ + MEM_STATIC size_t BIT_lookBits(const BIT_DStream_t* bitD, U32 nbBits) +{ +#if defined(__BMI__) && defined(__GNUC__) /* experimental; fails if bitD->bitsConsumed + nbBits > sizeof(bitD->bitContainer)*8 */ + return BIT_getMiddleBits(bitD->bitContainer, (sizeof(bitD->bitContainer)*8) - bitD->bitsConsumed - nbBits, nbBits); +#else + U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1; + return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask); +#endif +} + +/*! BIT_lookBitsFast() : +* unsafe version; only works only if nbBits >= 1 */ +MEM_STATIC size_t BIT_lookBitsFast(const BIT_DStream_t* bitD, U32 nbBits) +{ + U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1; + return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask); +} + +MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits) +{ + bitD->bitsConsumed += nbBits; +} + +/*! BIT_readBits() : + * Read (consume) next n bits from local register and update. + * Pay attention to not read more than nbBits contained into local register. + * @return : extracted value. + */ +MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits) +{ + size_t const value = BIT_lookBits(bitD, nbBits); + BIT_skipBits(bitD, nbBits); + return value; +} + +/*! BIT_readBitsFast() : +* unsafe version; only works only if nbBits >= 1 */ +MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits) +{ + size_t const value = BIT_lookBitsFast(bitD, nbBits); + BIT_skipBits(bitD, nbBits); + return value; +} + +/*! BIT_reloadDStream() : +* Refill `bitD` from buffer previously set in BIT_initDStream() . +* This function is safe, it guarantees it will not read beyond src buffer. +* @return : status of `BIT_DStream_t` internal register. + if status == BIT_DStream_unfinished, internal register is filled with >= (sizeof(bitD->bitContainer)*8 - 7) bits */ +MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD) +{ + if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8)) /* should not happen => corruption detected */ + return BIT_DStream_overflow; + + if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer)) { + bitD->ptr -= bitD->bitsConsumed >> 3; + bitD->bitsConsumed &= 7; + bitD->bitContainer = MEM_readLEST(bitD->ptr); + return BIT_DStream_unfinished; + } + if (bitD->ptr == bitD->start) { + if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer; + return BIT_DStream_completed; + } + { U32 nbBytes = bitD->bitsConsumed >> 3; + BIT_DStream_status result = BIT_DStream_unfinished; + if (bitD->ptr - nbBytes < bitD->start) { + nbBytes = (U32)(bitD->ptr - bitD->start); /* ptr > start */ + result = BIT_DStream_endOfBuffer; + } + bitD->ptr -= nbBytes; + bitD->bitsConsumed -= nbBytes*8; + bitD->bitContainer = MEM_readLEST(bitD->ptr); /* reminder : srcSize > sizeof(bitD) */ + return result; + } +} + +/*! BIT_endOfDStream() : +* @return Tells if DStream has exactly reached its end (all bits consumed). +*/ +MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream) +{ + return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8)); +} + +#if defined (__cplusplus) +} +#endif + +#endif /* BITSTREAM_H_MODULE */ diff --git a/contrib/lizard/lib/entropy/entropy_common.c b/contrib/lizard/lib/entropy/entropy_common.c new file mode 100644 index 00000000000..72bc398da38 --- /dev/null +++ b/contrib/lizard/lib/entropy/entropy_common.c @@ -0,0 +1,231 @@ +/* + Common functions of New Generation Entropy library + Copyright (C) 2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +*************************************************************************** */ + +/* ************************************* +* Dependencies +***************************************/ +#include "mem.h" +#include "error_private.h" /* ERR_*, ERROR */ +#define FSE_STATIC_LINKING_ONLY /* FSE_MIN_TABLELOG */ +#include "fse.h" +#define HUF_STATIC_LINKING_ONLY /* HUF_TABLELOG_ABSOLUTEMAX */ +#include "huf.h" + + +/*-**************************************** +* Version +******************************************/ +unsigned FSE_versionNumber(void) { return FSE_VERSION_NUMBER; } + + +/*-**************************************** +* FSE Error Management +******************************************/ +unsigned FSE_isError(size_t code) { return ERR_isError(code); } + +const char* FSE_getErrorName(size_t code) { return ERR_getErrorName(code); } + + +/* ************************************************************** +* HUF Error Management +****************************************************************/ +unsigned HUF_isError(size_t code) { return ERR_isError(code); } + +const char* HUF_getErrorName(size_t code) { return ERR_getErrorName(code); } + + +/*-************************************************************** +* FSE NCount encoding-decoding +****************************************************************/ +size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr, + const void* headerBuffer, size_t hbSize) +{ + const BYTE* const istart = (const BYTE*) headerBuffer; + const BYTE* const iend = istart + hbSize; + const BYTE* ip = istart; + int nbBits; + int remaining; + int threshold; + U32 bitStream; + int bitCount; + unsigned charnum = 0; + int previous0 = 0; + + if (hbSize < 4) return ERROR(srcSize_wrong); + bitStream = MEM_readLE32(ip); + nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */ + if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge); + bitStream >>= 4; + bitCount = 4; + *tableLogPtr = nbBits; + remaining = (1<1) & (charnum<=*maxSVPtr)) { + if (previous0) { + unsigned n0 = charnum; + while ((bitStream & 0xFFFF) == 0xFFFF) { + n0 += 24; + if (ip < iend-5) { + ip += 2; + bitStream = MEM_readLE32(ip) >> bitCount; + } else { + bitStream >>= 16; + bitCount += 16; + } } + while ((bitStream & 3) == 3) { + n0 += 3; + bitStream >>= 2; + bitCount += 2; + } + n0 += bitStream & 3; + bitCount += 2; + if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall); + while (charnum < n0) normalizedCounter[charnum++] = 0; + if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) { + ip += bitCount>>3; + bitCount &= 7; + bitStream = MEM_readLE32(ip) >> bitCount; + } else { + bitStream >>= 2; + } } + { int const max = (2*threshold-1) - remaining; + int count; + + if ((bitStream & (threshold-1)) < (U32)max) { + count = bitStream & (threshold-1); + bitCount += nbBits-1; + } else { + count = bitStream & (2*threshold-1); + if (count >= threshold) count -= max; + bitCount += nbBits; + } + + count--; /* extra accuracy */ + remaining -= count < 0 ? -count : count; /* -1 means +1 */ + normalizedCounter[charnum++] = (short)count; + previous0 = !count; + while (remaining < threshold) { + nbBits--; + threshold >>= 1; + } + + if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) { + ip += bitCount>>3; + bitCount &= 7; + } else { + bitCount -= (int)(8 * (iend - 4 - ip)); + ip = iend - 4; + } + bitStream = MEM_readLE32(ip) >> (bitCount & 31); + } } /* while ((remaining>1) & (charnum<=*maxSVPtr)) */ + if (remaining != 1) return ERROR(corruption_detected); + if (bitCount > 32) return ERROR(corruption_detected); + *maxSVPtr = charnum-1; + + ip += (bitCount+7)>>3; + return ip-istart; +} + + +/*! HUF_readStats() : + Read compact Huffman tree, saved by HUF_writeCTable(). + `huffWeight` is destination buffer. + `rankStats` is assumed to be a table of at least HUF_TABLELOG_MAX U32. + @return : size read from `src` , or an error Code . + Note : Needed by HUF_readCTable() and HUF_readDTableX?() . +*/ +size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats, + U32* nbSymbolsPtr, U32* tableLogPtr, + const void* src, size_t srcSize) +{ + U32 weightTotal; + const BYTE* ip = (const BYTE*) src; + size_t iSize; + size_t oSize; + + if (!srcSize) return ERROR(srcSize_wrong); + iSize = ip[0]; + /* memset(huffWeight, 0, hwSize); *//* is not necessary, even though some analyzer complain ... */ + + if (iSize >= 128) { /* special header */ + oSize = iSize - 127; + iSize = ((oSize+1)/2); + if (iSize+1 > srcSize) return ERROR(srcSize_wrong); + if (oSize >= hwSize) return ERROR(corruption_detected); + ip += 1; + { U32 n; + for (n=0; n> 4; + huffWeight[n+1] = ip[n/2] & 15; + } } } + else { /* header compressed with FSE (normal case) */ + FSE_DTable fseWorkspace[FSE_DTABLE_SIZE_U32(6)]; /* 6 is max possible tableLog for HUF header (maybe even 5, to be tested) */ + if (iSize+1 > srcSize) return ERROR(srcSize_wrong); + oSize = FSE_decompress_wksp(huffWeight, hwSize-1, ip+1, iSize, fseWorkspace, 6); /* max (hwSize-1) values decoded, as last one is implied */ + if (FSE_isError(oSize)) return oSize; + } + + /* collect weight stats */ + memset(rankStats, 0, (HUF_TABLELOG_MAX + 1) * sizeof(U32)); + weightTotal = 0; + { U32 n; for (n=0; n= HUF_TABLELOG_MAX) return ERROR(corruption_detected); + rankStats[huffWeight[n]]++; + weightTotal += (1 << huffWeight[n]) >> 1; + } } + if (weightTotal == 0) return ERROR(corruption_detected); + + /* get last non-null symbol weight (implied, total must be 2^n) */ + { U32 const tableLog = BIT_highbit32(weightTotal) + 1; + if (tableLog > HUF_TABLELOG_MAX) return ERROR(corruption_detected); + *tableLogPtr = tableLog; + /* determine last weight */ + { U32 const total = 1 << tableLog; + U32 const rest = total - weightTotal; + U32 const verif = 1 << BIT_highbit32(rest); + U32 const lastWeight = BIT_highbit32(rest) + 1; + if (verif != rest) return ERROR(corruption_detected); /* last value must be a clean power of 2 */ + huffWeight[oSize] = (BYTE)lastWeight; + rankStats[lastWeight]++; + } } + + /* check tree construction validity */ + if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected); /* by construction : at least 2 elts of rank 1, must be even */ + + /* results */ + *nbSymbolsPtr = (U32)(oSize+1); + return iSize+1; +} diff --git a/contrib/lizard/lib/entropy/error_private.h b/contrib/lizard/lib/entropy/error_private.h new file mode 100644 index 00000000000..1340c16bf25 --- /dev/null +++ b/contrib/lizard/lib/entropy/error_private.h @@ -0,0 +1,115 @@ +/* ****************************************************************** + Error codes and messages + Copyright (C) 2013-2016, Yann Collet + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Homepage : http://www.zstd.net +****************************************************************** */ +/* Note : this module is expected to remain private, do not expose it */ + +#ifndef ERROR_H_MODULE +#define ERROR_H_MODULE + +#if defined (__cplusplus) +extern "C" { +#endif + + +/* **************************************** +* Dependencies +******************************************/ +#include /* size_t */ +#include "error_public.h" /* enum list */ + + +/* **************************************** +* Compiler-specific +******************************************/ +#if defined(__GNUC__) +# define ERR_STATIC static __attribute__((unused)) +#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) +# define ERR_STATIC static inline +#elif defined(_MSC_VER) +# define ERR_STATIC static __inline +#else +# define ERR_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */ +#endif + + +/*-**************************************** +* Customization (error_public.h) +******************************************/ +typedef FSE_ErrorCode ERR_enum; +#define PREFIX(name) FSE_error_##name + + +/*-**************************************** +* Error codes handling +******************************************/ +#ifdef ERROR +# undef ERROR /* reported already defined on VS 2015 (Rich Geldreich) */ +#endif +#define ERROR(name) ((size_t)-PREFIX(name)) + +ERR_STATIC unsigned ERR_isError(size_t code) { return (code > ERROR(maxCode)); } + +ERR_STATIC ERR_enum ERR_getErrorCode(size_t code) { if (!ERR_isError(code)) return (ERR_enum)0; return (ERR_enum) (0-code); } + + +/*-**************************************** +* Error Strings +******************************************/ + +ERR_STATIC const char* ERR_getErrorString(ERR_enum code) +{ + static const char* notErrorCode = "Unspecified error code"; + switch( code ) + { + case PREFIX(no_error): return "No error detected"; + case PREFIX(GENERIC): return "Error (generic)"; + case PREFIX(dstSize_tooSmall): return "Destination buffer is too small"; + case PREFIX(srcSize_wrong): return "Src size incorrect"; + case PREFIX(corruption_detected): return "Corrupted block detected"; + case PREFIX(tableLog_tooLarge): return "tableLog requires too much memory : unsupported"; + case PREFIX(maxSymbolValue_tooLarge): return "Unsupported max Symbol Value : too large"; + case PREFIX(maxSymbolValue_tooSmall): return "Specified maxSymbolValue is too small"; + case PREFIX(maxCode): + default: return notErrorCode; + } +} + +ERR_STATIC const char* ERR_getErrorName(size_t code) +{ + return ERR_getErrorString(ERR_getErrorCode(code)); +} + +#if defined (__cplusplus) +} +#endif + +#endif /* ERROR_H_MODULE */ diff --git a/contrib/lizard/lib/entropy/error_public.h b/contrib/lizard/lib/entropy/error_public.h new file mode 100644 index 00000000000..a8524719889 --- /dev/null +++ b/contrib/lizard/lib/entropy/error_public.h @@ -0,0 +1,64 @@ +/* ****************************************************************** + Error codes list + Copyright (C) 2016, Yann Collet + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ +#ifndef ERROR_PUBLIC_H_MODULE +#define ERROR_PUBLIC_H_MODULE + +#if defined (__cplusplus) +extern "C" { +#endif + + +/* **************************************** +* error codes list +******************************************/ +typedef enum { + FSE_error_no_error, + FSE_error_GENERIC, + FSE_error_dstSize_tooSmall, + FSE_error_srcSize_wrong, + FSE_error_corruption_detected, + FSE_error_tableLog_tooLarge, + FSE_error_maxSymbolValue_tooLarge, + FSE_error_maxSymbolValue_tooSmall, + FSE_error_maxCode +} FSE_ErrorCode; + +/* note : compare with size_t function results using FSE_getError() */ + + +#if defined (__cplusplus) +} +#endif + +#endif /* ERROR_PUBLIC_H_MODULE */ diff --git a/contrib/lizard/lib/entropy/fse.h b/contrib/lizard/lib/entropy/fse.h new file mode 100644 index 00000000000..baac3903267 --- /dev/null +++ b/contrib/lizard/lib/entropy/fse.h @@ -0,0 +1,694 @@ +/* ****************************************************************** + FSE : Finite State Entropy codec + Public Prototypes declaration + Copyright (C) 2013-2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Source repository : https://github.com/Cyan4973/FiniteStateEntropy +****************************************************************** */ +#ifndef FSE_H +#define FSE_H + +#if defined (__cplusplus) +extern "C" { +#endif + + +/*-***************************************** +* Dependencies +******************************************/ +#include /* size_t, ptrdiff_t */ + + +/*-***************************************** +* FSE_PUBLIC_API : control library symbols visibility +******************************************/ +#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4) +# define FSE_PUBLIC_API __attribute__ ((visibility ("default"))) +#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) /* Visual expected */ +# define FSE_PUBLIC_API __declspec(dllexport) +#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1) +# define FSE_PUBLIC_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ +#else +# define FSE_PUBLIC_API +#endif + +/*------ Version ------*/ +#define FSE_VERSION_MAJOR 0 +#define FSE_VERSION_MINOR 9 +#define FSE_VERSION_RELEASE 0 + +#define FSE_LIB_VERSION FSE_VERSION_MAJOR.FSE_VERSION_MINOR.FSE_VERSION_RELEASE +#define FSE_QUOTE(str) #str +#define FSE_EXPAND_AND_QUOTE(str) FSE_QUOTE(str) +#define FSE_VERSION_STRING FSE_EXPAND_AND_QUOTE(FSE_LIB_VERSION) + +#define FSE_VERSION_NUMBER (FSE_VERSION_MAJOR *100*100 + FSE_VERSION_MINOR *100 + FSE_VERSION_RELEASE) +FSE_PUBLIC_API unsigned FSE_versionNumber(void); /**< library version number; to be used when checking dll version */ + +/*-**************************************** +* FSE simple functions +******************************************/ +/*! FSE_compress() : + Compress content of buffer 'src', of size 'srcSize', into destination buffer 'dst'. + 'dst' buffer must be already allocated. Compression runs faster is dstCapacity >= FSE_compressBound(srcSize). + @return : size of compressed data (<= dstCapacity). + Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!! + if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression instead. + if FSE_isError(return), compression failed (more details using FSE_getErrorName()) +*/ +FSE_PUBLIC_API size_t FSE_compress(void* dst, size_t dstCapacity, + const void* src, size_t srcSize); + +/*! FSE_decompress(): + Decompress FSE data from buffer 'cSrc', of size 'cSrcSize', + into already allocated destination buffer 'dst', of size 'dstCapacity'. + @return : size of regenerated data (<= maxDstSize), + or an error code, which can be tested using FSE_isError() . + + ** Important ** : FSE_decompress() does not decompress non-compressible nor RLE data !!! + Why ? : making this distinction requires a header. + Header management is intentionally delegated to the user layer, which can better manage special cases. +*/ +FSE_PUBLIC_API size_t FSE_decompress(void* dst, size_t dstCapacity, + const void* cSrc, size_t cSrcSize); + + +/*-***************************************** +* Tool functions +******************************************/ +FSE_PUBLIC_API size_t FSE_compressBound(size_t size); /* maximum compressed size */ + +/* Error Management */ +FSE_PUBLIC_API unsigned FSE_isError(size_t code); /* tells if a return value is an error code */ +FSE_PUBLIC_API const char* FSE_getErrorName(size_t code); /* provides error code string (useful for debugging) */ + + +/*-***************************************** +* FSE advanced functions +******************************************/ +/*! FSE_compress2() : + Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog' + Both parameters can be defined as '0' to mean : use default value + @return : size of compressed data + Special values : if return == 0, srcData is not compressible => Nothing is stored within cSrc !!! + if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression. + if FSE_isError(return), it's an error code. +*/ +FSE_PUBLIC_API size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); + + +/*-***************************************** +* FSE detailed API +******************************************/ +/*! +FSE_compress() does the following: +1. count symbol occurrence from source[] into table count[] +2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog) +3. save normalized counters to memory buffer using writeNCount() +4. build encoding table 'CTable' from normalized counters +5. encode the data stream using encoding table 'CTable' + +FSE_decompress() does the following: +1. read normalized counters with readNCount() +2. build decoding table 'DTable' from normalized counters +3. decode the data stream using decoding table 'DTable' + +The following API allows targeting specific sub-functions for advanced tasks. +For example, it's possible to compress several blocks using the same 'CTable', +or to save and provide normalized distribution using external method. +*/ + +/* *** COMPRESSION *** */ + +/*! FSE_count(): + Provides the precise count of each byte within a table 'count'. + 'count' is a table of unsigned int, of minimum size (*maxSymbolValuePtr+1). + *maxSymbolValuePtr will be updated if detected smaller than initial value. + @return : the count of the most frequent symbol (which is not identified). + if return == srcSize, there is only one symbol. + Can also return an error code, which can be tested with FSE_isError(). */ +FSE_PUBLIC_API size_t FSE_count(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize); + +/*! FSE_optimalTableLog(): + dynamically downsize 'tableLog' when conditions are met. + It saves CPU time, by using smaller tables, while preserving or even improving compression ratio. + @return : recommended tableLog (necessarily <= 'maxTableLog') */ +FSE_PUBLIC_API unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue); + +/*! FSE_normalizeCount(): + normalize counts so that sum(count[]) == Power_of_2 (2^tableLog) + 'normalizedCounter' is a table of short, of minimum size (maxSymbolValue+1). + @return : tableLog, + or an errorCode, which can be tested using FSE_isError() */ +FSE_PUBLIC_API size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t srcSize, unsigned maxSymbolValue); + +/*! FSE_NCountWriteBound(): + Provides the maximum possible size of an FSE normalized table, given 'maxSymbolValue' and 'tableLog'. + Typically useful for allocation purpose. */ +FSE_PUBLIC_API size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog); + +/*! FSE_writeNCount(): + Compactly save 'normalizedCounter' into 'buffer'. + @return : size of the compressed table, + or an errorCode, which can be tested using FSE_isError(). */ +FSE_PUBLIC_API size_t FSE_writeNCount (void* buffer, size_t bufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); + + +/*! Constructor and Destructor of FSE_CTable. + Note that FSE_CTable size depends on 'tableLog' and 'maxSymbolValue' */ +typedef unsigned FSE_CTable; /* don't allocate that. It's only meant to be more restrictive than void* */ +FSE_PUBLIC_API FSE_CTable* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue); +FSE_PUBLIC_API void FSE_freeCTable (FSE_CTable* ct); + +/*! FSE_buildCTable(): + Builds `ct`, which must be already allocated, using FSE_createCTable(). + @return : 0, or an errorCode, which can be tested using FSE_isError() */ +FSE_PUBLIC_API size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); + +/*! FSE_compress_usingCTable(): + Compress `src` using `ct` into `dst` which must be already allocated. + @return : size of compressed data (<= `dstCapacity`), + or 0 if compressed data could not fit into `dst`, + or an errorCode, which can be tested using FSE_isError() */ +FSE_PUBLIC_API size_t FSE_compress_usingCTable (void* dst, size_t dstCapacity, const void* src, size_t srcSize, const FSE_CTable* ct); + +/*! +Tutorial : +---------- +The first step is to count all symbols. FSE_count() does this job very fast. +Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have 'maxSymbolValuePtr[0]+1' cells. +'src' is a table of bytes of size 'srcSize'. All values within 'src' MUST be <= maxSymbolValuePtr[0] +maxSymbolValuePtr[0] will be updated, with its real value (necessarily <= original value) +FSE_count() will return the number of occurrence of the most frequent symbol. +This can be used to know if there is a single symbol within 'src', and to quickly evaluate its compressibility. +If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()). + +The next step is to normalize the frequencies. +FSE_normalizeCount() will ensure that sum of frequencies is == 2 ^'tableLog'. +It also guarantees a minimum of 1 to any Symbol with frequency >= 1. +You can use 'tableLog'==0 to mean "use default tableLog value". +If you are unsure of which tableLog value to use, you can ask FSE_optimalTableLog(), +which will provide the optimal valid tableLog given sourceSize, maxSymbolValue, and a user-defined maximum (0 means "default"). + +The result of FSE_normalizeCount() will be saved into a table, +called 'normalizedCounter', which is a table of signed short. +'normalizedCounter' must be already allocated, and have at least 'maxSymbolValue+1' cells. +The return value is tableLog if everything proceeded as expected. +It is 0 if there is a single symbol within distribution. +If there is an error (ex: invalid tableLog value), the function will return an ErrorCode (which can be tested using FSE_isError()). + +'normalizedCounter' can be saved in a compact manner to a memory area using FSE_writeNCount(). +'buffer' must be already allocated. +For guaranteed success, buffer size must be at least FSE_headerBound(). +The result of the function is the number of bytes written into 'buffer'. +If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError(); ex : buffer size too small). + +'normalizedCounter' can then be used to create the compression table 'CTable'. +The space required by 'CTable' must be already allocated, using FSE_createCTable(). +You can then use FSE_buildCTable() to fill 'CTable'. +If there is an error, both functions will return an ErrorCode (which can be tested using FSE_isError()). + +'CTable' can then be used to compress 'src', with FSE_compress_usingCTable(). +Similar to FSE_count(), the convention is that 'src' is assumed to be a table of char of size 'srcSize' +The function returns the size of compressed data (without header), necessarily <= `dstCapacity`. +If it returns '0', compressed data could not fit into 'dst'. +If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()). +*/ + + +/* *** DECOMPRESSION *** */ + +/*! FSE_readNCount(): + Read compactly saved 'normalizedCounter' from 'rBuffer'. + @return : size read from 'rBuffer', + or an errorCode, which can be tested using FSE_isError(). + maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */ +FSE_PUBLIC_API size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize); + +/*! Constructor and Destructor of FSE_DTable. + Note that its size depends on 'tableLog' */ +typedef unsigned FSE_DTable; /* don't allocate that. It's just a way to be more restrictive than void* */ +FSE_PUBLIC_API FSE_DTable* FSE_createDTable(unsigned tableLog); +FSE_PUBLIC_API void FSE_freeDTable(FSE_DTable* dt); + +/*! FSE_buildDTable(): + Builds 'dt', which must be already allocated, using FSE_createDTable(). + return : 0, or an errorCode, which can be tested using FSE_isError() */ +FSE_PUBLIC_API size_t FSE_buildDTable (FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); + +/*! FSE_decompress_usingDTable(): + Decompress compressed source `cSrc` of size `cSrcSize` using `dt` + into `dst` which must be already allocated. + @return : size of regenerated data (necessarily <= `dstCapacity`), + or an errorCode, which can be tested using FSE_isError() */ +FSE_PUBLIC_API size_t FSE_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt); + +/*! +Tutorial : +---------- +(Note : these functions only decompress FSE-compressed blocks. + If block is uncompressed, use memcpy() instead + If block is a single repeated byte, use memset() instead ) + +The first step is to obtain the normalized frequencies of symbols. +This can be performed by FSE_readNCount() if it was saved using FSE_writeNCount(). +'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short. +In practice, that means it's necessary to know 'maxSymbolValue' beforehand, +or size the table to handle worst case situations (typically 256). +FSE_readNCount() will provide 'tableLog' and 'maxSymbolValue'. +The result of FSE_readNCount() is the number of bytes read from 'rBuffer'. +Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that. +If there is an error, the function will return an error code, which can be tested using FSE_isError(). + +The next step is to build the decompression tables 'FSE_DTable' from 'normalizedCounter'. +This is performed by the function FSE_buildDTable(). +The space required by 'FSE_DTable' must be already allocated using FSE_createDTable(). +If there is an error, the function will return an error code, which can be tested using FSE_isError(). + +`FSE_DTable` can then be used to decompress `cSrc`, with FSE_decompress_usingDTable(). +`cSrcSize` must be strictly correct, otherwise decompression will fail. +FSE_decompress_usingDTable() result will tell how many bytes were regenerated (<=`dstCapacity`). +If there is an error, the function will return an error code, which can be tested using FSE_isError(). (ex: dst buffer too small) +*/ + + +#ifdef FSE_STATIC_LINKING_ONLY + +/* *** Dependency *** */ +#include "bitstream.h" + + +/* ***************************************** +* Static allocation +*******************************************/ +/* FSE buffer bounds */ +#define FSE_NCOUNTBOUND 512 +#define FSE_BLOCKBOUND(size) (size + (size>>7)) +#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size)) /* Macro version, useful for static allocation */ + +/* It is possible to statically allocate FSE CTable/DTable as a table of FSE_CTable/FSE_DTable using below macros */ +#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2)) +#define FSE_DTABLE_SIZE_U32(maxTableLog) (1 + (1<= `1024` unsigned + */ +size_t FSE_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr, + const void* source, size_t sourceSize, unsigned* workSpace); + +/** FSE_countFast() : + * same as FSE_count(), but blindly trusts that all byte values within src are <= *maxSymbolValuePtr + */ +size_t FSE_countFast(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize); + +/* FSE_countFast_wksp() : + * Same as FSE_countFast(), but using an externally provided scratch buffer. + * `workSpace` must be a table of minimum `1024` unsigned + */ +size_t FSE_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize, unsigned* workSpace); + +/*! FSE_count_simple + * Same as FSE_countFast(), but does not use any additional memory (not even on stack). + * This function is unsafe, and will segfault if any value within `src` is `> *maxSymbolValuePtr` (presuming it's also the size of `count`). +*/ +size_t FSE_count_simple(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize); + + + +unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus); +/**< same as FSE_optimalTableLog(), which used `minus==2` */ + +/* FSE_compress_wksp() : + * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`). + * FSE_WKSP_SIZE_U32() provides the minimum size required for `workSpace` as a table of FSE_CTable. + */ +#define FSE_WKSP_SIZE_U32(maxTableLog, maxSymbolValue) ( FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) + (1<<((maxTableLog>2)?(maxTableLog-2):0)) ) +size_t FSE_compress_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); + +size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits); +/**< build a fake FSE_CTable, designed for a flat distribution, where each symbol uses nbBits */ + +size_t FSE_buildCTable_rle (FSE_CTable* ct, unsigned char symbolValue); +/**< build a fake FSE_CTable, designed to compress always the same symbolValue */ + +/* FSE_buildCTable_wksp() : + * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`). + * `wkspSize` must be >= `(1<= BIT_DStream_completed + +When it's done, verify decompression is fully completed, by checking both DStream and the relevant states. +Checking if DStream has reached its end is performed by : + BIT_endOfDStream(&DStream); +Check also the states. There might be some symbols left there, if some high probability ones (>50%) are possible. + FSE_endOfDState(&DState); +*/ + + +/* ***************************************** +* FSE unsafe API +*******************************************/ +static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD); +/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */ + + +/* ***************************************** +* Implementation of inlined functions +*******************************************/ +typedef struct { + int deltaFindState; + U32 deltaNbBits; +} FSE_symbolCompressionTransform; /* total 8 bytes */ + +MEM_STATIC void FSE_initCState(FSE_CState_t* statePtr, const FSE_CTable* ct) +{ + const void* ptr = ct; + const U16* u16ptr = (const U16*) ptr; + const U32 tableLog = MEM_read16(ptr); + statePtr->value = (ptrdiff_t)1<stateTable = u16ptr+2; + statePtr->symbolTT = ((const U32*)ct + 1 + (tableLog ? (1<<(tableLog-1)) : 1)); + statePtr->stateLog = tableLog; +} + + +/*! FSE_initCState2() : +* Same as FSE_initCState(), but the first symbol to include (which will be the last to be read) +* uses the smallest state value possible, saving the cost of this symbol */ +MEM_STATIC void FSE_initCState2(FSE_CState_t* statePtr, const FSE_CTable* ct, U32 symbol) +{ + FSE_initCState(statePtr, ct); + { const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol]; + const U16* stateTable = (const U16*)(statePtr->stateTable); + U32 nbBitsOut = (U32)((symbolTT.deltaNbBits + (1<<15)) >> 16); + statePtr->value = (nbBitsOut << 16) - symbolTT.deltaNbBits; + statePtr->value = stateTable[(statePtr->value >> nbBitsOut) + symbolTT.deltaFindState]; + } +} + +MEM_STATIC void FSE_encodeSymbol(BIT_CStream_t* bitC, FSE_CState_t* statePtr, U32 symbol) +{ + const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol]; + const U16* const stateTable = (const U16*)(statePtr->stateTable); + U32 nbBitsOut = (U32)((statePtr->value + symbolTT.deltaNbBits) >> 16); + BIT_addBits(bitC, statePtr->value, nbBitsOut); + statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT.deltaFindState]; +} + +MEM_STATIC void FSE_flushCState(BIT_CStream_t* bitC, const FSE_CState_t* statePtr) +{ + BIT_addBits(bitC, statePtr->value, statePtr->stateLog); + BIT_flushBits(bitC); +} + + +/* ====== Decompression ====== */ + +typedef struct { + U16 tableLog; + U16 fastMode; +} FSE_DTableHeader; /* sizeof U32 */ + +typedef struct +{ + unsigned short newState; + unsigned char symbol; + unsigned char nbBits; +} FSE_decode_t; /* size == U32 */ + +MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt) +{ + const void* ptr = dt; + const FSE_DTableHeader* const DTableH = (const FSE_DTableHeader*)ptr; + DStatePtr->state = BIT_readBits(bitD, DTableH->tableLog); + BIT_reloadDStream(bitD); + DStatePtr->table = dt + 1; +} + +MEM_STATIC BYTE FSE_peekSymbol(const FSE_DState_t* DStatePtr) +{ + FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; + return DInfo.symbol; +} + +MEM_STATIC void FSE_updateState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD) +{ + FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; + U32 const nbBits = DInfo.nbBits; + size_t const lowBits = BIT_readBits(bitD, nbBits); + DStatePtr->state = DInfo.newState + lowBits; +} + +MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD) +{ + FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; + U32 const nbBits = DInfo.nbBits; + BYTE const symbol = DInfo.symbol; + size_t const lowBits = BIT_readBits(bitD, nbBits); + + DStatePtr->state = DInfo.newState + lowBits; + return symbol; +} + +/*! FSE_decodeSymbolFast() : + unsafe, only works if no symbol has a probability > 50% */ +MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD) +{ + FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; + U32 const nbBits = DInfo.nbBits; + BYTE const symbol = DInfo.symbol; + size_t const lowBits = BIT_readBitsFast(bitD, nbBits); + + DStatePtr->state = DInfo.newState + lowBits; + return symbol; +} + +MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr) +{ + return DStatePtr->state == 0; +} + + + +#ifndef FSE_COMMONDEFS_ONLY + +/* ************************************************************** +* Tuning parameters +****************************************************************/ +/*!MEMORY_USAGE : +* Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.) +* Increasing memory usage improves compression ratio +* Reduced memory usage can improve speed, due to cache effect +* Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */ +#ifndef FSE_MAX_MEMORY_USAGE +# define FSE_MAX_MEMORY_USAGE 14 +#endif +#ifndef FSE_DEFAULT_MEMORY_USAGE +# define FSE_DEFAULT_MEMORY_USAGE 13 +#endif + +/*!FSE_MAX_SYMBOL_VALUE : +* Maximum symbol value authorized. +* Required for proper stack allocation */ +#ifndef FSE_MAX_SYMBOL_VALUE +# define FSE_MAX_SYMBOL_VALUE 255 +#endif + +/* ************************************************************** +* template functions type & suffix +****************************************************************/ +#define FSE_FUNCTION_TYPE BYTE +#define FSE_FUNCTION_EXTENSION +#define FSE_DECODE_TYPE FSE_decode_t + + +#endif /* !FSE_COMMONDEFS_ONLY */ + + +/* *************************************************************** +* Constants +*****************************************************************/ +#define FSE_MAX_TABLELOG (FSE_MAX_MEMORY_USAGE-2) +#define FSE_MAX_TABLESIZE (1U< FSE_TABLELOG_ABSOLUTE_MAX +# error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported" +#endif + +#define FSE_TABLESTEP(tableSize) ((tableSize>>1) + (tableSize>>3) + 3) + + +#endif /* FSE_STATIC_LINKING_ONLY */ + + +#if defined (__cplusplus) +} +#endif + +#endif /* FSE_H */ diff --git a/contrib/lizard/lib/entropy/fse_compress.c b/contrib/lizard/lib/entropy/fse_compress.c new file mode 100644 index 00000000000..337b7a6ffac --- /dev/null +++ b/contrib/lizard/lib/entropy/fse_compress.c @@ -0,0 +1,848 @@ +/* ****************************************************************** + FSE : Finite State Entropy encoder + Copyright (C) 2013-2015, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ + +/* ************************************************************** +* Compiler specifics +****************************************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define FORCE_INLINE static __forceinline +# include /* For Visual 2005 */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ +#else +# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# ifdef __GNUC__ +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +# else +# define FORCE_INLINE static +# endif /* __STDC_VERSION__ */ +#endif + + +/* ************************************************************** +* Includes +****************************************************************/ +#include /* malloc, free, qsort */ +#include /* memcpy, memset */ +#include /* printf (debug) */ +#include "bitstream.h" +#define FSE_STATIC_LINKING_ONLY +#include "fse.h" + + +/* ************************************************************** +* Error Management +****************************************************************/ +#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ + + +/* ************************************************************** +* Templates +****************************************************************/ +/* + designed to be included + for type-specific functions (template emulation in C) + Objective is to write these functions only once, for improved maintenance +*/ + +/* safety checks */ +#ifndef FSE_FUNCTION_EXTENSION +# error "FSE_FUNCTION_EXTENSION must be defined" +#endif +#ifndef FSE_FUNCTION_TYPE +# error "FSE_FUNCTION_TYPE must be defined" +#endif + +/* Function names */ +#define FSE_CAT(X,Y) X##Y +#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y) +#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y) + + +/* Function templates */ + +/* FSE_buildCTable_wksp() : + * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`). + * wkspSize should be sized to handle worst case situation, which is `1<>1 : 1) ; + FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT); + U32 const step = FSE_TABLESTEP(tableSize); + U32 cumul[FSE_MAX_SYMBOL_VALUE+2]; + + FSE_FUNCTION_TYPE* const tableSymbol = (FSE_FUNCTION_TYPE*)workSpace; + U32 highThreshold = tableSize-1; + + /* CTable header */ + if (((size_t)1 << tableLog) * sizeof(FSE_FUNCTION_TYPE) > wkspSize) return ERROR(tableLog_tooLarge); + tableU16[-2] = (U16) tableLog; + tableU16[-1] = (U16) maxSymbolValue; + + /* For explanations on how to distribute symbol values over the table : + * http://fastcompression.blogspot.fr/2014/02/fse-distributing-symbol-values.html */ + + /* symbol start positions */ + { U32 u; + cumul[0] = 0; + for (u=1; u<=maxSymbolValue+1; u++) { + if (normalizedCounter[u-1]==-1) { /* Low proba symbol */ + cumul[u] = cumul[u-1] + 1; + tableSymbol[highThreshold--] = (FSE_FUNCTION_TYPE)(u-1); + } else { + cumul[u] = cumul[u-1] + normalizedCounter[u-1]; + } } + cumul[maxSymbolValue+1] = tableSize+1; + } + + /* Spread symbols */ + { U32 position = 0; + U32 symbol; + for (symbol=0; symbol<=maxSymbolValue; symbol++) { + int nbOccurences; + for (nbOccurences=0; nbOccurences highThreshold) position = (position + step) & tableMask; /* Low proba area */ + } } + + if (position!=0) return ERROR(GENERIC); /* Must have gone through all positions */ + } + + /* Build table */ + { U32 u; for (u=0; u> 3) + 3; + return maxSymbolValue ? maxHeaderSize : FSE_NCOUNTBOUND; /* maxSymbolValue==0 ? use default */ +} + +static size_t FSE_writeNCount_generic (void* header, size_t headerBufferSize, + const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog, + unsigned writeIsSafe) +{ + BYTE* const ostart = (BYTE*) header; + BYTE* out = ostart; + BYTE* const oend = ostart + headerBufferSize; + int nbBits; + const int tableSize = 1 << tableLog; + int remaining; + int threshold; + U32 bitStream; + int bitCount; + unsigned charnum = 0; + int previous0 = 0; + + bitStream = 0; + bitCount = 0; + /* Table Size */ + bitStream += (tableLog-FSE_MIN_TABLELOG) << bitCount; + bitCount += 4; + + /* Init */ + remaining = tableSize+1; /* +1 for extra accuracy */ + threshold = tableSize; + nbBits = tableLog+1; + + while (remaining>1) { /* stops at 1 */ + if (previous0) { + unsigned start = charnum; + while (!normalizedCounter[charnum]) charnum++; + while (charnum >= start+24) { + start+=24; + bitStream += 0xFFFFU << bitCount; + if ((!writeIsSafe) && (out > oend-2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ + out[0] = (BYTE) bitStream; + out[1] = (BYTE)(bitStream>>8); + out+=2; + bitStream>>=16; + } + while (charnum >= start+3) { + start+=3; + bitStream += 3 << bitCount; + bitCount += 2; + } + bitStream += (charnum-start) << bitCount; + bitCount += 2; + if (bitCount>16) { + if ((!writeIsSafe) && (out > oend - 2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out += 2; + bitStream >>= 16; + bitCount -= 16; + } } + { int count = normalizedCounter[charnum++]; + int const max = (2*threshold-1)-remaining; + remaining -= count < 0 ? -count : count; + count++; /* +1 for extra accuracy */ + if (count>=threshold) count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */ + bitStream += count << bitCount; + bitCount += nbBits; + bitCount -= (count>=1; + } + if (bitCount>16) { + if ((!writeIsSafe) && (out > oend - 2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out += 2; + bitStream >>= 16; + bitCount -= 16; + } } + + /* flush remaining bitStream */ + if ((!writeIsSafe) && (out > oend - 2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out+= (bitCount+7) /8; + + if (charnum > maxSymbolValue + 1) return ERROR(GENERIC); + + return (out-ostart); +} + + +size_t FSE_writeNCount (void* buffer, size_t bufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog) +{ + if (tableLog > FSE_MAX_TABLELOG) return ERROR(GENERIC); /* Unsupported */ + if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC); /* Unsupported */ + + if (bufferSize < FSE_NCountWriteBound(maxSymbolValue, tableLog)) + return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 0); + + return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 1); +} + + + +/*-************************************************************** +* Counting histogram +****************************************************************/ +/*! FSE_count_simple + This function counts byte values within `src`, and store the histogram into table `count`. + It doesn't use any additional memory. + But this function is unsafe : it doesn't check that all values within `src` can fit into `count`. + For this reason, prefer using a table `count` with 256 elements. + @return : count of most numerous element +*/ +size_t FSE_count_simple(unsigned* count, unsigned* maxSymbolValuePtr, + const void* src, size_t srcSize) +{ + const BYTE* ip = (const BYTE*)src; + const BYTE* const end = ip + srcSize; + unsigned maxSymbolValue = *maxSymbolValuePtr; + unsigned max=0; + + memset(count, 0, (maxSymbolValue+1)*sizeof(*count)); + if (srcSize==0) { *maxSymbolValuePtr = 0; return 0; } + + while (ip max) max = count[s]; } + + return (size_t)max; +} + + +/* FSE_count_parallel_wksp() : + * Same as FSE_count_parallel(), but using an externally provided scratch buffer. + * `workSpace` size must be a minimum of `1024 * sizeof(unsigned)`` */ +static size_t FSE_count_parallel_wksp( + unsigned* count, unsigned* maxSymbolValuePtr, + const void* source, size_t sourceSize, + unsigned checkMax, unsigned* const workSpace) +{ + const BYTE* ip = (const BYTE*)source; + const BYTE* const iend = ip+sourceSize; + unsigned maxSymbolValue = *maxSymbolValuePtr; + unsigned max=0; + U32* const Counting1 = workSpace; + U32* const Counting2 = Counting1 + 256; + U32* const Counting3 = Counting2 + 256; + U32* const Counting4 = Counting3 + 256; + + memset(Counting1, 0, 4*256*sizeof(unsigned)); + + /* safety checks */ + if (!sourceSize) { + memset(count, 0, maxSymbolValue + 1); + *maxSymbolValuePtr = 0; + return 0; + } + if (!maxSymbolValue) maxSymbolValue = 255; /* 0 == default */ + + /* by stripes of 16 bytes */ + { U32 cached = MEM_read32(ip); ip += 4; + while (ip < iend-15) { + U32 c = cached; cached = MEM_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + c = cached; cached = MEM_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + c = cached; cached = MEM_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + c = cached; cached = MEM_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + } + ip-=4; + } + + /* finish last symbols */ + while (ipmaxSymbolValue; s--) { + Counting1[s] += Counting2[s] + Counting3[s] + Counting4[s]; + if (Counting1[s]) return ERROR(maxSymbolValue_tooSmall); + } } + + { U32 s; for (s=0; s<=maxSymbolValue; s++) { + count[s] = Counting1[s] + Counting2[s] + Counting3[s] + Counting4[s]; + if (count[s] > max) max = count[s]; + } } + + while (!count[maxSymbolValue]) maxSymbolValue--; + *maxSymbolValuePtr = maxSymbolValue; + return (size_t)max; +} + +/* FSE_countFast_wksp() : + * Same as FSE_countFast(), but using an externally provided scratch buffer. + * `workSpace` size must be table of >= `1024` unsigned */ +size_t FSE_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr, + const void* source, size_t sourceSize, unsigned* workSpace) +{ + if (sourceSize < 1500) return FSE_count_simple(count, maxSymbolValuePtr, source, sourceSize); + return FSE_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, 0, workSpace); +} + +/* fast variant (unsafe : won't check if src contains values beyond count[] limit) */ +size_t FSE_countFast(unsigned* count, unsigned* maxSymbolValuePtr, + const void* source, size_t sourceSize) +{ + unsigned tmpCounters[1024]; + return FSE_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, tmpCounters); +} + +/* FSE_count_wksp() : + * Same as FSE_count(), but using an externally provided scratch buffer. + * `workSpace` size must be table of >= `1024` unsigned */ +size_t FSE_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr, + const void* source, size_t sourceSize, unsigned* workSpace) +{ + if (*maxSymbolValuePtr < 255) + return FSE_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, 1, workSpace); + *maxSymbolValuePtr = 255; + return FSE_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, workSpace); +} + +size_t FSE_count(unsigned* count, unsigned* maxSymbolValuePtr, + const void* src, size_t srcSize) +{ + unsigned tmpCounters[1024]; + return FSE_count_wksp(count, maxSymbolValuePtr, src, srcSize, tmpCounters); +} + + + +/*-************************************************************** +* FSE Compression Code +****************************************************************/ +/*! FSE_sizeof_CTable() : + FSE_CTable is a variable size structure which contains : + `U16 tableLog;` + `U16 maxSymbolValue;` + `U16 nextStateNumber[1 << tableLog];` // This size is variable + `FSE_symbolCompressionTransform symbolTT[maxSymbolValue+1];` // This size is variable +Allocation is manual (C standard does not support variable-size structures). +*/ +size_t FSE_sizeof_CTable (unsigned maxSymbolValue, unsigned tableLog) +{ + if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); + return FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32); +} + +FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog) +{ + size_t size; + if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX; + size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32); + return (FSE_CTable*)malloc(size); +} + +void FSE_freeCTable (FSE_CTable* ct) { free(ct); } + +/* provides the minimum logSize to safely represent a distribution */ +static unsigned FSE_minTableLog(size_t srcSize, unsigned maxSymbolValue) +{ + U32 minBitsSrc = BIT_highbit32((U32)(srcSize - 1)) + 1; + U32 minBitsSymbols = BIT_highbit32(maxSymbolValue) + 2; + U32 minBits = minBitsSrc < minBitsSymbols ? minBitsSrc : minBitsSymbols; + return minBits; +} + +unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus) +{ + U32 maxBitsSrc = BIT_highbit32((U32)(srcSize - 1)) - minus; + U32 tableLog = maxTableLog; + U32 minBits = FSE_minTableLog(srcSize, maxSymbolValue); + if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG; + if (maxBitsSrc < tableLog) tableLog = maxBitsSrc; /* Accuracy can be reduced */ + if (minBits > tableLog) tableLog = minBits; /* Need a minimum to safely represent all symbol values */ + if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG; + if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG; + return tableLog; +} + +unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue) +{ + return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 2); +} + + +/* Secondary normalization method. + To be used when primary method fails. */ + +static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count, size_t total, U32 maxSymbolValue) +{ + U32 s; + U32 distributed = 0; + U32 ToDistribute; + + /* Init */ + U32 const lowThreshold = (U32)(total >> tableLog); + U32 lowOne = (U32)((total * 3) >> (tableLog + 1)); + + for (s=0; s<=maxSymbolValue; s++) { + if (count[s] == 0) { + norm[s]=0; + continue; + } + if (count[s] <= lowThreshold) { + norm[s] = -1; + distributed++; + total -= count[s]; + continue; + } + if (count[s] <= lowOne) { + norm[s] = 1; + distributed++; + total -= count[s]; + continue; + } + norm[s]=-2; + } + ToDistribute = (1 << tableLog) - distributed; + + if ((total / ToDistribute) > lowOne) { + /* risk of rounding to zero */ + lowOne = (U32)((total * 3) / (ToDistribute * 2)); + for (s=0; s<=maxSymbolValue; s++) { + if ((norm[s] == -2) && (count[s] <= lowOne)) { + norm[s] = 1; + distributed++; + total -= count[s]; + continue; + } } + ToDistribute = (1 << tableLog) - distributed; + } + + if (distributed == maxSymbolValue+1) { + /* all values are pretty poor; + probably incompressible data (should have already been detected); + find max, then give all remaining points to max */ + U32 maxV = 0, maxC = 0; + for (s=0; s<=maxSymbolValue; s++) + if (count[s] > maxC) maxV=s, maxC=count[s]; + norm[maxV] += (short)ToDistribute; + return 0; + } + + { U64 const vStepLog = 62 - tableLog; + U64 const mid = (1ULL << (vStepLog-1)) - 1; + U64 const rStep = ((((U64)1<> vStepLog); + U32 const sEnd = (U32)(end >> vStepLog); + U32 const weight = sEnd - sStart; + if (weight < 1) + return ERROR(GENERIC); + norm[s] = (short)weight; + tmpTotal = end; + } } } + + return 0; +} + + +size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog, + const unsigned* count, size_t total, + unsigned maxSymbolValue) +{ + /* Sanity checks */ + if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG; + if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC); /* Unsupported size */ + if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); /* Unsupported size */ + if (tableLog < FSE_minTableLog(total, maxSymbolValue)) return ERROR(GENERIC); /* Too small tableLog, compression potentially impossible */ + + { U32 const rtbTable[] = { 0, 473195, 504333, 520860, 550000, 700000, 750000, 830000 }; + U64 const scale = 62 - tableLog; + U64 const step = ((U64)1<<62) / total; /* <== here, one division ! */ + U64 const vStep = 1ULL<<(scale-20); + int stillToDistribute = 1<> tableLog); + + for (s=0; s<=maxSymbolValue; s++) { + if (count[s] == total) return 0; /* rle special case */ + if (count[s] == 0) { normalizedCounter[s]=0; continue; } + if (count[s] <= lowThreshold) { + normalizedCounter[s] = -1; + stillToDistribute--; + } else { + short proba = (short)((count[s]*step) >> scale); + if (proba<8) { + U64 restToBeat = vStep * rtbTable[proba]; + proba += (count[s]*step) - ((U64)proba< restToBeat; + } + if (proba > largestP) largestP=proba, largest=s; + normalizedCounter[s] = proba; + stillToDistribute -= proba; + } } + if (-stillToDistribute >= (normalizedCounter[largest] >> 1)) { + /* corner case, need another normalization method */ + size_t const errorCode = FSE_normalizeM2(normalizedCounter, tableLog, count, total, maxSymbolValue); + if (FSE_isError(errorCode)) return errorCode; + } + else normalizedCounter[largest] += (short)stillToDistribute; + } + +#if 0 + { /* Print Table (debug) */ + U32 s; + U32 nTotal = 0; + for (s=0; s<=maxSymbolValue; s++) + printf("%3i: %4i \n", s, normalizedCounter[s]); + for (s=0; s<=maxSymbolValue; s++) + nTotal += abs(normalizedCounter[s]); + if (nTotal != (1U<>1); /* assumption : tableLog >= 1 */ + FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT); + unsigned s; + + /* Sanity checks */ + if (nbBits < 1) return ERROR(GENERIC); /* min size */ + + /* header */ + tableU16[-2] = (U16) nbBits; + tableU16[-1] = (U16) maxSymbolValue; + + /* Build table */ + for (s=0; s FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) { /* test bit 2 */ + FSE_encodeSymbol(&bitC, &CState2, *--ip); + FSE_encodeSymbol(&bitC, &CState1, *--ip); + FSE_FLUSHBITS(&bitC); + } + + /* 2 or 4 encoding per loop */ + while ( ip>istart ) { + + FSE_encodeSymbol(&bitC, &CState2, *--ip); + + if (sizeof(bitC.bitContainer)*8 < FSE_MAX_TABLELOG*2+7 ) /* this test must be static */ + FSE_FLUSHBITS(&bitC); + + FSE_encodeSymbol(&bitC, &CState1, *--ip); + + if (sizeof(bitC.bitContainer)*8 > FSE_MAX_TABLELOG*4+7 ) { /* this test must be static */ + FSE_encodeSymbol(&bitC, &CState2, *--ip); + FSE_encodeSymbol(&bitC, &CState1, *--ip); + } + + FSE_FLUSHBITS(&bitC); + } + + FSE_flushCState(&bitC, &CState2); + FSE_flushCState(&bitC, &CState1); + return BIT_closeCStream(&bitC); +} + +size_t FSE_compress_usingCTable (void* dst, size_t dstSize, + const void* src, size_t srcSize, + const FSE_CTable* ct) +{ + unsigned const fast = (dstSize >= FSE_BLOCKBOUND(srcSize)); + + if (fast) + return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 1); + else + return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 0); +} + + +size_t FSE_compressBound(size_t size) { return FSE_COMPRESSBOUND(size); } + +#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return f +#define CHECK_F(f) { CHECK_V_F(_var_err__, f); } + +/* FSE_compress_wksp() : + * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`). + * `wkspSize` size must be `(1< not compressible */ + if (maxCount < (srcSize >> 7)) return 0; /* Heuristic : not compressible enough */ + } + + tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue); + CHECK_F( FSE_normalizeCount(norm, tableLog, count, srcSize, maxSymbolValue) ); + + /* Write table description header */ + { CHECK_V_F(nc_err, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) ); + op += nc_err; + } + + /* Compress */ + CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, scratchBufferSize) ); + { CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, src, srcSize, CTable) ); + if (cSize == 0) return 0; /* not enough space for compressed data */ + op += cSize; + } + + /* check compressibility */ + if ( (size_t)(op-ostart) >= srcSize-1 ) return 0; + + return op-ostart; +} + +typedef struct { + FSE_CTable CTable_max[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)]; + BYTE scratchBuffer[1 << FSE_MAX_TABLELOG]; +} fseWkspMax_t; + +size_t FSE_compress2 (void* dst, size_t dstCapacity, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog) +{ + fseWkspMax_t scratchBuffer; + FSE_STATIC_ASSERT(sizeof(scratchBuffer) >= FSE_WKSP_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)); /* compilation failures here means scratchBuffer is not large enough */ + if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); + return FSE_compress_wksp(dst, dstCapacity, src, srcSize, maxSymbolValue, tableLog, &scratchBuffer, sizeof(scratchBuffer)); +} + +size_t FSE_compress (void* dst, size_t dstCapacity, const void* src, size_t srcSize) +{ + return FSE_compress2(dst, dstCapacity, src, srcSize, FSE_MAX_SYMBOL_VALUE, FSE_DEFAULT_TABLELOG); +} + + +#endif /* FSE_COMMONDEFS_ONLY */ diff --git a/contrib/lizard/lib/entropy/fse_decompress.c b/contrib/lizard/lib/entropy/fse_decompress.c new file mode 100644 index 00000000000..1479a5e8217 --- /dev/null +++ b/contrib/lizard/lib/entropy/fse_decompress.c @@ -0,0 +1,329 @@ +/* ****************************************************************** + FSE : Finite State Entropy decoder + Copyright (C) 2013-2015, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ + + +/* ************************************************************** +* Compiler specifics +****************************************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define FORCE_INLINE static __forceinline +# include /* For Visual 2005 */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ +#else +# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# ifdef __GNUC__ +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +# else +# define FORCE_INLINE static +# endif /* __STDC_VERSION__ */ +#endif + + +/* ************************************************************** +* Includes +****************************************************************/ +#include /* malloc, free, qsort */ +#include /* memcpy, memset */ +#include /* printf (debug) */ +#include "bitstream.h" +#define FSE_STATIC_LINKING_ONLY +#include "fse.h" + + +/* ************************************************************** +* Error Management +****************************************************************/ +#define FSE_isError ERR_isError +#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ + +/* check and forward error code */ +#define CHECK_F(f) { size_t const e = f; if (FSE_isError(e)) return e; } + + +/* ************************************************************** +* Templates +****************************************************************/ +/* + designed to be included + for type-specific functions (template emulation in C) + Objective is to write these functions only once, for improved maintenance +*/ + +/* safety checks */ +#ifndef FSE_FUNCTION_EXTENSION +# error "FSE_FUNCTION_EXTENSION must be defined" +#endif +#ifndef FSE_FUNCTION_TYPE +# error "FSE_FUNCTION_TYPE must be defined" +#endif + +/* Function names */ +#define FSE_CAT(X,Y) X##Y +#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y) +#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y) + + +/* Function templates */ +FSE_DTable* FSE_createDTable (unsigned tableLog) +{ + if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX; + return (FSE_DTable*)malloc( FSE_DTABLE_SIZE_U32(tableLog) * sizeof (U32) ); +} + +void FSE_freeDTable (FSE_DTable* dt) +{ + free(dt); +} + +size_t FSE_buildDTable(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog) +{ + void* const tdPtr = dt+1; /* because *dt is unsigned, 32-bits aligned on 32-bits */ + FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (tdPtr); + U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1]; + + U32 const maxSV1 = maxSymbolValue + 1; + U32 const tableSize = 1 << tableLog; + U32 highThreshold = tableSize-1; + + /* Sanity Checks */ + if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge); + if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); + + /* Init, lay down lowprob symbols */ + { FSE_DTableHeader DTableH; + DTableH.tableLog = (U16)tableLog; + DTableH.fastMode = 1; + { S16 const largeLimit= (S16)(1 << (tableLog-1)); + U32 s; + for (s=0; s= largeLimit) DTableH.fastMode=0; + symbolNext[s] = normalizedCounter[s]; + } } } + memcpy(dt, &DTableH, sizeof(DTableH)); + } + + /* Spread symbols */ + { U32 const tableMask = tableSize-1; + U32 const step = FSE_TABLESTEP(tableSize); + U32 s, position = 0; + for (s=0; s highThreshold) position = (position + step) & tableMask; /* lowprob area */ + } } + if (position!=0) return ERROR(GENERIC); /* position must reach all cells once, otherwise normalizedCounter is incorrect */ + } + + /* Build Decoding table */ + { U32 u; + for (u=0; utableLog = 0; + DTableH->fastMode = 0; + + cell->newState = 0; + cell->symbol = symbolValue; + cell->nbBits = 0; + + return 0; +} + + +size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits) +{ + void* ptr = dt; + FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr; + void* dPtr = dt + 1; + FSE_decode_t* const dinfo = (FSE_decode_t*)dPtr; + const unsigned tableSize = 1 << nbBits; + const unsigned tableMask = tableSize - 1; + const unsigned maxSV1 = tableMask+1; + unsigned s; + + /* Sanity checks */ + if (nbBits < 1) return ERROR(GENERIC); /* min size */ + + /* Build Decoding Table */ + DTableH->tableLog = (U16)nbBits; + DTableH->fastMode = 1; + for (s=0; s sizeof(bitD.bitContainer)*8) /* This test must be static */ + BIT_reloadDStream(&bitD); + + op[1] = FSE_GETSYMBOL(&state2); + + if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */ + { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } } + + op[2] = FSE_GETSYMBOL(&state1); + + if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */ + BIT_reloadDStream(&bitD); + + op[3] = FSE_GETSYMBOL(&state2); + } + + /* tail */ + /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */ + while (1) { + if (op>(omax-2)) return ERROR(dstSize_tooSmall); + *op++ = FSE_GETSYMBOL(&state1); + if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) { + *op++ = FSE_GETSYMBOL(&state2); + break; + } + + if (op>(omax-2)) return ERROR(dstSize_tooSmall); + *op++ = FSE_GETSYMBOL(&state2); + if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) { + *op++ = FSE_GETSYMBOL(&state1); + break; + } } + + return op-ostart; +} + + +size_t FSE_decompress_usingDTable(void* dst, size_t originalSize, + const void* cSrc, size_t cSrcSize, + const FSE_DTable* dt) +{ + const void* ptr = dt; + const FSE_DTableHeader* DTableH = (const FSE_DTableHeader*)ptr; + const U32 fastMode = DTableH->fastMode; + + /* select fast mode (static) */ + if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1); + return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0); +} + + +size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog) +{ + const BYTE* const istart = (const BYTE*)cSrc; + const BYTE* ip = istart; + short counting[FSE_MAX_SYMBOL_VALUE+1]; + unsigned tableLog; + unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE; + + /* normal FSE decoding mode */ + size_t const NCountLength = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize); + if (FSE_isError(NCountLength)) return NCountLength; + //if (NCountLength >= cSrcSize) return ERROR(srcSize_wrong); /* too small input size; supposed to be already checked in NCountLength, only remaining case : NCountLength==cSrcSize */ + if (tableLog > maxLog) return ERROR(tableLog_tooLarge); + ip += NCountLength; + cSrcSize -= NCountLength; + + CHECK_F( FSE_buildDTable (workSpace, counting, maxSymbolValue, tableLog) ); + + return FSE_decompress_usingDTable (dst, dstCapacity, ip, cSrcSize, workSpace); /* always return, even if it is an error code */ +} + + +typedef FSE_DTable DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)]; + +size_t FSE_decompress(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize) +{ + DTable_max_t dt; /* Static analyzer seems unable to understand this table will be properly initialized later */ + return FSE_decompress_wksp(dst, dstCapacity, cSrc, cSrcSize, dt, FSE_MAX_TABLELOG); +} + + + +#endif /* FSE_COMMONDEFS_ONLY */ diff --git a/contrib/lizard/lib/entropy/huf.h b/contrib/lizard/lib/entropy/huf.h new file mode 100644 index 00000000000..48c7f831f55 --- /dev/null +++ b/contrib/lizard/lib/entropy/huf.h @@ -0,0 +1,250 @@ +/* ****************************************************************** + Huffman coder, part of New Generation Entropy library + header file + Copyright (C) 2013-2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Source repository : https://github.com/Cyan4973/FiniteStateEntropy +****************************************************************** */ +#ifndef HUF_H_298734234 +#define HUF_H_298734234 + +#if defined (__cplusplus) +extern "C" { +#endif + + +/* *** Dependencies *** */ +#include /* size_t */ + + +/*-*** PUBLIC_API : control library symbols visibility *** */ +#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4) +# define HUF_PUBLIC_API __attribute__ ((visibility ("default"))) +#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) /* Visual expected */ +# define HUF_PUBLIC_API __declspec(dllexport) +#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1) +# define HUF_PUBLIC_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ +#else +# define HUF_PUBLIC_API +#endif + + +/* *** simple functions *** */ +/** +HUF_compress() : + Compress content from buffer 'src', of size 'srcSize', into buffer 'dst'. + 'dst' buffer must be already allocated. + Compression runs faster if `dstCapacity` >= HUF_compressBound(srcSize). + `srcSize` must be <= `HUF_BLOCKSIZE_MAX` == 128 KB. + @return : size of compressed data (<= `dstCapacity`). + Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!! + if return == 1, srcData is a single repeated byte symbol (RLE compression). + if HUF_isError(return), compression failed (more details using HUF_getErrorName()) +*/ +HUF_PUBLIC_API size_t HUF_compress(void* dst, size_t dstCapacity, + const void* src, size_t srcSize); + +/** +HUF_decompress() : + Decompress HUF data from buffer 'cSrc', of size 'cSrcSize', + into already allocated buffer 'dst', of minimum size 'dstSize'. + `originalSize` : **must** be the ***exact*** size of original (uncompressed) data. + Note : in contrast with FSE, HUF_decompress can regenerate + RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data, + because it knows size to regenerate. + @return : size of regenerated data (== originalSize), + or an error code, which can be tested using HUF_isError() +*/ +HUF_PUBLIC_API size_t HUF_decompress(void* dst, size_t originalSize, + const void* cSrc, size_t cSrcSize); + + +/* *** Tool functions *** */ +#define HUF_BLOCKSIZE_MAX (128 * 1024) /**< maximum input size for a single block compressed with HUF_compress */ +HUF_PUBLIC_API size_t HUF_compressBound(size_t size); /**< maximum compressed size (worst case) */ + +/* Error Management */ +HUF_PUBLIC_API unsigned HUF_isError(size_t code); /**< tells if a return value is an error code */ +HUF_PUBLIC_API const char* HUF_getErrorName(size_t code); /**< provides error code string (useful for debugging) */ + + +/* *** Advanced function *** */ + +/** HUF_compress2() : + * Same as HUF_compress(), but offers direct control over `maxSymbolValue` and `tableLog` . + * `tableLog` must be `<= HUF_TABLELOG_MAX` . */ +HUF_PUBLIC_API size_t HUF_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); + +/** HUF_compress4X_wksp() : +* Same as HUF_compress2(), but uses externally allocated `workSpace`, which must be a table of >= 1024 unsigned */ +HUF_PUBLIC_API size_t HUF_compress4X_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); /**< `workSpace` must be a table of at least 1024 unsigned */ + + + +#ifdef HUF_STATIC_LINKING_ONLY + +/* *** Dependencies *** */ +#include "mem.h" /* U32 */ + + +/* *** Constants *** */ +#define HUF_TABLELOG_ABSOLUTEMAX 15 /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */ +#define HUF_TABLELOG_MAX 12 /* max configured tableLog (for static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */ +#define HUF_TABLELOG_DEFAULT 11 /* tableLog by default, when not specified */ +#define HUF_SYMBOLVALUE_MAX 255 +#if (HUF_TABLELOG_MAX > HUF_TABLELOG_ABSOLUTEMAX) +# error "HUF_TABLELOG_MAX is too large !" +#endif + + +/* **************************************** +* Static allocation +******************************************/ +/* HUF buffer bounds */ +#define HUF_CTABLEBOUND 129 +#define HUF_BLOCKBOUND(size) (size + (size>>8) + 8) /* only true if incompressible pre-filtered with fast heuristic */ +#define HUF_COMPRESSBOUND(size) (HUF_CTABLEBOUND + HUF_BLOCKBOUND(size)) /* Macro version, useful for static allocation */ + +/* static allocation of HUF's Compression Table */ +#define HUF_CREATE_STATIC_CTABLE(name, maxSymbolValue) \ + U32 name##hb[maxSymbolValue+1]; \ + void* name##hv = &(name##hb); \ + HUF_CElt* name = (HUF_CElt*)(name##hv) /* no final ; */ + +/* static allocation of HUF's DTable */ +typedef U32 HUF_DTable; +#define HUF_DTABLE_SIZE(maxTableLog) (1 + (1<<(maxTableLog))) +#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \ + HUF_DTable DTable[HUF_DTABLE_SIZE((maxTableLog)-1)] = { ((U32)((maxTableLog)-1) * 0x01000001) } +#define HUF_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \ + HUF_DTable DTable[HUF_DTABLE_SIZE(maxTableLog)] = { ((U32)(maxTableLog) * 0x01000001) } + + +/* **************************************** +* Advanced decompression functions +******************************************/ +size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */ +size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */ + +size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< decodes RLE and uncompressed */ +size_t HUF_decompress4X_hufOnly(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< considers RLE and uncompressed as errors */ +size_t HUF_decompress4X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */ +size_t HUF_decompress4X4_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */ + + +/* **************************************** +* HUF detailed API +******************************************/ +/*! +HUF_compress() does the following: +1. count symbol occurrence from source[] into table count[] using FSE_count() +2. (optional) refine tableLog using HUF_optimalTableLog() +3. build Huffman table from count using HUF_buildCTable() +4. save Huffman table to memory buffer using HUF_writeCTable() +5. encode the data stream using HUF_compress4X_usingCTable() + +The following API allows targeting specific sub-functions for advanced tasks. +For example, it's possible to compress several blocks using the same 'CTable', +or to save and regenerate 'CTable' using external methods. +*/ +/* FSE_count() : find it within "fse.h" */ +unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue); +typedef struct HUF_CElt_s HUF_CElt; /* incomplete type */ +size_t HUF_buildCTable (HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue, unsigned maxNbBits); +size_t HUF_writeCTable (void* dst, size_t maxDstSize, const HUF_CElt* CTable, unsigned maxSymbolValue, unsigned huffLog); +size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable); + + +/** HUF_buildCTable_wksp() : + * Same as HUF_buildCTable(), but using externally allocated scratch buffer. + * `workSpace` must be aligned on 4-bytes boundaries, and be at least as large as a table of 1024 unsigned. + */ +size_t HUF_buildCTable_wksp (HUF_CElt* tree, const U32* count, U32 maxSymbolValue, U32 maxNbBits, void* workSpace, size_t wkspSize); + +/*! HUF_readStats() : + Read compact Huffman tree, saved by HUF_writeCTable(). + `huffWeight` is destination buffer. + @return : size read from `src` , or an error Code . + Note : Needed by HUF_readCTable() and HUF_readDTableXn() . */ +size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats, + U32* nbSymbolsPtr, U32* tableLogPtr, + const void* src, size_t srcSize); + +/** HUF_readCTable() : +* Loading a CTable saved with HUF_writeCTable() */ +size_t HUF_readCTable (HUF_CElt* CTable, unsigned maxSymbolValue, const void* src, size_t srcSize); + + +/* +HUF_decompress() does the following: +1. select the decompression algorithm (X2, X4) based on pre-computed heuristics +2. build Huffman table from save, using HUF_readDTableXn() +3. decode 1 or 4 segments in parallel using HUF_decompressSXn_usingDTable +*/ + +/** HUF_selectDecoder() : +* Tells which decoder is likely to decode faster, +* based on a set of pre-determined metrics. +* @return : 0==HUF_decompress4X2, 1==HUF_decompress4X4 . +* Assumption : 0 < cSrcSize < dstSize <= 128 KB */ +U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize); + +size_t HUF_readDTableX2 (HUF_DTable* DTable, const void* src, size_t srcSize); +size_t HUF_readDTableX4 (HUF_DTable* DTable, const void* src, size_t srcSize); + +size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); +size_t HUF_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); +size_t HUF_decompress4X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); + + +/* single stream variants */ + +size_t HUF_compress1X (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); +size_t HUF_compress1X_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); /**< `workSpace` must be a table of at least 1024 unsigned */ +size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable); + +size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /* single-symbol decoder */ +size_t HUF_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /* double-symbol decoder */ + +size_t HUF_decompress1X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); +size_t HUF_decompress1X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */ +size_t HUF_decompress1X4_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */ + +size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); /**< automatic selection of sing or double symbol decoder, based on DTable */ +size_t HUF_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); +size_t HUF_decompress1X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); + +#endif /* HUF_STATIC_LINKING_ONLY */ + + +#if defined (__cplusplus) +} +#endif + +#endif /* HUF_H_298734234 */ diff --git a/contrib/lizard/lib/entropy/huf_compress.c b/contrib/lizard/lib/entropy/huf_compress.c new file mode 100644 index 00000000000..a47a1a2c761 --- /dev/null +++ b/contrib/lizard/lib/entropy/huf_compress.c @@ -0,0 +1,612 @@ +/* ****************************************************************** + Huffman encoder, part of New Generation Entropy library + Copyright (C) 2013-2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ + +/* ************************************************************** +* Compiler specifics +****************************************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +#endif + + +/* ************************************************************** +* Includes +****************************************************************/ +#include /* memcpy, memset */ +#include /* printf (debug) */ +#include "bitstream.h" +#define FSE_STATIC_LINKING_ONLY /* FSE_optimalTableLog_internal */ +#include "fse.h" /* header compression */ +#define HUF_STATIC_LINKING_ONLY +#include "huf.h" + + +/* ************************************************************** +* Error Management +****************************************************************/ +#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ +#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return f +#define CHECK_F(f) { CHECK_V_F(_var_err__, f); } + + +/* ************************************************************** +* Utils +****************************************************************/ +unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue) +{ + return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 1); +} + + +/* ******************************************************* +* HUF : Huffman block compression +*********************************************************/ +/* HUF_compressWeights() : + * Same as FSE_compress(), but dedicated to huff0's weights compression. + * The use case needs much less stack memory. + * Note : all elements within weightTable are supposed to be <= HUF_TABLELOG_MAX. + */ +#define MAX_FSE_TABLELOG_FOR_HUFF_HEADER 6 +size_t HUF_compressWeights (void* dst, size_t dstSize, const void* weightTable, size_t wtSize) +{ + BYTE* const ostart = (BYTE*) dst; + BYTE* op = ostart; + BYTE* const oend = ostart + dstSize; + + U32 maxSymbolValue = HUF_TABLELOG_MAX; + U32 tableLog = MAX_FSE_TABLELOG_FOR_HUFF_HEADER; + + FSE_CTable CTable[FSE_CTABLE_SIZE_U32(MAX_FSE_TABLELOG_FOR_HUFF_HEADER, HUF_TABLELOG_MAX)]; + BYTE scratchBuffer[1< not compressible */ + } + + tableLog = FSE_optimalTableLog(tableLog, wtSize, maxSymbolValue); + CHECK_F( FSE_normalizeCount(norm, tableLog, count, wtSize, maxSymbolValue) ); + + /* Write table description header */ + { CHECK_V_F(hSize, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) ); + op += hSize; + } + + /* Compress */ + CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, sizeof(scratchBuffer)) ); + { CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, weightTable, wtSize, CTable) ); + if (cSize == 0) return 0; /* not enough space for compressed data */ + op += cSize; + } + + return op-ostart; +} + + +struct HUF_CElt_s { + U16 val; + BYTE nbBits; +}; /* typedef'd to HUF_CElt within "huf.h" */ + +/*! HUF_writeCTable() : + `CTable` : huffman tree to save, using huf representation. + @return : size of saved CTable */ +size_t HUF_writeCTable (void* dst, size_t maxDstSize, + const HUF_CElt* CTable, U32 maxSymbolValue, U32 huffLog) +{ + BYTE bitsToWeight[HUF_TABLELOG_MAX + 1]; /* precomputed conversion table */ + BYTE huffWeight[HUF_SYMBOLVALUE_MAX]; + BYTE* op = (BYTE*)dst; + U32 n; + + /* check conditions */ + if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge); + + /* convert to weight */ + bitsToWeight[0] = 0; + for (n=1; n1) & (hSize < maxSymbolValue/2)) { /* FSE compressed */ + op[0] = (BYTE)hSize; + return hSize+1; + } } + + /* write raw values as 4-bits (max : 15) */ + if (maxSymbolValue > (256-128)) return ERROR(GENERIC); /* should not happen : likely means source cannot be compressed */ + if (((maxSymbolValue+1)/2) + 1 > maxDstSize) return ERROR(dstSize_tooSmall); /* not enough space within dst buffer */ + op[0] = (BYTE)(128 /*special case*/ + (maxSymbolValue-1)); + huffWeight[maxSymbolValue] = 0; /* to be sure it doesn't cause msan issue in final combination */ + for (n=0; n HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge); + if (nbSymbols > maxSymbolValue+1) return ERROR(maxSymbolValue_tooSmall); + + /* Prepare base value per rank */ + { U32 n, nextRankStart = 0; + for (n=1; n<=tableLog; n++) { + U32 current = nextRankStart; + nextRankStart += (rankVal[n] << (n-1)); + rankVal[n] = current; + } } + + /* fill nbBits */ + { U32 n; for (n=0; nn=tableLog+1 */ + U16 valPerRank[HUF_TABLELOG_MAX+2] = {0}; + { U32 n; for (n=0; n0; n--) { /* start at n=tablelog <-> w=1 */ + valPerRank[n] = min; /* get starting value within each rank */ + min += nbPerRank[n]; + min >>= 1; + } } + /* assign value within rank, symbol order */ + { U32 n; for (n=0; n<=maxSymbolValue; n++) CTable[n].val = valPerRank[CTable[n].nbBits]++; } + } + + return readSize; +} + + +typedef struct nodeElt_s { + U32 count; + U16 parent; + BYTE byte; + BYTE nbBits; +} nodeElt; + +static U32 HUF_setMaxHeight(nodeElt* huffNode, U32 lastNonNull, U32 maxNbBits) +{ + const U32 largestBits = huffNode[lastNonNull].nbBits; + if (largestBits <= maxNbBits) return largestBits; /* early exit : no elt > maxNbBits */ + + /* there are several too large elements (at least >= 2) */ + { int totalCost = 0; + const U32 baseCost = 1 << (largestBits - maxNbBits); + U32 n = lastNonNull; + + while (huffNode[n].nbBits > maxNbBits) { + totalCost += baseCost - (1 << (largestBits - huffNode[n].nbBits)); + huffNode[n].nbBits = (BYTE)maxNbBits; + n --; + } /* n stops at huffNode[n].nbBits <= maxNbBits */ + while (huffNode[n].nbBits == maxNbBits) n--; /* n end at index of smallest symbol using < maxNbBits */ + + /* renorm totalCost */ + totalCost >>= (largestBits - maxNbBits); /* note : totalCost is necessarily a multiple of baseCost */ + + /* repay normalized cost */ + { U32 const noSymbol = 0xF0F0F0F0; + U32 rankLast[HUF_TABLELOG_MAX+2]; + int pos; + + /* Get pos of last (smallest) symbol per rank */ + memset(rankLast, 0xF0, sizeof(rankLast)); + { U32 currentNbBits = maxNbBits; + for (pos=n ; pos >= 0; pos--) { + if (huffNode[pos].nbBits >= currentNbBits) continue; + currentNbBits = huffNode[pos].nbBits; /* < maxNbBits */ + rankLast[maxNbBits-currentNbBits] = pos; + } } + + while (totalCost > 0) { + U32 nBitsToDecrease = BIT_highbit32(totalCost) + 1; + for ( ; nBitsToDecrease > 1; nBitsToDecrease--) { + U32 highPos = rankLast[nBitsToDecrease]; + U32 lowPos = rankLast[nBitsToDecrease-1]; + if (highPos == noSymbol) continue; + if (lowPos == noSymbol) break; + { U32 const highTotal = huffNode[highPos].count; + U32 const lowTotal = 2 * huffNode[lowPos].count; + if (highTotal <= lowTotal) break; + } } + /* only triggered when no more rank 1 symbol left => find closest one (note : there is necessarily at least one !) */ + while ((nBitsToDecrease<=HUF_TABLELOG_MAX) && (rankLast[nBitsToDecrease] == noSymbol)) /* HUF_MAX_TABLELOG test just to please gcc 5+; but it should not be necessary */ + nBitsToDecrease ++; + totalCost -= 1 << (nBitsToDecrease-1); + if (rankLast[nBitsToDecrease-1] == noSymbol) + rankLast[nBitsToDecrease-1] = rankLast[nBitsToDecrease]; /* this rank is no longer empty */ + huffNode[rankLast[nBitsToDecrease]].nbBits ++; + if (rankLast[nBitsToDecrease] == 0) /* special case, reached largest symbol */ + rankLast[nBitsToDecrease] = noSymbol; + else { + rankLast[nBitsToDecrease]--; + if (huffNode[rankLast[nBitsToDecrease]].nbBits != maxNbBits-nBitsToDecrease) + rankLast[nBitsToDecrease] = noSymbol; /* this rank is now empty */ + } } /* while (totalCost > 0) */ + + while (totalCost < 0) { /* Sometimes, cost correction overshoot */ + if (rankLast[1] == noSymbol) { /* special case : no rank 1 symbol (using maxNbBits-1); let's create one from largest rank 0 (using maxNbBits) */ + while (huffNode[n].nbBits == maxNbBits) n--; + huffNode[n+1].nbBits--; + rankLast[1] = n+1; + totalCost++; + continue; + } + huffNode[ rankLast[1] + 1 ].nbBits--; + rankLast[1]++; + totalCost ++; + } } } /* there are several too large elements (at least >= 2) */ + + return maxNbBits; +} + + +typedef struct { + U32 base; + U32 current; +} rankPos; + +static void HUF_sort(nodeElt* huffNode, const U32* count, U32 maxSymbolValue) +{ + rankPos rank[32]; + U32 n; + + memset(rank, 0, sizeof(rank)); + for (n=0; n<=maxSymbolValue; n++) { + U32 r = BIT_highbit32(count[n] + 1); + rank[r].base ++; + } + for (n=30; n>0; n--) rank[n-1].base += rank[n].base; + for (n=0; n<32; n++) rank[n].current = rank[n].base; + for (n=0; n<=maxSymbolValue; n++) { + U32 const c = count[n]; + U32 const r = BIT_highbit32(c+1) + 1; + U32 pos = rank[r].current++; + while ((pos > rank[r].base) && (c > huffNode[pos-1].count)) huffNode[pos]=huffNode[pos-1], pos--; + huffNode[pos].count = c; + huffNode[pos].byte = (BYTE)n; + } +} + + +/** HUF_buildCTable_wksp() : + * Same as HUF_buildCTable(), but using externally allocated scratch buffer. + * `workSpace` must be aligned on 4-bytes boundaries, and be at least as large as a table of 1024 unsigned. + */ +#define STARTNODE (HUF_SYMBOLVALUE_MAX+1) +typedef nodeElt huffNodeTable[2*HUF_SYMBOLVALUE_MAX+1 +1]; +size_t HUF_buildCTable_wksp (HUF_CElt* tree, const U32* count, U32 maxSymbolValue, U32 maxNbBits, void* workSpace, size_t wkspSize) +{ + nodeElt* const huffNode0 = (nodeElt*)workSpace; + nodeElt* const huffNode = huffNode0+1; + U32 n, nonNullRank; + int lowS, lowN; + U16 nodeNb = STARTNODE; + U32 nodeRoot; + + /* safety checks */ + if (wkspSize < sizeof(huffNodeTable)) return ERROR(GENERIC); /* workSpace is not large enough */ + if (maxNbBits == 0) maxNbBits = HUF_TABLELOG_DEFAULT; + if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(GENERIC); + memset(huffNode0, 0, sizeof(huffNodeTable)); + + /* sort, decreasing order */ + HUF_sort(huffNode, count, maxSymbolValue); + + /* init for parents */ + nonNullRank = maxSymbolValue; + while(huffNode[nonNullRank].count == 0) nonNullRank--; + lowS = nonNullRank; nodeRoot = nodeNb + lowS - 1; lowN = nodeNb; + huffNode[nodeNb].count = huffNode[lowS].count + huffNode[lowS-1].count; + huffNode[lowS].parent = huffNode[lowS-1].parent = nodeNb; + nodeNb++; lowS-=2; + for (n=nodeNb; n<=nodeRoot; n++) huffNode[n].count = (U32)(1U<<30); + huffNode0[0].count = (U32)(1U<<31); /* fake entry, strong barrier */ + + /* create parents */ + while (nodeNb <= nodeRoot) { + U32 n1 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++; + U32 n2 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++; + huffNode[nodeNb].count = huffNode[n1].count + huffNode[n2].count; + huffNode[n1].parent = huffNode[n2].parent = nodeNb; + nodeNb++; + } + + /* distribute weights (unlimited tree height) */ + huffNode[nodeRoot].nbBits = 0; + for (n=nodeRoot-1; n>=STARTNODE; n--) + huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1; + for (n=0; n<=nonNullRank; n++) + huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1; + + /* enforce maxTableLog */ + maxNbBits = HUF_setMaxHeight(huffNode, nonNullRank, maxNbBits); + + /* fill result into tree (val, nbBits) */ + { U16 nbPerRank[HUF_TABLELOG_MAX+1] = {0}; + U16 valPerRank[HUF_TABLELOG_MAX+1] = {0}; + if (maxNbBits > HUF_TABLELOG_MAX) return ERROR(GENERIC); /* check fit into table */ + for (n=0; n<=nonNullRank; n++) + nbPerRank[huffNode[n].nbBits]++; + /* determine stating value per rank */ + { U16 min = 0; + for (n=maxNbBits; n>0; n--) { + valPerRank[n] = min; /* get starting value within each rank */ + min += nbPerRank[n]; + min >>= 1; + } } + for (n=0; n<=maxSymbolValue; n++) + tree[huffNode[n].byte].nbBits = huffNode[n].nbBits; /* push nbBits per symbol, symbol order */ + for (n=0; n<=maxSymbolValue; n++) + tree[n].val = valPerRank[tree[n].nbBits]++; /* assign value within rank, symbol order */ + } + + return maxNbBits; +} + +/** HUF_buildCTable() : + * Note : count is used before tree is written, so they can safely overlap + */ +size_t HUF_buildCTable (HUF_CElt* tree, const U32* count, U32 maxSymbolValue, U32 maxNbBits) +{ + huffNodeTable nodeTable; + return HUF_buildCTable_wksp(tree, count, maxSymbolValue, maxNbBits, nodeTable, sizeof(nodeTable)); +} + +static void HUF_encodeSymbol(BIT_CStream_t* bitCPtr, U32 symbol, const HUF_CElt* CTable) +{ + BIT_addBitsFast(bitCPtr, CTable[symbol].val, CTable[symbol].nbBits); +} + +size_t HUF_compressBound(size_t size) { return HUF_COMPRESSBOUND(size); } + +#define HUF_FLUSHBITS(s) (fast ? BIT_flushBitsFast(s) : BIT_flushBits(s)) + +#define HUF_FLUSHBITS_1(stream) \ + if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*2+7) HUF_FLUSHBITS(stream) + +#define HUF_FLUSHBITS_2(stream) \ + if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*4+7) HUF_FLUSHBITS(stream) + +size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable) +{ + const BYTE* ip = (const BYTE*) src; + BYTE* const ostart = (BYTE*)dst; + BYTE* const oend = ostart + dstSize; + BYTE* op = ostart; + size_t n; + const unsigned fast = (dstSize >= HUF_BLOCKBOUND(srcSize)); + BIT_CStream_t bitC; + + /* init */ + if (dstSize < 8) return 0; /* not enough space to compress */ + { size_t const initErr = BIT_initCStream(&bitC, op, oend-op); + if (HUF_isError(initErr)) return 0; } + + n = srcSize & ~3; /* join to mod 4 */ + switch (srcSize & 3) + { + case 3 : HUF_encodeSymbol(&bitC, ip[n+ 2], CTable); + HUF_FLUSHBITS_2(&bitC); + // fallthrough + case 2 : HUF_encodeSymbol(&bitC, ip[n+ 1], CTable); + HUF_FLUSHBITS_1(&bitC); + // fallthrough + case 1 : HUF_encodeSymbol(&bitC, ip[n+ 0], CTable); + HUF_FLUSHBITS(&bitC); + // fallthrough + case 0 : + default: ; + } + + for (; n>0; n-=4) { /* note : n&3==0 at this stage */ + HUF_encodeSymbol(&bitC, ip[n- 1], CTable); + HUF_FLUSHBITS_1(&bitC); + HUF_encodeSymbol(&bitC, ip[n- 2], CTable); + HUF_FLUSHBITS_2(&bitC); + HUF_encodeSymbol(&bitC, ip[n- 3], CTable); + HUF_FLUSHBITS_1(&bitC); + HUF_encodeSymbol(&bitC, ip[n- 4], CTable); + HUF_FLUSHBITS(&bitC); + } + + return BIT_closeCStream(&bitC); +} + + +size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable) +{ + size_t const segmentSize = (srcSize+3)/4; /* first 3 segments */ + const BYTE* ip = (const BYTE*) src; + const BYTE* const iend = ip + srcSize; + BYTE* const ostart = (BYTE*) dst; + BYTE* const oend = ostart + dstSize; + BYTE* op = ostart; + + if (dstSize < 6 + 1 + 1 + 1 + 8) return 0; /* minimum space to compress successfully */ + if (srcSize < 12) return 0; /* no saving possible : too small input */ + op += 6; /* jumpTable */ + + { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, segmentSize, CTable) ); + if (cSize==0) return 0; + MEM_writeLE16(ostart, (U16)cSize); + op += cSize; + } + + ip += segmentSize; + { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, segmentSize, CTable) ); + if (cSize==0) return 0; + MEM_writeLE16(ostart+2, (U16)cSize); + op += cSize; + } + + ip += segmentSize; + { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, segmentSize, CTable) ); + if (cSize==0) return 0; + MEM_writeLE16(ostart+4, (U16)cSize); + op += cSize; + } + + ip += segmentSize; + { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, iend-ip, CTable) ); + if (cSize==0) return 0; + op += cSize; + } + + return op-ostart; +} + + +/* `workSpace` must a table of at least 1024 unsigned */ +static size_t HUF_compress_internal ( + void* dst, size_t dstSize, + const void* src, size_t srcSize, + unsigned maxSymbolValue, unsigned huffLog, + unsigned singleStream, + void* workSpace, size_t wkspSize) +{ + BYTE* const ostart = (BYTE*)dst; + BYTE* const oend = ostart + dstSize; + BYTE* op = ostart; + + union { + U32 count[HUF_SYMBOLVALUE_MAX+1]; + HUF_CElt CTable[HUF_SYMBOLVALUE_MAX+1]; + } table; /* `count` can overlap with `CTable`; saves 1 KB */ + + /* checks & inits */ + if (wkspSize < sizeof(huffNodeTable)) return ERROR(GENERIC); + if (!srcSize) return 0; /* Uncompressed (note : 1 means rle, so first byte must be correct) */ + if (!dstSize) return 0; /* cannot fit within dst budget */ + if (srcSize > HUF_BLOCKSIZE_MAX) return ERROR(srcSize_wrong); /* current block size limit */ + if (huffLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge); + if (!maxSymbolValue) maxSymbolValue = HUF_SYMBOLVALUE_MAX; + if (!huffLog) huffLog = HUF_TABLELOG_DEFAULT; + + /* Scan input and build symbol stats */ + { CHECK_V_F(largest, FSE_count_wksp (table.count, &maxSymbolValue, (const BYTE*)src, srcSize, (U32*)workSpace) ); + if (largest == srcSize) { *ostart = ((const BYTE*)src)[0]; return 1; } /* single symbol, rle */ + if (largest <= (srcSize >> 7)+1) return 0; /* Fast heuristic : not compressible enough */ + } + + /* Build Huffman Tree */ + huffLog = HUF_optimalTableLog(huffLog, srcSize, maxSymbolValue); + { CHECK_V_F(maxBits, HUF_buildCTable_wksp (table.CTable, table.count, maxSymbolValue, huffLog, workSpace, wkspSize) ); + huffLog = (U32)maxBits; + } + + /* Write table description header */ + { CHECK_V_F(hSize, HUF_writeCTable (op, dstSize, table.CTable, maxSymbolValue, huffLog) ); + if (hSize + 12 >= srcSize) return 0; /* not useful to try compression */ + op += hSize; + } + + /* Compress */ + { size_t const cSize = (singleStream) ? + HUF_compress1X_usingCTable(op, oend - op, src, srcSize, table.CTable) : /* single segment */ + HUF_compress4X_usingCTable(op, oend - op, src, srcSize, table.CTable); + if (HUF_isError(cSize)) return cSize; + if (cSize==0) return 0; /* uncompressible */ + op += cSize; + } + + /* check compressibility */ + if ((size_t)(op-ostart) >= srcSize-1) + return 0; + + return op-ostart; +} + + +size_t HUF_compress1X_wksp (void* dst, size_t dstSize, + const void* src, size_t srcSize, + unsigned maxSymbolValue, unsigned huffLog, + void* workSpace, size_t wkspSize) +{ + return HUF_compress_internal(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, 1 /* single stream */, workSpace, wkspSize); +} + +size_t HUF_compress1X (void* dst, size_t dstSize, + const void* src, size_t srcSize, + unsigned maxSymbolValue, unsigned huffLog) +{ + unsigned workSpace[1024]; + return HUF_compress1X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace)); +} + +size_t HUF_compress4X_wksp (void* dst, size_t dstSize, + const void* src, size_t srcSize, + unsigned maxSymbolValue, unsigned huffLog, + void* workSpace, size_t wkspSize) +{ + return HUF_compress_internal(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, 0 /* 4 streams */, workSpace, wkspSize); +} + +size_t HUF_compress2 (void* dst, size_t dstSize, + const void* src, size_t srcSize, + unsigned maxSymbolValue, unsigned huffLog) +{ + unsigned workSpace[1024]; + return HUF_compress4X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace)); +} + +size_t HUF_compress (void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + return HUF_compress2(dst, maxDstSize, src, (U32)srcSize, 255, HUF_TABLELOG_DEFAULT); +} diff --git a/contrib/lizard/lib/entropy/huf_decompress.c b/contrib/lizard/lib/entropy/huf_decompress.c new file mode 100644 index 00000000000..a342dfb1e4a --- /dev/null +++ b/contrib/lizard/lib/entropy/huf_decompress.c @@ -0,0 +1,885 @@ +/* ****************************************************************** + Huffman decoder, part of New Generation Entropy library + Copyright (C) 2013-2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ + +/* ************************************************************** +* Compiler specifics +****************************************************************/ +#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) +/* inline is defined */ +#elif defined(_MSC_VER) || defined(__GNUC__) +# define inline __inline +#else +# define inline /* disable inline */ +#endif + +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +#endif + + +/* ************************************************************** +* Dependencies +****************************************************************/ +#include /* memcpy, memset */ +#include "bitstream.h" /* BIT_* */ +#include "fse.h" /* header compression */ +#define HUF_STATIC_LINKING_ONLY +#include "huf.h" + + +/* ************************************************************** +* Error Management +****************************************************************/ +#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ + + +/*-***************************/ +/* generic DTableDesc */ +/*-***************************/ + +typedef struct { BYTE maxTableLog; BYTE tableType; BYTE tableLog; BYTE reserved; } DTableDesc; + +static DTableDesc HUF_getDTableDesc(const HUF_DTable* table) +{ + DTableDesc dtd; + memcpy(&dtd, table, sizeof(dtd)); + return dtd; +} + + +/*-***************************/ +/* single-symbol decoding */ +/*-***************************/ + +typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX2; /* single-symbol decoding */ + +size_t HUF_readDTableX2 (HUF_DTable* DTable, const void* src, size_t srcSize) +{ + BYTE huffWeight[HUF_SYMBOLVALUE_MAX + 1]; + U32 rankVal[HUF_TABLELOG_ABSOLUTEMAX + 1]; /* large enough for values from 0 to 16 */ + U32 tableLog = 0; + U32 nbSymbols = 0; + size_t iSize; + void* const dtPtr = DTable + 1; + HUF_DEltX2* const dt = (HUF_DEltX2*)dtPtr; + + HUF_STATIC_ASSERT(sizeof(DTableDesc) == sizeof(HUF_DTable)); + /* memset(huffWeight, 0, sizeof(huffWeight)); */ /* is not necessary, even though some analyzer complain ... */ + + iSize = HUF_readStats(huffWeight, HUF_SYMBOLVALUE_MAX + 1, rankVal, &nbSymbols, &tableLog, src, srcSize); + if (HUF_isError(iSize)) return iSize; + + /* Table header */ + { DTableDesc dtd = HUF_getDTableDesc(DTable); + if (tableLog > (U32)(dtd.maxTableLog+1)) return ERROR(tableLog_tooLarge); /* DTable too small, huffman tree cannot fit in */ + dtd.tableType = 0; + dtd.tableLog = (BYTE)tableLog; + memcpy(DTable, &dtd, sizeof(dtd)); + } + + /* Prepare ranks */ + { U32 n, nextRankStart = 0; + for (n=1; n> 1; + U32 i; + HUF_DEltX2 D; + D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w); + for (i = rankVal[w]; i < rankVal[w] + length; i++) + dt[i] = D; + rankVal[w] += length; + } } + + return iSize; +} + + +static BYTE HUF_decodeSymbolX2(BIT_DStream_t* Dstream, const HUF_DEltX2* dt, const U32 dtLog) +{ + size_t const val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */ + BYTE const c = dt[val].byte; + BIT_skipBits(Dstream, dt[val].nbBits); + return c; +} + +#define HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \ + *ptr++ = HUF_decodeSymbolX2(DStreamPtr, dt, dtLog) + +#define HUF_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \ + if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \ + HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) + +#define HUF_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \ + if (MEM_64bits()) \ + HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) + +static inline size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog) +{ + BYTE* const pStart = p; + + /* up to 4 symbols at a time */ + while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-4)) { + HUF_DECODE_SYMBOLX2_2(p, bitDPtr); + HUF_DECODE_SYMBOLX2_1(p, bitDPtr); + HUF_DECODE_SYMBOLX2_2(p, bitDPtr); + HUF_DECODE_SYMBOLX2_0(p, bitDPtr); + } + + /* closer to the end */ + while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd)) + HUF_DECODE_SYMBOLX2_0(p, bitDPtr); + + /* no more data to retrieve from bitstream, hence no need to reload */ + while (p < pEnd) + HUF_DECODE_SYMBOLX2_0(p, bitDPtr); + + return pEnd-pStart; +} + +static size_t HUF_decompress1X2_usingDTable_internal( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + BYTE* op = (BYTE*)dst; + BYTE* const oend = op + dstSize; + const void* dtPtr = DTable + 1; + const HUF_DEltX2* const dt = (const HUF_DEltX2*)dtPtr; + BIT_DStream_t bitD; + DTableDesc const dtd = HUF_getDTableDesc(DTable); + U32 const dtLog = dtd.tableLog; + + { size_t const errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize); + if (HUF_isError(errorCode)) return errorCode; } + + HUF_decodeStreamX2(op, &bitD, oend, dt, dtLog); + + /* check */ + if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected); + + return dstSize; +} + +size_t HUF_decompress1X2_usingDTable( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + DTableDesc dtd = HUF_getDTableDesc(DTable); + if (dtd.tableType != 0) return ERROR(GENERIC); + return HUF_decompress1X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); +} + +size_t HUF_decompress1X2_DCtx (HUF_DTable* DCtx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + const BYTE* ip = (const BYTE*) cSrc; + + size_t const hSize = HUF_readDTableX2 (DCtx, cSrc, cSrcSize); + if (HUF_isError(hSize)) return hSize; + if (hSize >= cSrcSize) return ERROR(srcSize_wrong); + ip += hSize; cSrcSize -= hSize; + + return HUF_decompress1X2_usingDTable_internal (dst, dstSize, ip, cSrcSize, DCtx); +} + +size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_TABLELOG_MAX); + return HUF_decompress1X2_DCtx (DTable, dst, dstSize, cSrc, cSrcSize); +} + + +static size_t HUF_decompress4X2_usingDTable_internal( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + /* Check */ + if (cSrcSize < 10) return ERROR(corruption_detected); /* strict minimum : jump table + 1 byte per stream */ + + { const BYTE* const istart = (const BYTE*) cSrc; + BYTE* const ostart = (BYTE*) dst; + BYTE* const oend = ostart + dstSize; + const void* const dtPtr = DTable + 1; + const HUF_DEltX2* const dt = (const HUF_DEltX2*)dtPtr; + + /* Init */ + BIT_DStream_t bitD1; + BIT_DStream_t bitD2; + BIT_DStream_t bitD3; + BIT_DStream_t bitD4; + size_t const length1 = MEM_readLE16(istart); + size_t const length2 = MEM_readLE16(istart+2); + size_t const length3 = MEM_readLE16(istart+4); + size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6); + const BYTE* const istart1 = istart + 6; /* jumpTable */ + const BYTE* const istart2 = istart1 + length1; + const BYTE* const istart3 = istart2 + length2; + const BYTE* const istart4 = istart3 + length3; + const size_t segmentSize = (dstSize+3) / 4; + BYTE* const opStart2 = ostart + segmentSize; + BYTE* const opStart3 = opStart2 + segmentSize; + BYTE* const opStart4 = opStart3 + segmentSize; + BYTE* op1 = ostart; + BYTE* op2 = opStart2; + BYTE* op3 = opStart3; + BYTE* op4 = opStart4; + U32 endSignal; + DTableDesc const dtd = HUF_getDTableDesc(DTable); + U32 const dtLog = dtd.tableLog; + + if (length4 > cSrcSize) return ERROR(corruption_detected); /* overflow */ + { size_t const errorCode = BIT_initDStream(&bitD1, istart1, length1); + if (HUF_isError(errorCode)) return errorCode; } + { size_t const errorCode = BIT_initDStream(&bitD2, istart2, length2); + if (HUF_isError(errorCode)) return errorCode; } + { size_t const errorCode = BIT_initDStream(&bitD3, istart3, length3); + if (HUF_isError(errorCode)) return errorCode; } + { size_t const errorCode = BIT_initDStream(&bitD4, istart4, length4); + if (HUF_isError(errorCode)) return errorCode; } + + /* 16-32 symbols per loop (4-8 symbols per stream) */ + endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); + for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; ) { + HUF_DECODE_SYMBOLX2_2(op1, &bitD1); + HUF_DECODE_SYMBOLX2_2(op2, &bitD2); + HUF_DECODE_SYMBOLX2_2(op3, &bitD3); + HUF_DECODE_SYMBOLX2_2(op4, &bitD4); + HUF_DECODE_SYMBOLX2_1(op1, &bitD1); + HUF_DECODE_SYMBOLX2_1(op2, &bitD2); + HUF_DECODE_SYMBOLX2_1(op3, &bitD3); + HUF_DECODE_SYMBOLX2_1(op4, &bitD4); + HUF_DECODE_SYMBOLX2_2(op1, &bitD1); + HUF_DECODE_SYMBOLX2_2(op2, &bitD2); + HUF_DECODE_SYMBOLX2_2(op3, &bitD3); + HUF_DECODE_SYMBOLX2_2(op4, &bitD4); + HUF_DECODE_SYMBOLX2_0(op1, &bitD1); + HUF_DECODE_SYMBOLX2_0(op2, &bitD2); + HUF_DECODE_SYMBOLX2_0(op3, &bitD3); + HUF_DECODE_SYMBOLX2_0(op4, &bitD4); + endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); + } + + /* check corruption */ + if (op1 > opStart2) return ERROR(corruption_detected); + if (op2 > opStart3) return ERROR(corruption_detected); + if (op3 > opStart4) return ERROR(corruption_detected); + /* note : op4 supposed already verified within main loop */ + + /* finish bitStreams one by one */ + HUF_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog); + HUF_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog); + HUF_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog); + HUF_decodeStreamX2(op4, &bitD4, oend, dt, dtLog); + + /* check */ + endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4); + if (!endSignal) return ERROR(corruption_detected); + + /* decoded size */ + return dstSize; + } +} + + +size_t HUF_decompress4X2_usingDTable( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + DTableDesc dtd = HUF_getDTableDesc(DTable); + if (dtd.tableType != 0) return ERROR(GENERIC); + return HUF_decompress4X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); +} + + +size_t HUF_decompress4X2_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + const BYTE* ip = (const BYTE*) cSrc; + + size_t const hSize = HUF_readDTableX2 (dctx, cSrc, cSrcSize); + if (HUF_isError(hSize)) return hSize; + if (hSize >= cSrcSize) return ERROR(srcSize_wrong); + ip += hSize; cSrcSize -= hSize; + + return HUF_decompress4X2_usingDTable_internal (dst, dstSize, ip, cSrcSize, dctx); +} + +size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_TABLELOG_MAX); + return HUF_decompress4X2_DCtx(DTable, dst, dstSize, cSrc, cSrcSize); +} + + +/* *************************/ +/* double-symbols decoding */ +/* *************************/ +typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUF_DEltX4; /* double-symbols decoding */ + +typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t; + +/* HUF_fillDTableX4Level2() : + * `rankValOrigin` must be a table of at least (HUF_TABLELOG_MAX + 1) U32 */ +static void HUF_fillDTableX4Level2(HUF_DEltX4* DTable, U32 sizeLog, const U32 consumed, + const U32* rankValOrigin, const int minWeight, + const sortedSymbol_t* sortedSymbols, const U32 sortedListSize, + U32 nbBitsBaseline, U16 baseSeq) +{ + HUF_DEltX4 DElt; + U32 rankVal[HUF_TABLELOG_MAX + 1]; + + /* get pre-calculated rankVal */ + memcpy(rankVal, rankValOrigin, sizeof(rankVal)); + + /* fill skipped values */ + if (minWeight>1) { + U32 i, skipSize = rankVal[minWeight]; + MEM_writeLE16(&(DElt.sequence), baseSeq); + DElt.nbBits = (BYTE)(consumed); + DElt.length = 1; + for (i = 0; i < skipSize; i++) + DTable[i] = DElt; + } + + /* fill DTable */ + { U32 s; for (s=0; s= 1 */ + + rankVal[weight] += length; + } } +} + +typedef U32 rankVal_t[HUF_TABLELOG_MAX][HUF_TABLELOG_MAX + 1]; + +static void HUF_fillDTableX4(HUF_DEltX4* DTable, const U32 targetLog, + const sortedSymbol_t* sortedList, const U32 sortedListSize, + const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight, + const U32 nbBitsBaseline) +{ + U32 rankVal[HUF_TABLELOG_MAX + 1]; + const int scaleLog = nbBitsBaseline - targetLog; /* note : targetLog >= srcLog, hence scaleLog <= 1 */ + const U32 minBits = nbBitsBaseline - maxWeight; + U32 s; + + memcpy(rankVal, rankValOrigin, sizeof(rankVal)); + + /* fill DTable */ + for (s=0; s= minBits) { /* enough room for a second symbol */ + U32 sortedRank; + int minWeight = nbBits + scaleLog; + if (minWeight < 1) minWeight = 1; + sortedRank = rankStart[minWeight]; + HUF_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits, + rankValOrigin[nbBits], minWeight, + sortedList+sortedRank, sortedListSize-sortedRank, + nbBitsBaseline, symbol); + } else { + HUF_DEltX4 DElt; + MEM_writeLE16(&(DElt.sequence), symbol); + DElt.nbBits = (BYTE)(nbBits); + DElt.length = 1; + { U32 const end = start + length; + U32 u; + for (u = start; u < end; u++) DTable[u] = DElt; + } } + rankVal[weight] += length; + } +} + +size_t HUF_readDTableX4 (HUF_DTable* DTable, const void* src, size_t srcSize) +{ + BYTE weightList[HUF_SYMBOLVALUE_MAX + 1]; + sortedSymbol_t sortedSymbol[HUF_SYMBOLVALUE_MAX + 1]; + U32 rankStats[HUF_TABLELOG_MAX + 1] = { 0 }; + U32 rankStart0[HUF_TABLELOG_MAX + 2] = { 0 }; + U32* const rankStart = rankStart0+1; + rankVal_t rankVal; + U32 tableLog, maxW, sizeOfSort, nbSymbols; + DTableDesc dtd = HUF_getDTableDesc(DTable); + U32 const maxTableLog = dtd.maxTableLog; + size_t iSize; + void* dtPtr = DTable+1; /* force compiler to avoid strict-aliasing */ + HUF_DEltX4* const dt = (HUF_DEltX4*)dtPtr; + + HUF_STATIC_ASSERT(sizeof(HUF_DEltX4) == sizeof(HUF_DTable)); /* if compilation fails here, assertion is false */ + if (maxTableLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge); + /* memset(weightList, 0, sizeof(weightList)); */ /* is not necessary, even though some analyzer complain ... */ + + iSize = HUF_readStats(weightList, HUF_SYMBOLVALUE_MAX + 1, rankStats, &nbSymbols, &tableLog, src, srcSize); + if (HUF_isError(iSize)) return iSize; + + /* check result */ + if (tableLog > maxTableLog) return ERROR(tableLog_tooLarge); /* DTable can't fit code depth */ + + /* find maxWeight */ + for (maxW = tableLog; rankStats[maxW]==0; maxW--) {} /* necessarily finds a solution before 0 */ + + /* Get start index of each weight */ + { U32 w, nextRankStart = 0; + for (w=1; w> consumed; + } } } } + + HUF_fillDTableX4(dt, maxTableLog, + sortedSymbol, sizeOfSort, + rankStart0, rankVal, maxW, + tableLog+1); + + dtd.tableLog = (BYTE)maxTableLog; + dtd.tableType = 1; + memcpy(DTable, &dtd, sizeof(dtd)); + return iSize; +} + + +static U32 HUF_decodeSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog) +{ + size_t const val = BIT_lookBitsFast(DStream, dtLog); /* note : dtLog >= 1 */ + memcpy(op, dt+val, 2); + BIT_skipBits(DStream, dt[val].nbBits); + return dt[val].length; +} + +static U32 HUF_decodeLastSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog) +{ + size_t const val = BIT_lookBitsFast(DStream, dtLog); /* note : dtLog >= 1 */ + memcpy(op, dt+val, 1); + if (dt[val].length==1) BIT_skipBits(DStream, dt[val].nbBits); + else { + if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8)) { + BIT_skipBits(DStream, dt[val].nbBits); + if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8)) + DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8); /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */ + } } + return 1; +} + + +#define HUF_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \ + ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) + +#define HUF_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \ + if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \ + ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) + +#define HUF_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \ + if (MEM_64bits()) \ + ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) + +static inline size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog) +{ + BYTE* const pStart = p; + + /* up to 8 symbols at a time */ + while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd-(sizeof(bitDPtr->bitContainer)-1))) { + HUF_DECODE_SYMBOLX4_2(p, bitDPtr); + HUF_DECODE_SYMBOLX4_1(p, bitDPtr); + HUF_DECODE_SYMBOLX4_2(p, bitDPtr); + HUF_DECODE_SYMBOLX4_0(p, bitDPtr); + } + + /* closer to end : up to 2 symbols at a time */ + while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p <= pEnd-2)) + HUF_DECODE_SYMBOLX4_0(p, bitDPtr); + + while (p <= pEnd-2) + HUF_DECODE_SYMBOLX4_0(p, bitDPtr); /* no need to reload : reached the end of DStream */ + + if (p < pEnd) + p += HUF_decodeLastSymbolX4(p, bitDPtr, dt, dtLog); + + return p-pStart; +} + + +static size_t HUF_decompress1X4_usingDTable_internal( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + BIT_DStream_t bitD; + + /* Init */ + { size_t const errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize); + if (HUF_isError(errorCode)) return errorCode; + } + + /* decode */ + { BYTE* const ostart = (BYTE*) dst; + BYTE* const oend = ostart + dstSize; + const void* const dtPtr = DTable+1; /* force compiler to not use strict-aliasing */ + const HUF_DEltX4* const dt = (const HUF_DEltX4*)dtPtr; + DTableDesc const dtd = HUF_getDTableDesc(DTable); + HUF_decodeStreamX4(ostart, &bitD, oend, dt, dtd.tableLog); + } + + /* check */ + if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected); + + /* decoded size */ + return dstSize; +} + +size_t HUF_decompress1X4_usingDTable( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + DTableDesc dtd = HUF_getDTableDesc(DTable); + if (dtd.tableType != 1) return ERROR(GENERIC); + return HUF_decompress1X4_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); +} + +size_t HUF_decompress1X4_DCtx (HUF_DTable* DCtx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + const BYTE* ip = (const BYTE*) cSrc; + + size_t const hSize = HUF_readDTableX4 (DCtx, cSrc, cSrcSize); + if (HUF_isError(hSize)) return hSize; + if (hSize >= cSrcSize) return ERROR(srcSize_wrong); + ip += hSize; cSrcSize -= hSize; + + return HUF_decompress1X4_usingDTable_internal (dst, dstSize, ip, cSrcSize, DCtx); +} + +size_t HUF_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_TABLELOG_MAX); + return HUF_decompress1X4_DCtx(DTable, dst, dstSize, cSrc, cSrcSize); +} + +static size_t HUF_decompress4X4_usingDTable_internal( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + if (cSrcSize < 10) return ERROR(corruption_detected); /* strict minimum : jump table + 1 byte per stream */ + + { const BYTE* const istart = (const BYTE*) cSrc; + BYTE* const ostart = (BYTE*) dst; + BYTE* const oend = ostart + dstSize; + const void* const dtPtr = DTable+1; + const HUF_DEltX4* const dt = (const HUF_DEltX4*)dtPtr; + + /* Init */ + BIT_DStream_t bitD1; + BIT_DStream_t bitD2; + BIT_DStream_t bitD3; + BIT_DStream_t bitD4; + size_t const length1 = MEM_readLE16(istart); + size_t const length2 = MEM_readLE16(istart+2); + size_t const length3 = MEM_readLE16(istart+4); + size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6); + const BYTE* const istart1 = istart + 6; /* jumpTable */ + const BYTE* const istart2 = istart1 + length1; + const BYTE* const istart3 = istart2 + length2; + const BYTE* const istart4 = istart3 + length3; + size_t const segmentSize = (dstSize+3) / 4; + BYTE* const opStart2 = ostart + segmentSize; + BYTE* const opStart3 = opStart2 + segmentSize; + BYTE* const opStart4 = opStart3 + segmentSize; + BYTE* op1 = ostart; + BYTE* op2 = opStart2; + BYTE* op3 = opStart3; + BYTE* op4 = opStart4; + U32 endSignal; + DTableDesc const dtd = HUF_getDTableDesc(DTable); + U32 const dtLog = dtd.tableLog; + + if (length4 > cSrcSize) return ERROR(corruption_detected); /* overflow */ + { size_t const errorCode = BIT_initDStream(&bitD1, istart1, length1); + if (HUF_isError(errorCode)) return errorCode; } + { size_t const errorCode = BIT_initDStream(&bitD2, istart2, length2); + if (HUF_isError(errorCode)) return errorCode; } + { size_t const errorCode = BIT_initDStream(&bitD3, istart3, length3); + if (HUF_isError(errorCode)) return errorCode; } + { size_t const errorCode = BIT_initDStream(&bitD4, istart4, length4); + if (HUF_isError(errorCode)) return errorCode; } + + /* 16-32 symbols per loop (4-8 symbols per stream) */ + endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); + for ( ; (endSignal==BIT_DStream_unfinished) & (op4<(oend-(sizeof(bitD4.bitContainer)-1))) ; ) { + HUF_DECODE_SYMBOLX4_2(op1, &bitD1); + HUF_DECODE_SYMBOLX4_2(op2, &bitD2); + HUF_DECODE_SYMBOLX4_2(op3, &bitD3); + HUF_DECODE_SYMBOLX4_2(op4, &bitD4); + HUF_DECODE_SYMBOLX4_1(op1, &bitD1); + HUF_DECODE_SYMBOLX4_1(op2, &bitD2); + HUF_DECODE_SYMBOLX4_1(op3, &bitD3); + HUF_DECODE_SYMBOLX4_1(op4, &bitD4); + HUF_DECODE_SYMBOLX4_2(op1, &bitD1); + HUF_DECODE_SYMBOLX4_2(op2, &bitD2); + HUF_DECODE_SYMBOLX4_2(op3, &bitD3); + HUF_DECODE_SYMBOLX4_2(op4, &bitD4); + HUF_DECODE_SYMBOLX4_0(op1, &bitD1); + HUF_DECODE_SYMBOLX4_0(op2, &bitD2); + HUF_DECODE_SYMBOLX4_0(op3, &bitD3); + HUF_DECODE_SYMBOLX4_0(op4, &bitD4); + + endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); + } + + /* check corruption */ + if (op1 > opStart2) return ERROR(corruption_detected); + if (op2 > opStart3) return ERROR(corruption_detected); + if (op3 > opStart4) return ERROR(corruption_detected); + /* note : op4 already verified within main loop */ + + /* finish bitStreams one by one */ + HUF_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog); + HUF_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog); + HUF_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog); + HUF_decodeStreamX4(op4, &bitD4, oend, dt, dtLog); + + /* check */ + { U32 const endCheck = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4); + if (!endCheck) return ERROR(corruption_detected); } + + /* decoded size */ + return dstSize; + } +} + + +size_t HUF_decompress4X4_usingDTable( + void* dst, size_t dstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + DTableDesc dtd = HUF_getDTableDesc(DTable); + if (dtd.tableType != 1) return ERROR(GENERIC); + return HUF_decompress4X4_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); +} + + +size_t HUF_decompress4X4_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + const BYTE* ip = (const BYTE*) cSrc; + + size_t hSize = HUF_readDTableX4 (dctx, cSrc, cSrcSize); + if (HUF_isError(hSize)) return hSize; + if (hSize >= cSrcSize) return ERROR(srcSize_wrong); + ip += hSize; cSrcSize -= hSize; + + return HUF_decompress4X4_usingDTable_internal(dst, dstSize, ip, cSrcSize, dctx); +} + +size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_TABLELOG_MAX); + return HUF_decompress4X4_DCtx(DTable, dst, dstSize, cSrc, cSrcSize); +} + + +/* ********************************/ +/* Generic decompression selector */ +/* ********************************/ + +size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + DTableDesc const dtd = HUF_getDTableDesc(DTable); + return dtd.tableType ? HUF_decompress1X4_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable) : + HUF_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable); +} + +size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize, + const void* cSrc, size_t cSrcSize, + const HUF_DTable* DTable) +{ + DTableDesc const dtd = HUF_getDTableDesc(DTable); + return dtd.tableType ? HUF_decompress4X4_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable) : + HUF_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable); +} + + +typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t; +static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] = +{ + /* single, double, quad */ + {{0,0}, {1,1}, {2,2}}, /* Q==0 : impossible */ + {{0,0}, {1,1}, {2,2}}, /* Q==1 : impossible */ + {{ 38,130}, {1313, 74}, {2151, 38}}, /* Q == 2 : 12-18% */ + {{ 448,128}, {1353, 74}, {2238, 41}}, /* Q == 3 : 18-25% */ + {{ 556,128}, {1353, 74}, {2238, 47}}, /* Q == 4 : 25-32% */ + {{ 714,128}, {1418, 74}, {2436, 53}}, /* Q == 5 : 32-38% */ + {{ 883,128}, {1437, 74}, {2464, 61}}, /* Q == 6 : 38-44% */ + {{ 897,128}, {1515, 75}, {2622, 68}}, /* Q == 7 : 44-50% */ + {{ 926,128}, {1613, 75}, {2730, 75}}, /* Q == 8 : 50-56% */ + {{ 947,128}, {1729, 77}, {3359, 77}}, /* Q == 9 : 56-62% */ + {{1107,128}, {2083, 81}, {4006, 84}}, /* Q ==10 : 62-69% */ + {{1177,128}, {2379, 87}, {4785, 88}}, /* Q ==11 : 69-75% */ + {{1242,128}, {2415, 93}, {5155, 84}}, /* Q ==12 : 75-81% */ + {{1349,128}, {2644,106}, {5260,106}}, /* Q ==13 : 81-87% */ + {{1455,128}, {2422,124}, {4174,124}}, /* Q ==14 : 87-93% */ + {{ 722,128}, {1891,145}, {1936,146}}, /* Q ==15 : 93-99% */ +}; + +/** HUF_selectDecoder() : +* Tells which decoder is likely to decode faster, +* based on a set of pre-determined metrics. +* @return : 0==HUF_decompress4X2, 1==HUF_decompress4X4 . +* Assumption : 0 < cSrcSize < dstSize <= 128 KB */ +U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize) +{ + /* decoder timing evaluation */ + U32 const Q = (U32)(cSrcSize * 16 / dstSize); /* Q < 16 since dstSize > cSrcSize */ + U32 const D256 = (U32)(dstSize >> 8); + U32 const DTime0 = algoTime[Q][0].tableTime + (algoTime[Q][0].decode256Time * D256); + U32 DTime1 = algoTime[Q][1].tableTime + (algoTime[Q][1].decode256Time * D256); + DTime1 += DTime1 >> 3; /* advantage to algorithm using less memory, for cache eviction */ + + return DTime1 < DTime0; +} + + +typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); + +size_t HUF_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + static const decompressionAlgo decompress[2] = { HUF_decompress4X2, HUF_decompress4X4 }; + + /* validation checks */ + if (dstSize == 0) return ERROR(dstSize_tooSmall); + if (cSrcSize > dstSize) return ERROR(corruption_detected); /* invalid */ + if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; } /* not compressed */ + if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; } /* RLE */ + + { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); + return decompress[algoNb](dst, dstSize, cSrc, cSrcSize); + } +} + +size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + /* validation checks */ + if (dstSize == 0) return ERROR(dstSize_tooSmall); + if (cSrcSize > dstSize) return ERROR(corruption_detected); /* invalid */ + if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; } /* not compressed */ + if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; } /* RLE */ + + { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); + return algoNb ? HUF_decompress4X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) : + HUF_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ; + } +} + +size_t HUF_decompress4X_hufOnly (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + /* validation checks */ + if (dstSize == 0) return ERROR(dstSize_tooSmall); + if ((cSrcSize >= dstSize) || (cSrcSize <= 1)) return ERROR(corruption_detected); /* invalid */ + + { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); + return algoNb ? HUF_decompress4X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) : + HUF_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ; + } +} + +size_t HUF_decompress1X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) +{ + /* validation checks */ + if (dstSize == 0) return ERROR(dstSize_tooSmall); + if (cSrcSize > dstSize) return ERROR(corruption_detected); /* invalid */ + if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; } /* not compressed */ + if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; } /* RLE */ + + { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); + return algoNb ? HUF_decompress1X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) : + HUF_decompress1X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ; + } +} diff --git a/contrib/lizard/lib/entropy/mem.h b/contrib/lizard/lib/entropy/mem.h new file mode 100644 index 00000000000..708d897a1cc --- /dev/null +++ b/contrib/lizard/lib/entropy/mem.h @@ -0,0 +1,372 @@ +/** + * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. + * All rights reserved. + * + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + +#ifndef MEM_H_MODULE +#define MEM_H_MODULE + +#if defined (__cplusplus) +extern "C" { +#endif + +/*-**************************************** +* Dependencies +******************************************/ +#include /* size_t, ptrdiff_t */ +#include /* memcpy */ + + +/*-**************************************** +* Compiler specifics +******************************************/ +#if defined(_MSC_VER) /* Visual Studio */ +# include /* _byteswap_ulong */ +# include /* _byteswap_* */ +#endif +#if defined(__GNUC__) +# define MEM_STATIC static __inline __attribute__((unused)) +#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) +# define MEM_STATIC static inline +#elif defined(_MSC_VER) +# define MEM_STATIC static __inline +#else +# define MEM_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */ +#endif + +/* code only tested on 32 and 64 bits systems */ +#define MEM_STATIC_ASSERT(c) { enum { XXH_static_assert = 1/(int)(!!(c)) }; } +MEM_STATIC void MEM_check(void) { MEM_STATIC_ASSERT((sizeof(size_t)==4) || (sizeof(size_t)==8)); } + + +/*-************************************************************** +* Basic Types +*****************************************************************/ +#if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) +# include + typedef uint8_t BYTE; + typedef uint16_t U16; + typedef int16_t S16; + typedef uint32_t U32; + typedef int32_t S32; + typedef uint64_t U64; + typedef int64_t S64; + typedef intptr_t iPtrDiff; +#else + typedef unsigned char BYTE; + typedef unsigned short U16; + typedef signed short S16; + typedef unsigned int U32; + typedef signed int S32; + typedef unsigned long long U64; + typedef signed long long S64; + typedef ptrdiff_t iPtrDiff; +#endif + + +/*-************************************************************** +* Memory I/O +*****************************************************************/ +/* MEM_FORCE_MEMORY_ACCESS : + * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable. + * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal. + * The below switch allow to select different access method for improved performance. + * Method 0 (default) : use `memcpy()`. Safe and portable. + * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable). + * This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`. + * Method 2 : direct access. This method is portable but violate C standard. + * It can generate buggy code on targets depending on alignment. + * In some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6) + * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details. + * Prefer these methods in priority order (0 > 1 > 2) + */ +#ifndef MEM_FORCE_MEMORY_ACCESS /* can be defined externally, on command line for example */ +# if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) ) +# define MEM_FORCE_MEMORY_ACCESS 2 +# elif defined(__INTEL_COMPILER) /*|| defined(_MSC_VER)*/ || \ + (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) )) +# define MEM_FORCE_MEMORY_ACCESS 1 +# endif +#endif + +MEM_STATIC unsigned MEM_32bits(void) { return sizeof(size_t)==4; } +MEM_STATIC unsigned MEM_64bits(void) { return sizeof(size_t)==8; } + +MEM_STATIC unsigned MEM_isLittleEndian(void) +{ + const union { U32 u; BYTE c[4]; } one = { 1 }; /* don't use static : performance detrimental */ + return one.c[0]; +} + +#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2) + +/* violates C standard, by lying on structure alignment. +Only use if no other choice to achieve best performance on target platform */ +MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; } +MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; } +MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; } +MEM_STATIC U64 MEM_readST(const void* memPtr) { return *(const size_t*) memPtr; } + +MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; } +MEM_STATIC void MEM_write32(void* memPtr, U32 value) { *(U32*)memPtr = value; } +MEM_STATIC void MEM_write64(void* memPtr, U64 value) { *(U64*)memPtr = value; } + +#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1) + +/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */ +/* currently only defined for gcc and icc */ +#if defined(_MSC_VER) || (defined(__INTEL_COMPILER) && defined(WIN32)) + __pragma( pack(push, 1) ) + typedef union { U16 u16; U32 u32; U64 u64; size_t st; } unalign; + __pragma( pack(pop) ) +#else + typedef union { U16 u16; U32 u32; U64 u64; size_t st; } __attribute__((packed)) unalign; +#endif + +MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; } +MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; } +MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; } +MEM_STATIC U64 MEM_readST(const void* ptr) { return ((const unalign*)ptr)->st; } + +MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; } +MEM_STATIC void MEM_write32(void* memPtr, U32 value) { ((unalign*)memPtr)->u32 = value; } +MEM_STATIC void MEM_write64(void* memPtr, U64 value) { ((unalign*)memPtr)->u64 = value; } + +#else + +/* default method, safe and standard. + can sometimes prove slower */ + +MEM_STATIC U16 MEM_read16(const void* memPtr) +{ + U16 val; memcpy(&val, memPtr, sizeof(val)); return val; +} + +MEM_STATIC U32 MEM_read32(const void* memPtr) +{ + U32 val; memcpy(&val, memPtr, sizeof(val)); return val; +} + +MEM_STATIC U64 MEM_read64(const void* memPtr) +{ + U64 val; memcpy(&val, memPtr, sizeof(val)); return val; +} + +MEM_STATIC size_t MEM_readST(const void* memPtr) +{ + size_t val; memcpy(&val, memPtr, sizeof(val)); return val; +} + +MEM_STATIC void MEM_write16(void* memPtr, U16 value) +{ + memcpy(memPtr, &value, sizeof(value)); +} + +MEM_STATIC void MEM_write32(void* memPtr, U32 value) +{ + memcpy(memPtr, &value, sizeof(value)); +} + +MEM_STATIC void MEM_write64(void* memPtr, U64 value) +{ + memcpy(memPtr, &value, sizeof(value)); +} + +#endif /* MEM_FORCE_MEMORY_ACCESS */ + +MEM_STATIC U32 MEM_swap32(U32 in) +{ +#if defined(_MSC_VER) /* Visual Studio */ + return _byteswap_ulong(in); +#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403) + return __builtin_bswap32(in); +#else + return ((in << 24) & 0xff000000 ) | + ((in << 8) & 0x00ff0000 ) | + ((in >> 8) & 0x0000ff00 ) | + ((in >> 24) & 0x000000ff ); +#endif +} + +MEM_STATIC U64 MEM_swap64(U64 in) +{ +#if defined(_MSC_VER) /* Visual Studio */ + return _byteswap_uint64(in); +#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403) + return __builtin_bswap64(in); +#else + return ((in << 56) & 0xff00000000000000ULL) | + ((in << 40) & 0x00ff000000000000ULL) | + ((in << 24) & 0x0000ff0000000000ULL) | + ((in << 8) & 0x000000ff00000000ULL) | + ((in >> 8) & 0x00000000ff000000ULL) | + ((in >> 24) & 0x0000000000ff0000ULL) | + ((in >> 40) & 0x000000000000ff00ULL) | + ((in >> 56) & 0x00000000000000ffULL); +#endif +} + +MEM_STATIC size_t MEM_swapST(size_t in) +{ + if (MEM_32bits()) + return (size_t)MEM_swap32((U32)in); + else + return (size_t)MEM_swap64((U64)in); +} + +/*=== Little endian r/w ===*/ + +MEM_STATIC U16 MEM_readLE16(const void* memPtr) +{ + if (MEM_isLittleEndian()) + return MEM_read16(memPtr); + else { + const BYTE* p = (const BYTE*)memPtr; + return (U16)(p[0] + (p[1]<<8)); + } +} + +MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val) +{ + if (MEM_isLittleEndian()) { + MEM_write16(memPtr, val); + } else { + BYTE* p = (BYTE*)memPtr; + p[0] = (BYTE)val; + p[1] = (BYTE)(val>>8); + } +} + +MEM_STATIC U32 MEM_readLE24(const void* memPtr) +{ + return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16); +} + +MEM_STATIC void MEM_writeLE24(void* memPtr, U32 val) +{ + MEM_writeLE16(memPtr, (U16)val); + ((BYTE*)memPtr)[2] = (BYTE)(val>>16); +} + +MEM_STATIC U32 MEM_readLE32(const void* memPtr) +{ + if (MEM_isLittleEndian()) + return MEM_read32(memPtr); + else + return MEM_swap32(MEM_read32(memPtr)); +} + +MEM_STATIC void MEM_writeLE32(void* memPtr, U32 val32) +{ + if (MEM_isLittleEndian()) + MEM_write32(memPtr, val32); + else + MEM_write32(memPtr, MEM_swap32(val32)); +} + +MEM_STATIC U64 MEM_readLE64(const void* memPtr) +{ + if (MEM_isLittleEndian()) + return MEM_read64(memPtr); + else + return MEM_swap64(MEM_read64(memPtr)); +} + +MEM_STATIC void MEM_writeLE64(void* memPtr, U64 val64) +{ + if (MEM_isLittleEndian()) + MEM_write64(memPtr, val64); + else + MEM_write64(memPtr, MEM_swap64(val64)); +} + +MEM_STATIC size_t MEM_readLEST(const void* memPtr) +{ + if (MEM_32bits()) + return (size_t)MEM_readLE32(memPtr); + else + return (size_t)MEM_readLE64(memPtr); +} + +MEM_STATIC void MEM_writeLEST(void* memPtr, size_t val) +{ + if (MEM_32bits()) + MEM_writeLE32(memPtr, (U32)val); + else + MEM_writeLE64(memPtr, (U64)val); +} + +/*=== Big endian r/w ===*/ + +MEM_STATIC U32 MEM_readBE32(const void* memPtr) +{ + if (MEM_isLittleEndian()) + return MEM_swap32(MEM_read32(memPtr)); + else + return MEM_read32(memPtr); +} + +MEM_STATIC void MEM_writeBE32(void* memPtr, U32 val32) +{ + if (MEM_isLittleEndian()) + MEM_write32(memPtr, MEM_swap32(val32)); + else + MEM_write32(memPtr, val32); +} + +MEM_STATIC U64 MEM_readBE64(const void* memPtr) +{ + if (MEM_isLittleEndian()) + return MEM_swap64(MEM_read64(memPtr)); + else + return MEM_read64(memPtr); +} + +MEM_STATIC void MEM_writeBE64(void* memPtr, U64 val64) +{ + if (MEM_isLittleEndian()) + MEM_write64(memPtr, MEM_swap64(val64)); + else + MEM_write64(memPtr, val64); +} + +MEM_STATIC size_t MEM_readBEST(const void* memPtr) +{ + if (MEM_32bits()) + return (size_t)MEM_readBE32(memPtr); + else + return (size_t)MEM_readBE64(memPtr); +} + +MEM_STATIC void MEM_writeBEST(void* memPtr, size_t val) +{ + if (MEM_32bits()) + MEM_writeBE32(memPtr, (U32)val); + else + MEM_writeBE64(memPtr, (U64)val); +} + + +/* function safe only for comparisons */ +MEM_STATIC U32 MEM_readMINMATCH(const void* memPtr, U32 length) +{ + switch (length) + { + default : + case 4 : return MEM_read32(memPtr); + case 3 : if (MEM_isLittleEndian()) + return MEM_read32(memPtr)<<8; + else + return MEM_read32(memPtr)>>8; + } +} + +#if defined (__cplusplus) +} +#endif + +#endif /* MEM_H_MODULE */ diff --git a/contrib/lizard/lib/liblizard.pc.in b/contrib/lizard/lib/liblizard.pc.in new file mode 100644 index 00000000000..233c2e08faf --- /dev/null +++ b/contrib/lizard/lib/liblizard.pc.in @@ -0,0 +1,15 @@ +# Lizard - Fast LZ compression algorithm +# Copyright (C) 2011-2014, Yann Collet +# Copyright (C) 2016-2017, Przemyslaw Skibinski +# BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + +prefix=@PREFIX@ +libdir=@LIBDIR@ +includedir=@INCLUDEDIR@ + +Name: lizard +Description: extremely fast lossless compression algorithm library +URL: http://github.com/inikep/lizard +Version: @VERSION@ +Libs: -L@LIBDIR@ -llizard +Cflags: -I@INCLUDEDIR@ diff --git a/contrib/lizard/lib/lizard_common.h b/contrib/lizard/lib/lizard_common.h new file mode 100644 index 00000000000..45730cf4e86 --- /dev/null +++ b/contrib/lizard/lib/lizard_common.h @@ -0,0 +1,504 @@ +/* + Lizard - Fast LZ compression algorithm + Copyright (C) 2011-2015, Yann Collet + Copyright (C) 2016-2017, Przemyslaw Skibinski + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +#ifndef LIZARD_COMMON_H_2983 +#define LIZARD_COMMON_H_2983 + +#if defined (__cplusplus) +extern "C" { +#endif + + +/*-************************************ +* Memory routines +**************************************/ +#include /* malloc, calloc, free */ +#include /* memset, memcpy */ +#include /* intptr_t */ +#include "entropy/mem.h" +#include "lizard_compress.h" /* LIZARD_GCC_VERSION */ + +//#define LIZARD_USE_LOGS +#define LIZARD_LOG_COMPRESS(...) //printf(__VA_ARGS__) +#define LIZARD_LOG_DECOMPRESS(...) //printf(__VA_ARGS__) + +#define LIZARD_LOG_COMPRESS_LZ4(...) //printf(__VA_ARGS__) +#define COMPLOG_CODEWORDS_LZ4(...) //printf(__VA_ARGS__) +#define LIZARD_LOG_DECOMPRESS_LZ4(...) //printf(__VA_ARGS__) +#define DECOMPLOG_CODEWORDS_LZ4(...) //printf(__VA_ARGS__) + +#define LIZARD_LOG_COMPRESS_LIZv1(...) //printf(__VA_ARGS__) +#define COMPLOG_CODEWORDS_LIZv1(...) //printf(__VA_ARGS__) +#define LIZARD_LOG_DECOMPRESS_LIZv1(...) //printf(__VA_ARGS__) +#define DECOMPLOG_CODEWORDS_LIZv1(...) //printf(__VA_ARGS__) + + + + +/*-************************************ +* Common Constants +**************************************/ +#define MINMATCH 4 +//#define USE_LZ4_ONLY +//#define LIZARD_USE_TEST + +#define LIZARD_DICT_SIZE (1<<24) +#define WILDCOPYLENGTH 16 +#define LASTLITERALS WILDCOPYLENGTH +#define MFLIMIT (WILDCOPYLENGTH+MINMATCH) + +#define LIZARD_MAX_PRICE (1<<28) +#define LIZARD_INIT_LAST_OFFSET 0 +#define LIZARD_MAX_16BIT_OFFSET (1<<16) +#define MM_LONGOFF 16 +#define LIZARD_BLOCK_SIZE_PAD (LIZARD_BLOCK_SIZE+32) +#define LIZARD_COMPRESS_ADD_BUF (5*LIZARD_BLOCK_SIZE_PAD) +#ifndef LIZARD_NO_HUFFMAN + #define LIZARD_COMPRESS_ADD_HUF HUF_compressBound(LIZARD_BLOCK_SIZE_PAD) + #define LIZARD_HUF_BLOCK_SIZE LIZARD_BLOCK_SIZE +#else + #define LIZARD_COMPRESS_ADD_HUF 0 + #define LIZARD_HUF_BLOCK_SIZE 1 +#endif + +/* LZ4 codewords */ +#define ML_BITS_LZ4 4 +#define ML_MASK_LZ4 ((1U<= 1<<16 */ + U32 sufficientLength; /* used only by optimal parser: size of matches which is acceptable: larger == more compression, slower */ + U32 fullSearch; /* used only by optimal parser: perform full search of matches: 1 == more compression, slower */ + Lizard_parser_type parserType; + Lizard_decompress_type decompressType; +} Lizard_parameters; + + +struct Lizard_stream_s +{ + const BYTE* end; /* next block here to continue on current prefix */ + const BYTE* base; /* All index relative to this position */ + const BYTE* dictBase; /* alternate base for extDict */ + U32 dictLimit; /* below that point, need extDict */ + U32 lowLimit; /* below that point, no more dict */ + U32 nextToUpdate; /* index from which to continue dictionary update */ + U32 allocatedMemory; + int compressionLevel; + Lizard_parameters params; + U32 hashTableSize; + U32 chainTableSize; + U32* chainTable; + U32* hashTable; + int last_off; + const BYTE* off24pos; + U32 huffType; + U32 comprStreamLen; + + BYTE* huffBase; + BYTE* huffEnd; + BYTE* offset16Base; + BYTE* offset24Base; + BYTE* lenBase; + BYTE* literalsBase; + BYTE* flagsBase; + BYTE* offset16Ptr; + BYTE* offset24Ptr; + BYTE* lenPtr; + BYTE* literalsPtr; + BYTE* flagsPtr; + BYTE* offset16End; + BYTE* offset24End; + BYTE* lenEnd; + BYTE* literalsEnd; + BYTE* flagsEnd; + U32 flagFreq[256]; + U32 litFreq[256]; + U32 litSum, flagSum; + U32 litPriceSum, log2LitSum, log2FlagSum; + U32 cachedPrice; + U32 cachedLitLength; + const BYTE* cachedLiterals; + const BYTE* diffBase; + const BYTE* srcBase; + const BYTE* destBase; +}; + +struct Lizard_streamDecode_s { + const BYTE* externalDict; + size_t extDictSize; + const BYTE* prefixEnd; + size_t prefixSize; +}; + +struct Lizard_dstream_s +{ + const BYTE* offset16Ptr; + const BYTE* offset24Ptr; + const BYTE* lenPtr; + const BYTE* literalsPtr; + const BYTE* flagsPtr; + const BYTE* offset16End; + const BYTE* offset24End; + const BYTE* lenEnd; + const BYTE* literalsEnd; + const BYTE* flagsEnd; + const BYTE* diffBase; + intptr_t last_off; +}; + +typedef struct Lizard_dstream_s Lizard_dstream_t; + +/* ************************************* +* HC Pre-defined compression levels +***************************************/ +#define LIZARD_WINDOWLOG_LZ4 16 +#define LIZARD_CHAINLOG_LZ4 LIZARD_WINDOWLOG_LZ4 +#define LIZARD_HASHLOG_LZ4 18 +#define LIZARD_HASHLOG_LZ4SM 12 + +#define LIZARD_WINDOWLOG_LIZv1 22 +#define LIZARD_CHAINLOG_LIZv1 LIZARD_WINDOWLOG_LIZv1 +#define LIZARD_HASHLOG_LIZv1 18 + + + +static const Lizard_parameters Lizard_defaultParameters[LIZARD_MAX_CLEVEL+1-LIZARD_MIN_CLEVEL] = +{ + /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ + { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4SM, 0, 0, 0, 0, 0, 0, Lizard_parser_fastSmall, Lizard_coderwords_LZ4 }, // level 10 + { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_fast, Lizard_coderwords_LZ4 }, // level 11 + { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_noChain, Lizard_coderwords_LZ4 }, // level 12 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 2, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 13 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 4, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 14 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 8, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 15 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 16, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 16 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 256, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 17 + { LIZARD_WINDOWLOG_LZ4, LIZARD_WINDOWLOG_LZ4+1, LIZARD_HASHLOG_LZ4, 16, 16, 4, 0, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LZ4 }, // level 18 + { LIZARD_WINDOWLOG_LZ4, LIZARD_WINDOWLOG_LZ4+1, 23, 16, 256, 4, 0, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LZ4 }, // level 19 + /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ + { LIZARD_WINDOWLOG_LIZv1, 0, 14, 0, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_fastBig, Lizard_coderwords_LIZv1 }, // level 20 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 14, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 21 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 22 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 23 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 2, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 24 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 8, 4, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 25 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 8, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 26 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 128, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 27 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 24, 1<<10, 4, MM_LONGOFF, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 28 + { 24, 25, 23, 24, 1<<10, 4, MM_LONGOFF, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 29 +#ifndef LIZARD_NO_HUFFMAN + /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ + { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4SM, 0, 0, 0, 0, 0, 0, Lizard_parser_fastSmall, Lizard_coderwords_LZ4 }, // level 30 + { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_fast, Lizard_coderwords_LZ4 }, // level 31 + { LIZARD_WINDOWLOG_LZ4, 0, 14, 0, 0, 0, 0, 0, 0, Lizard_parser_noChain, Lizard_coderwords_LZ4 }, // level 32 + { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_noChain, Lizard_coderwords_LZ4 }, // level 33 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 2, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 34 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 4, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 35 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 8, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 36 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 16, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 37 + { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 256, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 38 + { LIZARD_WINDOWLOG_LZ4, LIZARD_WINDOWLOG_LZ4+1, 23, 16, 256, 4, 0, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LZ4 }, // level 39 + /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ + { LIZARD_WINDOWLOG_LIZv1, 0, 14, 0, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_fastBig, Lizard_coderwords_LIZv1 }, // level 40 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 14, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 41 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 42 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 43 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 2, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 44 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 8, 4, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 45 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 8, 4, MM_LONGOFF, 64, 0, Lizard_parser_optimalPrice, Lizard_coderwords_LIZv1 }, // level 46 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 8, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 47 + { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 128, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 48 + { 24, 25, 23, 24, 1<<10, 4, MM_LONGOFF, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 49 +#endif +// { 10, 10, 10, 0, 0, 4, 0, 0, 0, Lizard_fast }, // min values +// { 24, 24, 28, 24, 1<<24, 7, 0, 1<<24, 2, Lizard_optimal_price }, // max values +}; + + + +/*-************************************ +* Compiler Options +**************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define FORCE_INLINE static __forceinline +# include +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4293) /* disable: C4293: too large shift (32-bits) */ +#else +# if defined(__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */ +# if defined(__GNUC__) || defined(__clang__) +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +# else +# define FORCE_INLINE static +# endif /* __STDC_VERSION__ */ +#endif /* _MSC_VER */ + +#define LIZARD_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) +#if (LIZARD_GCC_VERSION >= 302) || (__INTEL_COMPILER >= 800) || defined(__clang__) +# define expect(expr,value) (__builtin_expect ((expr),(value)) ) +#else +# define expect(expr,value) (expr) +#endif + +#define likely(expr) expect((expr) != 0, 1) +#define unlikely(expr) expect((expr) != 0, 0) + +#define KB *(1 <<10) +#define MB *(1 <<20) +#define GB *(1U<<30) + +#define ALLOCATOR(n,s) calloc(n,s) +#define FREEMEM free +#define MEM_INIT memset +#ifndef MAX + #define MAX(a,b) ((a)>(b))?(a):(b) +#endif +#ifndef MIN + #define MIN(a,b) ((a)<(b)?(a):(b)) +#endif + +#if MINMATCH == 3 + #define MEM_readMINMATCH(ptr) (U32)(MEM_read32(ptr)<<8) +#else + #define MEM_readMINMATCH(ptr) (U32)(MEM_read32(ptr)) +#endif + + + + +/*-************************************ +* Reading and writing into memory +**************************************/ +#define STEPSIZE sizeof(size_t) + + +MEM_STATIC void Lizard_copy8(void* dst, const void* src) +{ + memcpy(dst,src,8); +} + +/* customized variant of memcpy, which can overwrite up to 7 bytes beyond dstEnd */ +MEM_STATIC void Lizard_wildCopy(void* dstPtr, const void* srcPtr, void* dstEnd) +{ + BYTE* d = (BYTE*)dstPtr; + const BYTE* s = (const BYTE*)srcPtr; + BYTE* const e = (BYTE*)dstEnd; + +#if 0 + const size_t l2 = 8 - (((size_t)d) & (sizeof(void*)-1)); + Lizard_copy8(d,s); if (d>e-9) return; + d+=l2; s+=l2; +#endif /* join to align */ + + do { Lizard_copy8(d,s); d+=8; s+=8; } while (d= 3) /* GCC Intrinsic */ + return 31 - __builtin_clz(val); +# else /* Software version */ + static const int DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 }; + U32 v = val; + int r; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + r = DeBruijnClz[(U32)(v * 0x07C4ACDDU) >> 27]; + return r; +# endif +} + + +/*-************************************ +* Common functions +**************************************/ +MEM_STATIC unsigned Lizard_NbCommonBytes (register size_t val) +{ + if (MEM_isLittleEndian()) { + if (MEM_64bits()) { +# if defined(_MSC_VER) && defined(_WIN64) && !defined(LIZARD_FORCE_SW_BITCOUNT) + unsigned long r = 0; + _BitScanForward64( &r, (U64)val ); + return (int)(r>>3); +# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) + return (__builtin_ctzll((U64)val) >> 3); +# else + static const int DeBruijnBytePos[64] = { 0, 0, 0, 0, 0, 1, 1, 2, 0, 3, 1, 3, 1, 4, 2, 7, 0, 2, 3, 6, 1, 5, 3, 5, 1, 3, 4, 4, 2, 5, 6, 7, 7, 0, 1, 2, 3, 3, 4, 6, 2, 6, 5, 5, 3, 4, 5, 6, 7, 1, 2, 4, 6, 4, 4, 5, 7, 2, 6, 5, 7, 6, 7, 7 }; + return DeBruijnBytePos[((U64)((val & -(long long)val) * 0x0218A392CDABBD3FULL)) >> 58]; +# endif + } else /* 32 bits */ { +# if defined(_MSC_VER) && !defined(LIZARD_FORCE_SW_BITCOUNT) + unsigned long r; + _BitScanForward( &r, (U32)val ); + return (int)(r>>3); +# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) + return (__builtin_ctz((U32)val) >> 3); +# else + static const int DeBruijnBytePos[32] = { 0, 0, 3, 0, 3, 1, 3, 0, 3, 2, 2, 1, 3, 2, 0, 1, 3, 3, 1, 2, 2, 2, 2, 0, 3, 1, 2, 0, 1, 0, 1, 1 }; + return DeBruijnBytePos[((U32)((val & -(S32)val) * 0x077CB531U)) >> 27]; +# endif + } + } else /* Big Endian CPU */ { + if (MEM_64bits()) { +# if defined(_MSC_VER) && defined(_WIN64) && !defined(LIZARD_FORCE_SW_BITCOUNT) + unsigned long r = 0; + _BitScanReverse64( &r, val ); + return (unsigned)(r>>3); +# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) + return (__builtin_clzll((U64)val) >> 3); +# else + unsigned r; + if (!(val>>32)) { r=4; } else { r=0; val>>=32; } + if (!(val>>16)) { r+=2; val>>=8; } else { val>>=24; } + r += (!val); + return r; +# endif + } else /* 32 bits */ { +# if defined(_MSC_VER) && !defined(LIZARD_FORCE_SW_BITCOUNT) + unsigned long r = 0; + _BitScanReverse( &r, (unsigned long)val ); + return (unsigned)(r>>3); +# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) + return (__builtin_clz((U32)val) >> 3); +# else + unsigned r; + if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; } + r += (!val); + return r; +# endif + } + } +} + +MEM_STATIC unsigned Lizard_count(const BYTE* pIn, const BYTE* pMatch, const BYTE* pInLimit) +{ + const BYTE* const pStart = pIn; + + while (likely(pIn + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOTLizard_hash4Ptr + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + + +/* ************************************* +* Includes +***************************************/ +#include "lizard_compress.h" +#include "lizard_common.h" +#include +#include // intptr_t +#ifndef USE_LZ4_ONLY + #ifdef LIZARD_USE_TEST + #include "test/lizard_common_test.h" + #include "test/lizard_compress_test.h" + #else + #include "lizard_compress_liz.h" + #endif +#endif +#include "lizard_compress_lz4.h" +#include "entropy/huf.h" + + +/* ************************************* +* Local Macros +***************************************/ +#define DELTANEXT(p) chainTable[(p) & contentMask] +#define LIZARD_MINIMAL_HUFF_GAIN(comprSize) (comprSize + (comprSize/8) + 512) +#define LIZARD_MINIMAL_BLOCK_GAIN(comprSize) (comprSize + (comprSize/32) + 512) + + +/*-************************************ +* Local Utils +**************************************/ +int Lizard_versionNumber (void) { return LIZARD_VERSION_NUMBER; } +int Lizard_compressBound(int isize) { return LIZARD_COMPRESSBOUND(isize); } +int Lizard_sizeofState_MinLevel() { return Lizard_sizeofState(LIZARD_MIN_CLEVEL); } + + + +/* ************************************* +* Hash functions +***************************************/ +#define HASH_UPDATE_LIMIT 8 /* equal to MEM_read64 */ +static const U32 prime4bytes = 2654435761U; +static const U64 prime5bytes = 889523592379ULL; +static const U64 prime6bytes = 227718039650203ULL; +static const U64 prime7bytes = 58295818150454627ULL; + +#if MINMATCH == 3 +static const U32 prime3bytes = 506832829U; +static U32 Lizard_hash3(U32 u, U32 h) { return (u * prime3bytes) << (32-24) >> (32-h) ; } +static size_t Lizard_hash3Ptr(const void* ptr, U32 h) { return Lizard_hash3(MEM_read32(ptr), h); } +#endif + +static U32 Lizard_hash4(U32 u, U32 h) { return (u * prime4bytes) >> (32-h) ; } +static size_t Lizard_hash4Ptr(const void* ptr, U32 h) { return Lizard_hash4(MEM_read32(ptr), h); } + +static size_t Lizard_hash5(U64 u, U32 h) { return (size_t)((u * prime5bytes) << (64-40) >> (64-h)) ; } +static size_t Lizard_hash5Ptr(const void* p, U32 h) { return Lizard_hash5(MEM_read64(p), h); } + +static size_t Lizard_hash6(U64 u, U32 h) { return (size_t)((u * prime6bytes) << (64-48) >> (64-h)) ; } +static size_t Lizard_hash6Ptr(const void* p, U32 h) { return Lizard_hash6(MEM_read64(p), h); } + +static size_t Lizard_hash7(U64 u, U32 h) { return (size_t)((u * prime7bytes) << (64-56) >> (64-h)) ; } +static size_t Lizard_hash7Ptr(const void* p, U32 h) { return Lizard_hash7(MEM_read64(p), h); } + +static size_t Lizard_hashPtr(const void* p, U32 hBits, U32 mls) +{ + switch(mls) + { + default: + case 4: return Lizard_hash4Ptr(p, hBits); + case 5: return Lizard_hash5Ptr(p, hBits); + case 6: return Lizard_hash6Ptr(p, hBits); + case 7: return Lizard_hash7Ptr(p, hBits); + } +} + + + + +/************************************** +* Internal functions +**************************************/ +/** Lizard_count_2segments() : +* can count match length with `ip` & `match` in 2 different segments. +* convention : on reaching mEnd, match count continue starting from iStart +*/ +static size_t Lizard_count_2segments(const BYTE* ip, const BYTE* match, const BYTE* iEnd, const BYTE* mEnd, const BYTE* iStart) +{ + const BYTE* const vEnd = MIN( ip + (mEnd - match), iEnd); + size_t const matchLength = Lizard_count(ip, match, vEnd); + if (match + matchLength != mEnd) return matchLength; + return matchLength + Lizard_count(ip+matchLength, iStart, iEnd); +} + + +void Lizard_initBlock(Lizard_stream_t* ctx) +{ + ctx->offset16Ptr = ctx->offset16Base; + ctx->offset24Ptr = ctx->offset24Base; + ctx->lenPtr = ctx->lenBase; + ctx->literalsPtr = ctx->literalsBase; + ctx->flagsPtr = ctx->flagsBase; + ctx->last_off = LIZARD_INIT_LAST_OFFSET; /* reset last offset */ +} + + +FORCE_INLINE int Lizard_writeStream(int useHuff, Lizard_stream_t* ctx, BYTE* streamPtr, uint32_t streamLen, BYTE** op, BYTE* oend) +{ + if (useHuff && streamLen > 1024) { +#ifndef LIZARD_NO_HUFFMAN + int useHuffBuf; + if (*op + 6 > oend) { LIZARD_LOG_COMPRESS("*op[%p] + 6 > oend[%p]\n", *op, oend); return -1; } + + useHuffBuf = ((size_t)(oend - (*op + 6)) < HUF_compressBound(streamLen)); + if (useHuffBuf) { + if (streamLen > LIZARD_BLOCK_SIZE) { LIZARD_LOG_COMPRESS("streamLen[%d] > LIZARD_BLOCK_SIZE\n", streamLen); return -1; } + ctx->comprStreamLen = (U32)HUF_compress(ctx->huffBase, ctx->huffEnd - ctx->huffBase, streamPtr, streamLen); + } else { + ctx->comprStreamLen = (U32)HUF_compress(*op + 6, oend - (*op + 6), streamPtr, streamLen); + } + + if (!HUF_isError(ctx->comprStreamLen)) { + if (ctx->comprStreamLen > 0 && (LIZARD_MINIMAL_HUFF_GAIN(ctx->comprStreamLen) < streamLen)) { /* compressible */ + MEM_writeLE24(*op, streamLen); + MEM_writeLE24(*op+3, ctx->comprStreamLen); + if (useHuffBuf) { + if ((size_t)(oend - (*op + 6)) < ctx->comprStreamLen) { LIZARD_LOG_COMPRESS("*op[%p] oend[%p] comprStreamLen[%d]\n", *op, oend, (int)ctx->comprStreamLen); return -1; } + memcpy(*op + 6, ctx->huffBase, ctx->comprStreamLen); + } + *op += ctx->comprStreamLen + 6; + LIZARD_LOG_COMPRESS("HUF_compress streamLen=%d comprStreamLen=%d\n", (int)streamLen, (int)ctx->comprStreamLen); + return 1; + } else { LIZARD_LOG_COMPRESS("HUF_compress ERROR comprStreamLen=%d streamLen=%d\n", (int)ctx->comprStreamLen, (int)streamLen); } + } else { LIZARD_LOG_COMPRESS("HUF_compress ERROR %d: %s\n", (int)ctx->comprStreamLen, HUF_getErrorName(ctx->comprStreamLen)); } +#else + LIZARD_LOG_COMPRESS("compiled with LIZARD_NO_HUFFMAN\n"); + (void)ctx; + return -1; +#endif + } else ctx->comprStreamLen = 0; + + if (*op + 3 + streamLen > oend) { LIZARD_LOG_COMPRESS("*op[%p] + 3 + streamLen[%d] > oend[%p]\n", *op, streamLen, oend); return -1; } + MEM_writeLE24(*op, streamLen); + *op += 3; + memcpy(*op, streamPtr, streamLen); + *op += streamLen; + LIZARD_LOG_COMPRESS("Uncompressed streamLen=%d\n", (int)streamLen); + return 0; +} + + +int Lizard_writeBlock(Lizard_stream_t* ctx, const BYTE* ip, uint32_t inputSize, BYTE** op, BYTE* oend) +{ + int res; + uint32_t flagsLen = (uint32_t)(ctx->flagsPtr - ctx->flagsBase); + uint32_t literalsLen = (uint32_t)(ctx->literalsPtr - ctx->literalsBase); + uint32_t lenLen = (uint32_t)(ctx->lenPtr - ctx->lenBase); + uint32_t offset16Len = (uint32_t)(ctx->offset16Ptr - ctx->offset16Base); + uint32_t offset24Len = (uint32_t)(ctx->offset24Ptr - ctx->offset24Base); + uint32_t sum = flagsLen + literalsLen + lenLen + offset16Len + offset24Len; +#ifdef LIZARD_USE_LOGS + uint32_t comprFlagsLen, comprLiteralsLen; +#endif + + BYTE* start = *op; + + if ((literalsLen < WILDCOPYLENGTH) || (sum+5*3+1 > inputSize)) goto _write_uncompressed; + + *start = 0; + *op += 1; + + res = Lizard_writeStream(0, ctx, ctx->lenBase, lenLen, op, oend); + if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_LEN); + + res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_OFFSET16, ctx, ctx->offset16Base, offset16Len, op, oend); + if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_OFFSET16); + + res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_OFFSET24, ctx, ctx->offset24Base, offset24Len, op, oend); + if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_OFFSET24); + + res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_FLAGS, ctx, ctx->flagsBase, flagsLen, op, oend); + if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_FLAGS); +#ifdef LIZARD_USE_LOGS + comprFlagsLen = ctx->comprStreamLen; +#endif + + res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_LITERALS, ctx, ctx->literalsBase, literalsLen, op, oend); + if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_LITERALS); +#ifdef LIZARD_USE_LOGS + comprLiteralsLen = ctx->comprStreamLen; + sum = (int)(*op-start); +#endif + + if (LIZARD_MINIMAL_BLOCK_GAIN((uint32_t)(*op-start)) > inputSize) goto _write_uncompressed; + + LIZARD_LOG_COMPRESS("%d: total=%d block=%d flagsLen[%.2f%%]=%d comprFlagsLen[%.2f%%]=%d literalsLen[%.2f%%]=%d comprLiteralsLen[%.2f%%]=%d lenLen=%d offset16Len[%.2f%%]=%d offset24Len[%.2f%%]=%d\n", (int)(ip - ctx->srcBase), + (int)(*op - ctx->destBase), sum, (flagsLen*100.0)/sum, flagsLen, (comprFlagsLen*100.0)/sum, comprFlagsLen, (literalsLen*100.0)/sum, literalsLen, (comprLiteralsLen*100.0)/sum, comprLiteralsLen, + lenLen, (offset16Len*100.0)/sum, offset16Len, (offset24Len*100.0)/sum, offset24Len); + return 0; + +_write_uncompressed: + LIZARD_LOG_COMPRESS("%d: total=%d block=%d UNCOMPRESSED inputSize=%u outSize=%d\n", (int)(ip - ctx->srcBase), + (int)(*op - ctx->destBase), (int)(*op-start), inputSize, (int)(oend-start)); + if ((uint32_t)(oend - start) < inputSize + 4) goto _output_error; + *start = LIZARD_FLAG_UNCOMPRESSED; + *op = start + 1; + MEM_writeLE24(*op, inputSize); + *op += 3; + memcpy(*op, ip, inputSize); + *op += inputSize; + return 0; + +_output_error: + LIZARD_LOG_COMPRESS("Lizard_writeBlock ERROR size=%d/%d flagsLen=%d literalsLen=%d lenLen=%d offset16Len=%d offset24Len=%d\n", (int)(*op-start), (int)(oend-start), flagsLen, literalsLen, lenLen, offset16Len, offset24Len); + return 1; +} + + +FORCE_INLINE int Lizard_encodeSequence ( + Lizard_stream_t* ctx, + const BYTE** ip, + const BYTE** anchor, + size_t matchLength, + const BYTE* const match) +{ +#ifdef USE_LZ4_ONLY + return Lizard_encodeSequence_LZ4(ctx, ip, anchor, matchLength, match); +#else + if (ctx->params.decompressType == Lizard_coderwords_LZ4) + return Lizard_encodeSequence_LZ4(ctx, ip, anchor, matchLength, match); + + return Lizard_encodeSequence_LIZv1(ctx, ip, anchor, matchLength, match); +#endif +} + + +FORCE_INLINE int Lizard_encodeLastLiterals ( + Lizard_stream_t* ctx, + const BYTE** ip, + const BYTE** anchor) +{ + LIZARD_LOG_COMPRESS("Lizard_encodeLastLiterals Lizard_coderwords_LZ4=%d\n", ctx->params.decompressType == Lizard_coderwords_LZ4); +#ifdef USE_LZ4_ONLY + return Lizard_encodeLastLiterals_LZ4(ctx, ip, anchor); +#else + if (ctx->params.decompressType == Lizard_coderwords_LZ4) + return Lizard_encodeLastLiterals_LZ4(ctx, ip, anchor); + + return Lizard_encodeLastLiterals_LIZv1(ctx, ip, anchor); +#endif +} + + +/************************************** +* Include parsers +**************************************/ +#include "lizard_parser_hashchain.h" +#include "lizard_parser_nochain.h" +#include "lizard_parser_fast.h" +#include "lizard_parser_fastsmall.h" +#include "lizard_parser_fastbig.h" +#ifndef USE_LZ4_ONLY + #include "lizard_parser_optimal.h" + #include "lizard_parser_lowestprice.h" + #include "lizard_parser_pricefast.h" +#endif + + +int Lizard_verifyCompressionLevel(int compressionLevel) +{ + if (compressionLevel > LIZARD_MAX_CLEVEL) compressionLevel = LIZARD_MAX_CLEVEL; + if (compressionLevel < LIZARD_MIN_CLEVEL) compressionLevel = LIZARD_DEFAULT_CLEVEL; + return compressionLevel; +} + + +int Lizard_sizeofState(int compressionLevel) +{ + Lizard_parameters params; + U32 hashTableSize, chainTableSize; + + compressionLevel = Lizard_verifyCompressionLevel(compressionLevel); + params = Lizard_defaultParameters[compressionLevel - LIZARD_MIN_CLEVEL]; +// hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog3)+((size_t)1 << params.hashLog))); + hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog))); + chainTableSize = (U32)(sizeof(U32)*((size_t)1 << params.contentLog)); + + return sizeof(Lizard_stream_t) + hashTableSize + chainTableSize + LIZARD_COMPRESS_ADD_BUF + (int)LIZARD_COMPRESS_ADD_HUF; +} + + +static void Lizard_init(Lizard_stream_t* ctx, const BYTE* start) +{ + // No need to use memset() on tables as values are always bound checked +#ifdef LIZARD_RESET_MEM + MEM_INIT((void*)ctx->hashTable, 0, ctx->hashTableSize); + MEM_INIT(ctx->chainTable, 0x01, ctx->chainTableSize); +#endif + // printf("memset hashTable=%p hashEnd=%p chainTable=%p chainEnd=%p\n", ctx->hashTable, ((BYTE*)ctx->hashTable) + ctx->hashTableSize, ctx->chainTable, ((BYTE*)ctx->chainTable)+ctx->chainTableSize); + ctx->nextToUpdate = LIZARD_DICT_SIZE; + ctx->base = start - LIZARD_DICT_SIZE; + ctx->end = start; + ctx->dictBase = start - LIZARD_DICT_SIZE; + ctx->dictLimit = LIZARD_DICT_SIZE; + ctx->lowLimit = LIZARD_DICT_SIZE; + ctx->last_off = LIZARD_INIT_LAST_OFFSET; + ctx->litSum = 0; +} + + +/* if ctx==NULL memory is allocated and returned as value */ +Lizard_stream_t* Lizard_initStream(Lizard_stream_t* ctx, int compressionLevel) +{ + Lizard_parameters params; + U32 hashTableSize, chainTableSize; + void *tempPtr; + + compressionLevel = Lizard_verifyCompressionLevel(compressionLevel); + params = Lizard_defaultParameters[compressionLevel - LIZARD_MIN_CLEVEL]; +// hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog3)+((size_t)1 << params.hashLog))); + hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog))); + chainTableSize = (U32)(sizeof(U32)*((size_t)1 << params.contentLog)); + + if (!ctx) + { + ctx = (Lizard_stream_t*)malloc(sizeof(Lizard_stream_t) + hashTableSize + chainTableSize + LIZARD_COMPRESS_ADD_BUF + LIZARD_COMPRESS_ADD_HUF); + if (!ctx) { printf("ERROR: Cannot allocate %d MB (compressionLevel=%d)\n", (int)(sizeof(Lizard_stream_t) + hashTableSize + chainTableSize)>>20, compressionLevel); return 0; } + LIZARD_LOG_COMPRESS("Allocated %d MB (compressionLevel=%d)\n", (int)(sizeof(Lizard_stream_t) + hashTableSize + chainTableSize)>>20, compressionLevel); + ctx->allocatedMemory = sizeof(Lizard_stream_t) + hashTableSize + chainTableSize + LIZARD_COMPRESS_ADD_BUF + (U32)LIZARD_COMPRESS_ADD_HUF; + // printf("malloc from=%p to=%p hashTable=%p hashEnd=%p chainTable=%p chainEnd=%p\n", ctx, ((BYTE*)ctx)+sizeof(Lizard_stream_t) + hashTableSize + chainTableSize, ctx->hashTable, ((BYTE*)ctx->hashTable) + hashTableSize, ctx->chainTable, ((BYTE*)ctx->chainTable)+chainTableSize); + } + + tempPtr = ctx; + ctx->hashTable = (U32*)(tempPtr) + sizeof(Lizard_stream_t)/4; + ctx->hashTableSize = hashTableSize; + ctx->chainTable = ctx->hashTable + hashTableSize/4; + ctx->chainTableSize = chainTableSize; + ctx->params = params; + ctx->compressionLevel = (unsigned)compressionLevel; + if (compressionLevel < 30) + ctx->huffType = 0; + else + ctx->huffType = LIZARD_FLAG_LITERALS + LIZARD_FLAG_FLAGS; // + LIZARD_FLAG_OFFSET16 + LIZARD_FLAG_OFFSET24; + + ctx->literalsBase = (BYTE*)ctx->hashTable + ctx->hashTableSize + ctx->chainTableSize; + ctx->flagsBase = ctx->literalsEnd = ctx->literalsBase + LIZARD_BLOCK_SIZE_PAD; + ctx->lenBase = ctx->flagsEnd = ctx->flagsBase + LIZARD_BLOCK_SIZE_PAD; + ctx->offset16Base = ctx->lenEnd = ctx->lenBase + LIZARD_BLOCK_SIZE_PAD; + ctx->offset24Base = ctx->offset16End = ctx->offset16Base + LIZARD_BLOCK_SIZE_PAD; + ctx->huffBase = ctx->offset24End = ctx->offset24Base + LIZARD_BLOCK_SIZE_PAD; + ctx->huffEnd = ctx->huffBase + LIZARD_COMPRESS_ADD_HUF; + + return ctx; +} + + + +Lizard_stream_t* Lizard_createStream(int compressionLevel) +{ + Lizard_stream_t* ctx = Lizard_initStream(NULL, compressionLevel); + if (ctx) ctx->base = NULL; + return ctx; +} + + +/* initialization */ +Lizard_stream_t* Lizard_resetStream(Lizard_stream_t* ctx, int compressionLevel) +{ + size_t wanted = Lizard_sizeofState(compressionLevel); + + if (ctx->allocatedMemory < wanted) { + Lizard_freeStream(ctx); + ctx = Lizard_createStream(compressionLevel); + } else { + Lizard_initStream(ctx, compressionLevel); + } + + if (ctx) ctx->base = NULL; + return ctx; +} + + +int Lizard_freeStream(Lizard_stream_t* ctx) +{ + if (ctx) { + free(ctx); + } + return 0; +} + + +int Lizard_loadDict(Lizard_stream_t* Lizard_streamPtr, const char* dictionary, int dictSize) +{ + Lizard_stream_t* ctxPtr = (Lizard_stream_t*) Lizard_streamPtr; + if (dictSize > LIZARD_DICT_SIZE) { + dictionary += dictSize - LIZARD_DICT_SIZE; + dictSize = LIZARD_DICT_SIZE; + } + Lizard_init(ctxPtr, (const BYTE*)dictionary); + if (dictSize >= HASH_UPDATE_LIMIT) Lizard_Insert (ctxPtr, (const BYTE*)dictionary + (dictSize - (HASH_UPDATE_LIMIT-1))); + ctxPtr->end = (const BYTE*)dictionary + dictSize; + return dictSize; +} + + +static void Lizard_setExternalDict(Lizard_stream_t* ctxPtr, const BYTE* newBlock) +{ + if (ctxPtr->end >= ctxPtr->base + HASH_UPDATE_LIMIT) Lizard_Insert (ctxPtr, ctxPtr->end - (HASH_UPDATE_LIMIT-1)); /* Referencing remaining dictionary content */ + /* Only one memory segment for extDict, so any previous extDict is lost at this stage */ + ctxPtr->lowLimit = ctxPtr->dictLimit; + ctxPtr->dictLimit = (U32)(ctxPtr->end - ctxPtr->base); + ctxPtr->dictBase = ctxPtr->base; + ctxPtr->base = newBlock - ctxPtr->dictLimit; + ctxPtr->end = newBlock; + ctxPtr->nextToUpdate = ctxPtr->dictLimit; /* match referencing will resume from there */ +} + + +/* dictionary saving */ +int Lizard_saveDict (Lizard_stream_t* Lizard_streamPtr, char* safeBuffer, int dictSize) +{ + Lizard_stream_t* const ctx = (Lizard_stream_t*)Lizard_streamPtr; + int const prefixSize = (int)(ctx->end - (ctx->base + ctx->dictLimit)); + if (dictSize > LIZARD_DICT_SIZE) dictSize = LIZARD_DICT_SIZE; + if (dictSize < 4) dictSize = 0; + if (dictSize > prefixSize) dictSize = prefixSize; + memmove(safeBuffer, ctx->end - dictSize, dictSize); + { U32 const endIndex = (U32)(ctx->end - ctx->base); + ctx->end = (const BYTE*)safeBuffer + dictSize; + ctx->base = ctx->end - endIndex; + ctx->dictLimit = endIndex - dictSize; + ctx->lowLimit = endIndex - dictSize; + if (ctx->nextToUpdate < ctx->dictLimit) ctx->nextToUpdate = ctx->dictLimit; + } + return dictSize; +} + +FORCE_INLINE int Lizard_compress_generic ( + void* ctxvoid, + const char* source, + char* dest, + int inputSize, + int maxOutputSize) +{ + Lizard_stream_t* ctx = (Lizard_stream_t*) ctxvoid; + size_t dictSize = (size_t)(ctx->end - ctx->base) - ctx->dictLimit; + const BYTE* ip = (const BYTE*) source; + BYTE* op = (BYTE*) dest; + BYTE* const oend = op + maxOutputSize; + int res; + + (void)dictSize; + LIZARD_LOG_COMPRESS("Lizard_compress_generic source=%p inputSize=%d dest=%p maxOutputSize=%d cLevel=%d dictBase=%p dictSize=%d\n", source, inputSize, dest, maxOutputSize, ctx->compressionLevel, ctx->dictBase, (int)dictSize); + *op++ = (BYTE)ctx->compressionLevel; + maxOutputSize--; // can be lower than 0 + ctx->end += inputSize; + ctx->srcBase = ctx->off24pos = ip; + ctx->destBase = (BYTE*)dest; + + while (inputSize > 0) + { + int inputPart = MIN(LIZARD_BLOCK_SIZE, inputSize); + + if (ctx->huffType) Lizard_rescaleFreqs(ctx); + Lizard_initBlock(ctx); + ctx->diffBase = ip; + + switch(ctx->params.parserType) + { + default: + case Lizard_parser_fastSmall: + res = Lizard_compress_fastSmall(ctx, ip, ip+inputPart); break; + case Lizard_parser_fast: + res = Lizard_compress_fast(ctx, ip, ip+inputPart); break; + case Lizard_parser_noChain: + res = Lizard_compress_noChain(ctx, ip, ip+inputPart); break; + case Lizard_parser_hashChain: + res = Lizard_compress_hashChain(ctx, ip, ip+inputPart); break; +#ifndef USE_LZ4_ONLY + case Lizard_parser_fastBig: + res = Lizard_compress_fastBig(ctx, ip, ip+inputPart); break; + case Lizard_parser_priceFast: + res = Lizard_compress_priceFast(ctx, ip, ip+inputPart); break; + case Lizard_parser_lowestPrice: + res = Lizard_compress_lowestPrice(ctx, ip, ip+inputPart); break; + case Lizard_parser_optimalPrice: + case Lizard_parser_optimalPriceBT: + res = Lizard_compress_optimalPrice(ctx, ip, ip+inputPart); break; +#else + case Lizard_parser_priceFast: + case Lizard_parser_lowestPrice: + case Lizard_parser_optimalPrice: + case Lizard_parser_optimalPriceBT: + res = 0; +#endif + } + + LIZARD_LOG_COMPRESS("Lizard_compress_generic res=%d inputPart=%d \n", res, inputPart); + if (res <= 0) return res; + + if (Lizard_writeBlock(ctx, ip, inputPart, &op, oend)) goto _output_error; + + ip += inputPart; + inputSize -= inputPart; + LIZARD_LOG_COMPRESS("Lizard_compress_generic in=%d out=%d\n", (int)(ip-(const BYTE*)source), (int)(op-(BYTE*)dest)); + } + + LIZARD_LOG_COMPRESS("Lizard_compress_generic total=%d\n", (int)(op-(BYTE*)dest)); + return (int)(op-(BYTE*)dest); +_output_error: + LIZARD_LOG_COMPRESS("Lizard_compress_generic ERROR\n"); + return 0; +} + + +int Lizard_compress_continue (Lizard_stream_t* ctxPtr, + const char* source, char* dest, + int inputSize, int maxOutputSize) +{ + /* auto-init if required */ + if (ctxPtr->base == NULL) Lizard_init(ctxPtr, (const BYTE*) source); + + /* Check overflow */ + if ((size_t)(ctxPtr->end - ctxPtr->base) > 2 GB) { + size_t dictSize = (size_t)(ctxPtr->end - ctxPtr->base) - ctxPtr->dictLimit; + if (dictSize > LIZARD_DICT_SIZE) dictSize = LIZARD_DICT_SIZE; + Lizard_loadDict((Lizard_stream_t*)ctxPtr, (const char*)(ctxPtr->end) - dictSize, (int)dictSize); + } + + /* Check if blocks follow each other */ + if ((const BYTE*)source != ctxPtr->end) + Lizard_setExternalDict(ctxPtr, (const BYTE*)source); + + /* Check overlapping input/dictionary space */ + { const BYTE* sourceEnd = (const BYTE*) source + inputSize; + const BYTE* const dictBegin = ctxPtr->dictBase + ctxPtr->lowLimit; + const BYTE* const dictEnd = ctxPtr->dictBase + ctxPtr->dictLimit; + if ((sourceEnd > dictBegin) && ((const BYTE*)source < dictEnd)) { + if (sourceEnd > dictEnd) sourceEnd = dictEnd; + ctxPtr->lowLimit = (U32)(sourceEnd - ctxPtr->dictBase); + if (ctxPtr->dictLimit - ctxPtr->lowLimit < 4) ctxPtr->lowLimit = ctxPtr->dictLimit; + } + } + + return Lizard_compress_generic (ctxPtr, source, dest, inputSize, maxOutputSize); +} + + +int Lizard_compress_extState (void* state, const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel) +{ + Lizard_stream_t* ctx = (Lizard_stream_t*) state; + if (((size_t)(state)&(sizeof(void*)-1)) != 0) return 0; /* Error : state is not aligned for pointers (32 or 64 bits) */ + + /* initialize stream */ + Lizard_initStream(ctx, compressionLevel); + Lizard_init((Lizard_stream_t*)state, (const BYTE*)src); + + return Lizard_compress_generic (state, src, dst, srcSize, maxDstSize); +} + + +int Lizard_compress(const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel) +{ + int cSize; + Lizard_stream_t* statePtr = Lizard_createStream(compressionLevel); + + if (!statePtr) return 0; + cSize = Lizard_compress_extState(statePtr, src, dst, srcSize, maxDstSize, compressionLevel); + + Lizard_freeStream(statePtr); + return cSize; +} + + +/************************************** +* Level1 functions +**************************************/ +int Lizard_compress_extState_MinLevel(void* state, const char* source, char* dest, int inputSize, int maxOutputSize) +{ + return Lizard_compress_extState(state, source, dest, inputSize, maxOutputSize, LIZARD_MIN_CLEVEL); +} + +int Lizard_compress_MinLevel(const char* source, char* dest, int inputSize, int maxOutputSize) +{ + return Lizard_compress(source, dest, inputSize, maxOutputSize, LIZARD_MIN_CLEVEL); +} + +Lizard_stream_t* Lizard_createStream_MinLevel(void) +{ + return Lizard_createStream(LIZARD_MIN_CLEVEL); +} + +Lizard_stream_t* Lizard_resetStream_MinLevel(Lizard_stream_t* Lizard_stream) +{ + return Lizard_resetStream (Lizard_stream, LIZARD_MIN_CLEVEL); +} diff --git a/contrib/lizard/lib/lizard_compress.h b/contrib/lizard/lib/lizard_compress.h new file mode 100644 index 00000000000..d84246e2099 --- /dev/null +++ b/contrib/lizard/lib/lizard_compress.h @@ -0,0 +1,208 @@ +/* + Lizard - Fast LZ compression algorithm + Header File + Copyright (C) 2011-2016, Yann Collet + Copyright (C) 2016-2017, Przemyslaw Skibinski + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ +#ifndef LIZARD_H_2983 +#define LIZARD_H_2983 + +#if defined (__cplusplus) +extern "C" { +#endif + +/* + * lizard_compress.h provides block compression functions. It gives full buffer control to user. + * Block compression functions are not-enough to send information, + * since it's still necessary to provide metadata (such as compressed size), + * and each application can do it in whichever way it wants. + * For interoperability, there is Lizard frame specification (lizard_Frame_format.md). + * A library is provided to take care of it, see lizard_frame.h. +*/ + + +/*^*************************************************************** +* Export parameters +*****************************************************************/ +/* +* LIZARD_DLL_EXPORT : +* Enable exporting of functions when building a Windows DLL +*/ +#if defined(LIZARD_DLL_EXPORT) && (LIZARD_DLL_EXPORT==1) +# define LIZARDLIB_API __declspec(dllexport) +#elif defined(LIZARD_DLL_IMPORT) && (LIZARD_DLL_IMPORT==1) +# define LIZARDLIB_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ +#else +# define LIZARDLIB_API +#endif + + +/*-************************************ +* Version +**************************************/ +#define LIZARD_VERSION_MAJOR 1 /* for breaking interface changes */ +#define LIZARD_VERSION_MINOR 0 /* for new (non-breaking) interface capabilities */ +#define LIZARD_VERSION_RELEASE 0 /* for tweaks, bug-fixes, or development */ + +#define LIZARD_VERSION_NUMBER (LIZARD_VERSION_MAJOR *100*100 + LIZARD_VERSION_MINOR *100 + LIZARD_VERSION_RELEASE) +int Lizard_versionNumber (void); + +#define LIZARD_LIB_VERSION LIZARD_VERSION_MAJOR.LIZARD_VERSION_MINOR.LIZARD_VERSION_RELEASE +#define LIZARD_QUOTE(str) #str +#define LIZARD_EXPAND_AND_QUOTE(str) LIZARD_QUOTE(str) +#define LIZARD_VERSION_STRING LIZARD_EXPAND_AND_QUOTE(LIZARD_LIB_VERSION) +const char* Lizard_versionString (void); + +typedef struct Lizard_stream_s Lizard_stream_t; + +#define LIZARD_MIN_CLEVEL 10 /* minimum compression level */ +#ifndef LIZARD_NO_HUFFMAN + #define LIZARD_MAX_CLEVEL 49 /* maximum compression level */ +#else + #define LIZARD_MAX_CLEVEL 29 /* maximum compression level */ +#endif +#define LIZARD_DEFAULT_CLEVEL 17 + + +/*-************************************ +* Simple Functions +**************************************/ + +LIZARDLIB_API int Lizard_compress (const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel); + +/* +Lizard_compress() : + Compresses 'sourceSize' bytes from buffer 'source' + into already allocated 'dest' buffer of size 'maxDestSize'. + Compression is guaranteed to succeed if 'maxDestSize' >= Lizard_compressBound(sourceSize). + It also runs faster, so it's a recommended setting. + If the function cannot compress 'source' into a more limited 'dest' budget, + compression stops *immediately*, and the function result is zero. + As a consequence, 'dest' content is not valid. + This function never writes outside 'dest' buffer, nor read outside 'source' buffer. + sourceSize : Max supported value is LIZARD_MAX_INPUT_VALUE + maxDestSize : full or partial size of buffer 'dest' (which must be already allocated) + return : the number of bytes written into buffer 'dest' (necessarily <= maxOutputSize) + or 0 if compression fails +*/ + + +/*-************************************ +* Advanced Functions +**************************************/ +#define LIZARD_MAX_INPUT_SIZE 0x7E000000 /* 2 113 929 216 bytes */ +#define LIZARD_BLOCK_SIZE (1<<17) +#define LIZARD_BLOCK64K_SIZE (1<<16) +#define LIZARD_COMPRESSBOUND(isize) ((unsigned)(isize) > (unsigned)LIZARD_MAX_INPUT_SIZE ? 0 : (isize) + 1 + 1 + ((isize/LIZARD_BLOCK_SIZE)+1)*4) + + +/*! +Lizard_compressBound() : + Provides the maximum size that Lizard compression may output in a "worst case" scenario (input data not compressible) + This function is primarily useful for memory allocation purposes (destination buffer size). + Macro LIZARD_COMPRESSBOUND() is also provided for compilation-time evaluation (stack memory allocation for example). + Note that Lizard_compress() compress faster when dest buffer size is >= Lizard_compressBound(srcSize) + inputSize : max supported value is LIZARD_MAX_INPUT_SIZE + return : maximum output size in a "worst case" scenario + or 0, if input size is too large ( > LIZARD_MAX_INPUT_SIZE) +*/ +LIZARDLIB_API int Lizard_compressBound(int inputSize); + + +/*! +Lizard_compress_extState() : + Same compression function, just using an externally allocated memory space to store compression state. + Use Lizard_sizeofState() to know how much memory must be allocated, + and allocate it on 8-bytes boundaries (using malloc() typically). + Then, provide it as 'void* state' to compression function. +*/ +LIZARDLIB_API int Lizard_sizeofState(int compressionLevel); + +LIZARDLIB_API int Lizard_compress_extState(void* state, const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel); + + + +/*-********************************************* +* Streaming Compression Functions +***********************************************/ + +/*! Lizard_createStream() will allocate and initialize an `Lizard_stream_t` structure. + * Lizard_freeStream() releases its memory. + * In the context of a DLL (liblizard), please use these methods rather than the static struct. + * They are more future proof, in case of a change of `Lizard_stream_t` size. + */ +LIZARDLIB_API Lizard_stream_t* Lizard_createStream(int compressionLevel); +LIZARDLIB_API int Lizard_freeStream (Lizard_stream_t* streamPtr); + + +/*! Lizard_resetStream() : + * Use this function to reset/reuse an allocated `Lizard_stream_t` structure + */ +LIZARDLIB_API Lizard_stream_t* Lizard_resetStream (Lizard_stream_t* streamPtr, int compressionLevel); + + +/*! Lizard_loadDict() : + * Use this function to load a static dictionary into Lizard_stream. + * Any previous data will be forgotten, only 'dictionary' will remain in memory. + * Loading a size of 0 is allowed. + * Return : dictionary size, in bytes (necessarily <= LIZARD_DICT_SIZE) + */ +LIZARDLIB_API int Lizard_loadDict (Lizard_stream_t* streamPtr, const char* dictionary, int dictSize); + + +/*! Lizard_compress_continue() : + * Compress buffer content 'src', using data from previously compressed blocks as dictionary to improve compression ratio. + * Important : Previous data blocks are assumed to still be present and unmodified ! + * 'dst' buffer must be already allocated. + * If maxDstSize >= Lizard_compressBound(srcSize), compression is guaranteed to succeed, and runs faster. + * If not, and if compressed data cannot fit into 'dst' buffer size, compression stops, and function returns a zero. + */ +LIZARDLIB_API int Lizard_compress_continue (Lizard_stream_t* streamPtr, const char* src, char* dst, int srcSize, int maxDstSize); + + +/*! Lizard_saveDict() : + * If previously compressed data block is not guaranteed to remain available at its memory location, + * save it into a safer place (char* safeBuffer). + * Note : you don't need to call Lizard_loadDict() afterwards, + * dictionary is immediately usable, you can therefore call Lizard_compress_continue(). + * Return : saved dictionary size in bytes (necessarily <= dictSize), or 0 if error. + */ +LIZARDLIB_API int Lizard_saveDict (Lizard_stream_t* streamPtr, char* safeBuffer, int dictSize); + + + + + +#if defined (__cplusplus) +} +#endif + +#endif /* LIZARD_H_2983827168210 */ diff --git a/contrib/lizard/lib/lizard_compress_liz.h b/contrib/lizard/lib/lizard_compress_liz.h new file mode 100644 index 00000000000..f531f36703a --- /dev/null +++ b/contrib/lizard/lib/lizard_compress_liz.h @@ -0,0 +1,301 @@ +#define LIZARD_FREQ_DIV 5 + +FORCE_INLINE void Lizard_setLog2Prices(Lizard_stream_t* ctx) +{ + ctx->log2LitSum = Lizard_highbit32(ctx->litSum+1); + ctx->log2FlagSum = Lizard_highbit32(ctx->flagSum+1); +} + + +MEM_STATIC void Lizard_rescaleFreqs(Lizard_stream_t* ctx) +{ + unsigned u; + + ctx->cachedLiterals = NULL; + ctx->cachedPrice = ctx->cachedLitLength = 0; + + ctx->litPriceSum = 0; + + if (ctx->litSum == 0) { + ctx->litSum = 2 * 256; + ctx->flagSum = 2 * 256; + + for (u=0; u < 256; u++) { + ctx->litFreq[u] = 2; + ctx->flagFreq[u] = 2; + } + } else { + ctx->litSum = 0; + ctx->flagSum = 0; + + for (u=0; u < 256; u++) { + ctx->litFreq[u] = 1 + (ctx->litFreq[u]>>LIZARD_FREQ_DIV); + ctx->litSum += ctx->litFreq[u]; + ctx->flagFreq[u] = 1 + (ctx->flagFreq[u]>>LIZARD_FREQ_DIV); + ctx->flagSum += ctx->flagFreq[u]; + } + } + + Lizard_setLog2Prices(ctx); +} + + +FORCE_INLINE int Lizard_encodeSequence_LIZv1 ( + Lizard_stream_t* ctx, + const BYTE** ip, + const BYTE** anchor, + size_t matchLength, + const BYTE* const match) +{ + U32 offset = (U32)(*ip - match); + size_t length = (size_t)(*ip - *anchor); + BYTE* token = (ctx->flagsPtr)++; + + if (length > 0 || offset < LIZARD_MAX_16BIT_OFFSET) { + /* Encode Literal length */ + // if ((limitedOutputBuffer) && (ctx->literalsPtr > oend - length - LIZARD_LENGTH_SIZE_LIZv1(length) - WILDCOPYLENGTH)) { LIZARD_LOG_COMPRESS_LIZv1("encodeSequence overflow1\n"); return 1; } /* Check output limit */ + if (length >= MAX_SHORT_LITLEN) + { size_t len; + *token = MAX_SHORT_LITLEN; + len = length - MAX_SHORT_LITLEN; + if (len >= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(len)); ctx->literalsPtr += 4; } + else if (len >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(len)); ctx->literalsPtr += 3; } + else *(ctx->literalsPtr)++ = (BYTE)len; + } + else *token = (BYTE)length; + + /* Copy Literals */ + Lizard_wildCopy(ctx->literalsPtr, *anchor, (ctx->literalsPtr) + length); +#ifndef LIZARD_NO_HUFFMAN + if (ctx->huffType) { + ctx->litSum += (U32)length; + ctx->litPriceSum += (U32)(length * ctx->log2LitSum); + { U32 u; + for (u=0; u < length; u++) { + ctx->litPriceSum -= Lizard_highbit32(ctx->litFreq[ctx->literalsPtr[u]]+1); + ctx->litFreq[ctx->literalsPtr[u]]++; + } } + } +#endif + ctx->literalsPtr += length; + + + if (offset >= LIZARD_MAX_16BIT_OFFSET) { + COMPLOG_CODEWORDS_LIZv1("T32+ literal=%u match=%u offset=%d\n", (U32)length, 0, 0); + *token+=(1<huffType) { + ctx->flagFreq[*token]++; + ctx->flagSum++; + } +#endif + token = (ctx->flagsPtr)++; + } + } + + /* Encode Offset */ + if (offset >= LIZARD_MAX_16BIT_OFFSET) // 24-bit offset + { + if (matchLength < MM_LONGOFF) printf("ERROR matchLength=%d/%d\n", (int)matchLength, MM_LONGOFF), exit(1); + + // if ((limitedOutputBuffer) && (ctx->literalsPtr > oend - 8 /*LIZARD_LENGTH_SIZE_LIZv1(length)*/)) { LIZARD_LOG_COMPRESS_LIZv1("encodeSequence overflow2\n"); return 1; } /* Check output limit */ + if (matchLength - MM_LONGOFF >= LIZARD_LAST_LONG_OFF) + { + size_t len = matchLength - MM_LONGOFF - LIZARD_LAST_LONG_OFF; + *token = LIZARD_LAST_LONG_OFF; + if (len >= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(len)); ctx->literalsPtr += 4; } + else if (len >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(len)); ctx->literalsPtr += 3; } + else *(ctx->literalsPtr)++ = (BYTE)len; + COMPLOG_CODEWORDS_LIZv1("T31 literal=%u match=%u offset=%d\n", 0, (U32)matchLength, offset); + } + else + { + COMPLOG_CODEWORDS_LIZv1("T0-30 literal=%u match=%u offset=%d\n", 0, (U32)matchLength, offset); + *token = (BYTE)(matchLength - MM_LONGOFF); + } + + MEM_writeLE24(ctx->offset24Ptr, offset); + ctx->offset24Ptr += 3; + ctx->last_off = offset; + ctx->off24pos = *ip; + } + else + { + COMPLOG_CODEWORDS_LIZv1("T32+ literal=%u match=%u offset=%d\n", (U32)length, (U32)matchLength, offset); + if (offset == 0) + { + *token+=(1<last_off); exit(1); } + + ctx->last_off = offset; + MEM_writeLE16(ctx->offset16Ptr, (U16)ctx->last_off); ctx->offset16Ptr += 2; + } + + /* Encode MatchLength */ + length = matchLength; + // if ((limitedOutputBuffer) && (ctx->literalsPtr > oend - 5 /*LIZARD_LENGTH_SIZE_LIZv1(length)*/)) { LIZARD_LOG_COMPRESS_LIZv1("encodeSequence overflow2\n"); return 1; } /* Check output limit */ + if (length >= MAX_SHORT_MATCHLEN) { + *token += (BYTE)(MAX_SHORT_MATCHLEN<= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(length)); ctx->literalsPtr += 4; } + else if (length >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(length)); ctx->literalsPtr += 3; } + else *(ctx->literalsPtr)++ = (BYTE)length; + } + else *token += (BYTE)(length<huffType) { + ctx->flagFreq[*token]++; + ctx->flagSum++; + Lizard_setLog2Prices(ctx); + } +#endif + + /* Prepare next loop */ + *ip += matchLength; + *anchor = *ip; + + return 0; +} + + +FORCE_INLINE int Lizard_encodeLastLiterals_LIZv1 ( + Lizard_stream_t* ctx, + const BYTE** ip, + const BYTE** anchor) +{ + size_t length = (int)(*ip - *anchor); + (void)ctx; + + memcpy(ctx->literalsPtr, *anchor, length); + ctx->literalsPtr += length; + return 0; +} + + +#define LIZARD_PRICE_MULT 1 +#define LIZARD_GET_TOKEN_PRICE_LIZv1(token) (LIZARD_PRICE_MULT * (ctx->log2FlagSum - Lizard_highbit32(ctx->flagFreq[token]+1))) + + +FORCE_INLINE size_t Lizard_get_price_LIZv1(Lizard_stream_t* const ctx, int rep, const BYTE *ip, const BYTE *off24pos, size_t litLength, U32 offset, size_t matchLength) +{ + size_t price = 0; + BYTE token = 0; +#ifndef LIZARD_NO_HUFFMAN + const BYTE* literals = ip - litLength; + U32 u; + if ((ctx->huffType) && (ctx->params.parserType != Lizard_parser_lowestPrice)) { + if (ctx->cachedLiterals == literals && litLength >= ctx->cachedLitLength) { + size_t const additional = litLength - ctx->cachedLitLength; + const BYTE* literals2 = ctx->cachedLiterals + ctx->cachedLitLength; + price = ctx->cachedPrice + LIZARD_PRICE_MULT * additional * ctx->log2LitSum; + for (u=0; u < additional; u++) + price -= LIZARD_PRICE_MULT * Lizard_highbit32(ctx->litFreq[literals2[u]]+1); + ctx->cachedPrice = (U32)price; + ctx->cachedLitLength = (U32)litLength; + } else { + price = LIZARD_PRICE_MULT * litLength * ctx->log2LitSum; + for (u=0; u < litLength; u++) + price -= LIZARD_PRICE_MULT * Lizard_highbit32(ctx->litFreq[literals[u]]+1); + + if (litLength >= 12) { + ctx->cachedLiterals = literals; + ctx->cachedPrice = (U32)price; + ctx->cachedLitLength = (U32)litLength; + } + } + } + else + price += 8*litLength; /* Copy Literals */ +#else + price += 8*litLength; /* Copy Literals */ + (void)ip; + (void)ctx; +#endif + + (void)off24pos; + (void)rep; + + if (litLength > 0 || offset < LIZARD_MAX_16BIT_OFFSET) { + /* Encode Literal length */ + if (litLength >= MAX_SHORT_LITLEN) + { size_t len = litLength - MAX_SHORT_LITLEN; + token = MAX_SHORT_LITLEN; + if (len >= (1<<16)) price += 32; + else if (len >= 254) price += 24; + else price += 8; + } + else token = (BYTE)litLength; + + if (offset >= LIZARD_MAX_16BIT_OFFSET) { + token+=(1<huffType && ctx->params.parserType != Lizard_parser_lowestPrice) + price += LIZARD_GET_TOKEN_PRICE_LIZv1(token); + else + price += 8; + } + } + + /* Encode Offset */ + if (offset >= LIZARD_MAX_16BIT_OFFSET) { // 24-bit offset + if (matchLength < MM_LONGOFF) return LIZARD_MAX_PRICE; // error + + if (matchLength - MM_LONGOFF >= LIZARD_LAST_LONG_OFF) { + size_t len = matchLength - MM_LONGOFF - LIZARD_LAST_LONG_OFF; + token = LIZARD_LAST_LONG_OFF; + if (len >= (1<<16)) price += 32; + else if (len >= 254) price += 24; + else price += 8; + } else { + token = (BYTE)(matchLength - MM_LONGOFF); + } + + price += 24; + } else { + size_t length; + if (offset == 0) { + token+=(1<= MAX_SHORT_MATCHLEN) { + token += (BYTE)(MAX_SHORT_MATCHLEN<= (1<<16)) price += 32; + else if (length >= 254) price += 24; + else price += 8; + } + else token += (BYTE)(length< 0 || matchLength > 0) { + int offset_load = Lizard_highbit32(offset); + if (ctx->huffType) { + price += ((offset_load>=20) ? ((offset_load-19)*4) : 0); + price += 4 + (matchLength==1); + } else { + price += ((offset_load>=16) ? ((offset_load-15)*4) : 0); + price += 6 + (matchLength==1); + } + if (ctx->huffType && ctx->params.parserType != Lizard_parser_lowestPrice) + price += LIZARD_GET_TOKEN_PRICE_LIZv1(token); + else + price += 8; + } else { + if (ctx->huffType && ctx->params.parserType != Lizard_parser_lowestPrice) + price += LIZARD_GET_TOKEN_PRICE_LIZv1(token); // 1=better ratio + } + + return price; +} diff --git a/contrib/lizard/lib/lizard_compress_lz4.h b/contrib/lizard/lib/lizard_compress_lz4.h new file mode 100644 index 00000000000..00686b6837e --- /dev/null +++ b/contrib/lizard/lib/lizard_compress_lz4.h @@ -0,0 +1,162 @@ +#define LIZARD_LENGTH_SIZE_LZ4(len) ((len >= (1<<16)+RUN_MASK_LZ4) ? 5 : ((len >= 254+RUN_MASK_LZ4) ? 3 : ((len >= RUN_MASK_LZ4) ? 1 : 0))) + +FORCE_INLINE int Lizard_encodeSequence_LZ4 ( + Lizard_stream_t* ctx, + const BYTE** ip, + const BYTE** anchor, + size_t matchLength, + const BYTE* const match) +{ + size_t length = (size_t)(*ip - *anchor); + BYTE* token = (ctx->flagsPtr)++; + (void) ctx; + + COMPLOG_CODEWORDS_LZ4("literal : %u -- match : %u -- offset : %u\n", (U32)(*ip - *anchor), (U32)matchLength, (U32)(*ip-match)); + + /* Encode Literal length */ + // if (ctx->literalsPtr > ctx->literalsEnd - length - LIZARD_LENGTH_SIZE_LZ4(length) - 2 - WILDCOPYLENGTH) { LIZARD_LOG_COMPRESS_LZ4("encodeSequence overflow1\n"); return 1; } /* Check output limit */ + if (length >= RUN_MASK_LZ4) + { size_t len = length - RUN_MASK_LZ4; + *token = RUN_MASK_LZ4; + if (len >= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(len)); ctx->literalsPtr += 4; } + else if (len >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(len)); ctx->literalsPtr += 3; } + else *(ctx->literalsPtr)++ = (BYTE)len; + } + else *token = (BYTE)length; + + /* Copy Literals */ + if (length > 0) { + Lizard_wildCopy(ctx->literalsPtr, *anchor, (ctx->literalsPtr) + length); +#if 0 //def LIZARD_USE_HUFFMAN + ctx->litSum += (U32)length; + ctx->litPriceSum += (U32)(length * ctx->log2LitSum); + { U32 u; + for (u=0; u < length; u++) { + ctx->litPriceSum -= Lizard_highbit32(ctx->litFreq[ctx->literalsPtr[u]]+1); + ctx->litFreq[ctx->literalsPtr[u]]++; + } } +#endif + ctx->literalsPtr += length; + } + + /* Encode Offset */ + MEM_writeLE16(ctx->literalsPtr, (U16)(*ip-match)); + ctx->literalsPtr+=2; + + /* Encode MatchLength */ + length = matchLength - MINMATCH; + // if (ctx->literalsPtr > ctx->literalsEnd - 5 /*LIZARD_LENGTH_SIZE_LZ4(length)*/) { LIZARD_LOG_COMPRESS_LZ4("encodeSequence overflow2\n"); return 1; } /* Check output limit */ + if (length >= ML_MASK_LZ4) { + *token += (BYTE)(ML_MASK_LZ4<= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(length)); ctx->literalsPtr += 4; } + else if (length >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(length)); ctx->literalsPtr += 3; } + else *(ctx->literalsPtr)++ = (BYTE)length; + } + else *token += (BYTE)(length<huffType) { + ctx->flagFreq[*token]++; + ctx->flagSum++; + Lizard_setLog2Prices(ctx); + } +#endif + + /* Prepare next loop */ + *ip += matchLength; + *anchor = *ip; + + return 0; +} + + +FORCE_INLINE int Lizard_encodeLastLiterals_LZ4 ( + Lizard_stream_t* ctx, + const BYTE** ip, + const BYTE** anchor) +{ + size_t length = (int)(*ip - *anchor); + + (void)ctx; + + memcpy(ctx->literalsPtr, *anchor, length); + ctx->literalsPtr += length; + return 0; +} + + +#define LIZARD_GET_TOKEN_PRICE_LZ4(token) (ctx->log2FlagSum - Lizard_highbit32(ctx->flagFreq[token]+1)) + +FORCE_INLINE size_t Lizard_get_price_LZ4(Lizard_stream_t* const ctx, const BYTE *ip, const size_t litLength, U32 offset, size_t matchLength) +{ + size_t price = 0; + BYTE token = 0; +#if 0 //def LIZARD_USE_HUFFMAN + const BYTE* literals = ip - litLength; + U32 u; + + if (ctx->cachedLiterals == literals && litLength >= ctx->cachedLitLength) { + size_t const additional = litLength - ctx->cachedLitLength; + const BYTE* literals2 = ctx->cachedLiterals + ctx->cachedLitLength; + price = ctx->cachedPrice + additional * ctx->log2LitSum; + for (u=0; u < additional; u++) + price -= Lizard_highbit32(ctx->litFreq[literals2[u]]+1); + ctx->cachedPrice = (U32)price; + ctx->cachedLitLength = (U32)litLength; + } else { + price = litLength * ctx->log2LitSum; + for (u=0; u < litLength; u++) + price -= Lizard_highbit32(ctx->litFreq[literals[u]]+1); + + if (litLength >= 12) { + ctx->cachedLiterals = literals; + ctx->cachedPrice = (U32)price; + ctx->cachedLitLength = (U32)litLength; + } + } +#else + price += 8*litLength; /* Copy Literals */ + (void)ip; + (void)ctx; +#endif + + /* Encode Literal length */ + if (litLength >= RUN_MASK_LZ4) { + size_t len = litLength - RUN_MASK_LZ4; + token = RUN_MASK_LZ4; + if (len >= (1<<16)) price += 32; + else if (len >= 254) price += 24; + else price += 8; + } + else token = (BYTE)litLength; + + + /* Encode MatchLength */ + if (offset) { + size_t length; + price += 16; /* Encode Offset */ + + if (offset < 8) return LIZARD_MAX_PRICE; // error + if (matchLength < MINMATCH) return LIZARD_MAX_PRICE; // error + + length = matchLength - MINMATCH; + if (length >= ML_MASK_LZ4) { + token += (BYTE)(ML_MASK_LZ4<= (1<<16)) price += 32; + else if (length >= 254) price += 24; + else price += 8; + } + else token += (BYTE)(length<huffType) { + if (offset > 0 || matchLength > 0) price += 2; + price += LIZARD_GET_TOKEN_PRICE_LZ4(token); + } else { + price += 8; // token + } + + return price; +} diff --git a/contrib/lizard/lib/lizard_decompress.c b/contrib/lizard/lib/lizard_decompress.c new file mode 100644 index 00000000000..df2eb699deb --- /dev/null +++ b/contrib/lizard/lib/lizard_decompress.c @@ -0,0 +1,372 @@ +/* + Lizard - Fast LZ compression algorithm + Copyright (C) 2011-2016, Yann Collet + Copyright (C) 2016-2017, Przemyslaw Skibinski + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + + +/************************************** +* Includes +**************************************/ +//#define LIZARD_STATS 1 // 0=simple stats, 1=more, 2=full +#ifdef LIZARD_STATS + #include "test/lizard_stats.h" +#endif +#include "lizard_compress.h" +#include "lizard_decompress.h" +#include "lizard_common.h" +#include // printf +#include // intptr_t + + +/*-************************************ +* Local Structures and types +**************************************/ +typedef enum { noDict = 0, withPrefix64k, usingExtDict } dict_directive; +typedef enum { full = 0, partial = 1 } earlyEnd_directive; + +#include "lizard_decompress_lz4.h" +#ifndef USE_LZ4_ONLY + #ifdef LIZARD_USE_TEST + #include "test/lizard_common_test.h" + #include "test/lizard_decompress_test.h" + #else + #include "lizard_decompress_liz.h" + #endif +#endif +#include "entropy/huf.h" + + +/*-***************************** +* Decompression functions +*******************************/ + +FORCE_INLINE size_t Lizard_readStream(int flag, const BYTE** ip, const BYTE* const iend, BYTE* op, BYTE* const oend, const BYTE** streamPtr, const BYTE** streamEnd, int streamFlag) +{ + if (!flag) { + if (*ip > iend - 3) return 0; + *streamPtr = *ip + 3; + *streamEnd = *streamPtr + MEM_readLE24(*ip); + if (*streamEnd < *streamPtr) return 0; + *ip = *streamEnd; +#ifdef LIZARD_STATS + uncompr_stream[streamFlag] += *streamEnd-*streamPtr; +#else + (void)streamFlag; +#endif + return 1; + } else { +#ifndef LIZARD_NO_HUFFMAN + size_t res, streamLen, comprStreamLen; + + if (*ip > iend - 6) return 0; + streamLen = MEM_readLE24(*ip); + comprStreamLen = MEM_readLE24(*ip + 3); + + if ((op > oend - streamLen) || (*ip + comprStreamLen > iend - 6)) return 0; + res = HUF_decompress(op, streamLen, *ip + 6, comprStreamLen); + if (HUF_isError(res) || (res != streamLen)) return 0; + + *ip += comprStreamLen + 6; + *streamPtr = op; + *streamEnd = *streamPtr + streamLen; +#ifdef LIZARD_STATS + compr_stream[streamFlag] += comprStreamLen + 6; + decompr_stream[streamFlag] += *streamEnd-*streamPtr; +#endif + return 1; +#else + fprintf(stderr, "compiled with LIZARD_NO_HUFFMAN\n"); + (void)op; (void)oend; + return 0; +#endif + } +} + + +FORCE_INLINE int Lizard_decompress_generic( + const char* source, + char* const dest, + int inputSize, + int outputSize, /* this value is the max size of Output Buffer. */ + int partialDecoding, /* full, partial */ + int targetOutputSize, /* only used if partialDecoding==partial */ + int dict, /* noDict, withPrefix64k, usingExtDict */ + const BYTE* const lowPrefix, /* == dest if dict == noDict */ + const BYTE* const dictStart, /* only if dict==usingExtDict */ + const size_t dictSize /* note : = 0 if noDict */ + ) +{ + /* Local Variables */ + const BYTE* ip = (const BYTE*) source, *istart = (const BYTE*) source; + const BYTE* const iend = ip + inputSize; + BYTE* op = (BYTE*) dest; + BYTE* const oend = op + outputSize; + BYTE* oexit = op + targetOutputSize; + Lizard_parameters params; + Lizard_dstream_t ctx; + BYTE* decompFlagsBase, *decompOff24Base, *decompOff16Base, *decompLiteralsBase = NULL; + int res, compressionLevel; + + if (inputSize < 1) { LIZARD_LOG_DECOMPRESS("inputSize=%d outputSize=%d targetOutputSize=%d partialDecoding=%d\n", inputSize, outputSize, targetOutputSize, partialDecoding); return 0; } + + compressionLevel = *ip++; + + if (compressionLevel < LIZARD_MIN_CLEVEL || compressionLevel > LIZARD_MAX_CLEVEL) { + LIZARD_LOG_DECOMPRESS("ERROR Lizard_decompress_generic inputSize=%d compressionLevel=%d\n", inputSize, compressionLevel); + return -1; + } + + LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic ip=%p inputSize=%d targetOutputSize=%d dest=%p outputSize=%d cLevel=%d dict=%d dictSize=%d dictStart=%p partialDecoding=%d\n", ip, inputSize, targetOutputSize, dest, outputSize, compressionLevel, dict, (int)dictSize, dictStart, partialDecoding); + + decompLiteralsBase = (BYTE*)malloc(4*LIZARD_HUF_BLOCK_SIZE); + if (!decompLiteralsBase) return -1; + decompFlagsBase = decompLiteralsBase + LIZARD_HUF_BLOCK_SIZE; + decompOff24Base = decompFlagsBase + LIZARD_HUF_BLOCK_SIZE; + decompOff16Base = decompOff24Base + LIZARD_HUF_BLOCK_SIZE; + +#ifdef LIZARD_STATS + init_stats(); +#endif + (void)istart; + + while (ip < iend) + { + res = *ip++; + if (res == LIZARD_FLAG_UNCOMPRESSED) /* uncompressed */ + { + uint32_t length; + if (ip > iend - 3) { LIZARD_LOG_DECOMPRESS("UNCOMPRESSED ip[%p] > iend[%p] - 3\n", ip, iend); goto _output_error; } + length = MEM_readLE24(ip); + ip += 3; + // printf("%d: total=%d block=%d UNCOMPRESSED op=%p oexit=%p oend=%p\n", (int)(op-(BYTE*)dest) ,(int)(ip-istart), length, op, oexit, oend); + if (ip + length > iend || op + length > oend) { LIZARD_LOG_DECOMPRESS("UNCOMPRESSED ip[%p]+length[%d] > iend[%p]\n", ip, length, iend); goto _output_error; } + memcpy(op, ip, length); + op += length; + ip += length; + if ((partialDecoding) && (op >= oexit)) break; +#ifdef LIZARD_STATS + uncompr_stream[LIZARD_STREAM_UNCOMPRESSED] += length; +#endif + continue; + } + + if (res&LIZARD_FLAG_LEN) { + LIZARD_LOG_DECOMPRESS("res=%d\n", res); goto _output_error; + } + + if (ip > iend - 5*3) goto _output_error; + ctx.lenPtr = (const BYTE*)ip + 3; + ctx.lenEnd = ctx.lenPtr + MEM_readLE24(ip); + if (ctx.lenEnd < ctx.lenPtr || (ctx.lenEnd > iend - 3)) goto _output_error; +#ifdef LIZARD_STATS + uncompr_stream[LIZARD_STREAM_LEN] += ctx.lenEnd-ctx.lenPtr + 3; +#endif + ip = ctx.lenEnd; + + { size_t streamLen; +#ifdef LIZARD_USE_LOGS + const BYTE* ipos; + size_t comprFlagsLen, comprLiteralsLen, total; +#endif + streamLen = Lizard_readStream(res&LIZARD_FLAG_OFFSET16, &ip, iend, decompOff16Base, decompOff16Base + LIZARD_HUF_BLOCK_SIZE, &ctx.offset16Ptr, &ctx.offset16End, LIZARD_STREAM_OFFSET16); + if (streamLen == 0) goto _output_error; + + streamLen = Lizard_readStream(res&LIZARD_FLAG_OFFSET24, &ip, iend, decompOff24Base, decompOff24Base + LIZARD_HUF_BLOCK_SIZE, &ctx.offset24Ptr, &ctx.offset24End, LIZARD_STREAM_OFFSET24); + if (streamLen == 0) goto _output_error; + +#ifdef LIZARD_USE_LOGS + ipos = ip; + streamLen = Lizard_readStream(res&LIZARD_FLAG_FLAGS, &ip, iend, decompFlagsBase, decompFlagsBase + LIZARD_HUF_BLOCK_SIZE, &ctx.flagsPtr, &ctx.flagsEnd, LIZARD_STREAM_FLAGS); + if (streamLen == 0) goto _output_error; + streamLen = (size_t)(ctx.flagsEnd-ctx.flagsPtr); + comprFlagsLen = ((size_t)(ip - ipos) + 3 >= streamLen) ? 0 : (size_t)(ip - ipos); + ipos = ip; +#else + streamLen = Lizard_readStream(res&LIZARD_FLAG_FLAGS, &ip, iend, decompFlagsBase, decompFlagsBase + LIZARD_HUF_BLOCK_SIZE, &ctx.flagsPtr, &ctx.flagsEnd, LIZARD_STREAM_FLAGS); + if (streamLen == 0) goto _output_error; +#endif + + streamLen = Lizard_readStream(res&LIZARD_FLAG_LITERALS, &ip, iend, decompLiteralsBase, decompLiteralsBase + LIZARD_HUF_BLOCK_SIZE, &ctx.literalsPtr, &ctx.literalsEnd, LIZARD_STREAM_LITERALS); + if (streamLen == 0) goto _output_error; +#ifdef LIZARD_USE_LOGS + streamLen = (size_t)(ctx.literalsEnd-ctx.literalsPtr); + comprLiteralsLen = ((size_t)(ip - ipos) + 3 >= streamLen) ? 0 : (size_t)(ip - ipos); + total = (size_t)(ip-(ctx.lenEnd-1)); +#endif + + if (ip > iend) goto _output_error; + + LIZARD_LOG_DECOMPRESS("%d: total=%d block=%d flagsLen=%d(HUF=%d) literalsLen=%d(HUF=%d) offset16Len=%d offset24Len=%d lengthsLen=%d \n", (int)(op-(BYTE*)dest) ,(int)(ip-istart), (int)total, + (int)(ctx.flagsEnd-ctx.flagsPtr), (int)comprFlagsLen, (int)(ctx.literalsEnd-ctx.literalsPtr), (int)comprLiteralsLen, + (int)(ctx.offset16End-ctx.offset16Ptr), (int)(ctx.offset24End-ctx.offset24Ptr), (int)(ctx.lenEnd-ctx.lenPtr)); + } + + ctx.last_off = -LIZARD_INIT_LAST_OFFSET; + params = Lizard_defaultParameters[compressionLevel - LIZARD_MIN_CLEVEL]; + if (params.decompressType == Lizard_coderwords_LZ4) + res = Lizard_decompress_LZ4(&ctx, op, outputSize, partialDecoding, targetOutputSize, dict, lowPrefix, dictStart, dictSize, compressionLevel); + else +#ifdef USE_LZ4_ONLY + res = Lizard_decompress_LZ4(&ctx, op, outputSize, partialDecoding, targetOutputSize, dict, lowPrefix, dictStart, dictSize, compressionLevel); +#else + res = Lizard_decompress_LIZv1(&ctx, op, outputSize, partialDecoding, targetOutputSize, dict, lowPrefix, dictStart, dictSize, compressionLevel); +#endif + LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic res=%d inputSize=%d\n", res, (int)(ctx.literalsEnd-ctx.lenEnd)); + + if (res <= 0) { free(decompLiteralsBase); return res; } + + op += res; + outputSize -= res; + if ((partialDecoding) && (op >= oexit)) break; + } + +#ifdef LIZARD_STATS + print_stats(); +#endif + + LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic total=%d\n", (int)(op-(BYTE*)dest)); + free(decompLiteralsBase); + return (int)(op-(BYTE*)dest); + +_output_error: + LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic ERROR ip=%p iend=%p\n", ip, iend); + free(decompLiteralsBase); + return -1; +} + + +int Lizard_decompress_safe(const char* source, char* dest, int compressedSize, int maxDecompressedSize) +{ + return Lizard_decompress_generic(source, dest, compressedSize, maxDecompressedSize, full, 0, noDict, (BYTE*)dest, NULL, 0); +} + +int Lizard_decompress_safe_partial(const char* source, char* dest, int compressedSize, int targetOutputSize, int maxDecompressedSize) +{ + return Lizard_decompress_generic(source, dest, compressedSize, maxDecompressedSize, partial, targetOutputSize, noDict, (BYTE*)dest, NULL, 0); +} + + +/*===== streaming decompression functions =====*/ + + +/* + * If you prefer dynamic allocation methods, + * Lizard_createStreamDecode() + * provides a pointer (void*) towards an initialized Lizard_streamDecode_t structure. + */ +Lizard_streamDecode_t* Lizard_createStreamDecode(void) +{ + Lizard_streamDecode_t* lizards = (Lizard_streamDecode_t*) ALLOCATOR(1, sizeof(Lizard_streamDecode_t)); + return lizards; +} + +int Lizard_freeStreamDecode (Lizard_streamDecode_t* Lizard_stream) +{ + FREEMEM(Lizard_stream); + return 0; +} + +/*! + * Lizard_setStreamDecode() : + * Use this function to instruct where to find the dictionary. + * This function is not necessary if previous data is still available where it was decoded. + * Loading a size of 0 is allowed (same effect as no dictionary). + * Return : 1 if OK, 0 if error + */ +int Lizard_setStreamDecode (Lizard_streamDecode_t* Lizard_streamDecode, const char* dictionary, int dictSize) +{ + Lizard_streamDecode_t* lizardsd = (Lizard_streamDecode_t*) Lizard_streamDecode; + lizardsd->prefixSize = (size_t) dictSize; + lizardsd->prefixEnd = (const BYTE*) dictionary + dictSize; + lizardsd->externalDict = NULL; + lizardsd->extDictSize = 0; + return 1; +} + +/* +*_continue() : + These decoding functions allow decompression of multiple blocks in "streaming" mode. + Previously decoded blocks must still be available at the memory position where they were decoded. + If it's not possible, save the relevant part of decoded data into a safe buffer, + and indicate where it stands using Lizard_setStreamDecode() +*/ +int Lizard_decompress_safe_continue (Lizard_streamDecode_t* Lizard_streamDecode, const char* source, char* dest, int compressedSize, int maxOutputSize) +{ + Lizard_streamDecode_t* lizardsd = (Lizard_streamDecode_t*) Lizard_streamDecode; + int result; + + if (lizardsd->prefixEnd == (BYTE*)dest) { + result = Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, + full, 0, usingExtDict, lizardsd->prefixEnd - lizardsd->prefixSize, lizardsd->externalDict, lizardsd->extDictSize); + if (result <= 0) return result; + lizardsd->prefixSize += result; + lizardsd->prefixEnd += result; + } else { + lizardsd->extDictSize = lizardsd->prefixSize; + lizardsd->externalDict = lizardsd->prefixEnd - lizardsd->extDictSize; + result = Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, + full, 0, usingExtDict, (BYTE*)dest, lizardsd->externalDict, lizardsd->extDictSize); + if (result <= 0) return result; + lizardsd->prefixSize = result; + lizardsd->prefixEnd = (BYTE*)dest + result; + } + + return result; +} + + +/* +Advanced decoding functions : +*_usingDict() : + These decoding functions work the same as "_continue" ones, + the dictionary must be explicitly provided within parameters +*/ + +int Lizard_decompress_safe_usingDict(const char* source, char* dest, int compressedSize, int maxOutputSize, const char* dictStart, int dictSize) +{ + if (dictSize==0) + return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, noDict, (BYTE*)dest, NULL, 0); + if (dictStart+dictSize == dest) + { + if (dictSize >= (int)(LIZARD_DICT_SIZE - 1)) + return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, withPrefix64k, (BYTE*)dest-LIZARD_DICT_SIZE, NULL, 0); + return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, noDict, (BYTE*)dest-dictSize, NULL, 0); + } + return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, usingExtDict, (BYTE*)dest, (const BYTE*)dictStart, dictSize); +} + +/* debug function */ +int Lizard_decompress_safe_forceExtDict(const char* source, char* dest, int compressedSize, int maxOutputSize, const char* dictStart, int dictSize) +{ + return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, usingExtDict, (BYTE*)dest, (const BYTE*)dictStart, dictSize); +} + diff --git a/contrib/lizard/lib/lizard_decompress.h b/contrib/lizard/lib/lizard_decompress.h new file mode 100644 index 00000000000..ad9fc8ee941 --- /dev/null +++ b/contrib/lizard/lib/lizard_decompress.h @@ -0,0 +1,152 @@ +/* + Lizard - Fast LZ compression algorithm + Header File + Copyright (C) 2011-2016, Yann Collet + Copyright (C) 2016-2017, Przemyslaw Skibinski + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ +#ifndef LIZARD_DECOMPRESS_H_2983 +#define LIZARD_DECOMPRESS_H_2983 + +#if defined (__cplusplus) +extern "C" { +#endif + + +/*^*************************************************************** +* Export parameters +*****************************************************************/ +/* +* LIZARD_DLL_EXPORT : +* Enable exporting of functions when building a Windows DLL +*/ +#if defined(LIZARD_DLL_EXPORT) && (LIZARD_DLL_EXPORT==1) +# define LIZARDDLIB_API __declspec(dllexport) +#elif defined(LIZARD_DLL_IMPORT) && (LIZARD_DLL_IMPORT==1) +# define LIZARDDLIB_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ +#else +# define LIZARDDLIB_API +#endif + + +/*-************************************ +* Simple Functions +**************************************/ + +/* +Lizard_decompress_safe() : + compressedSize : is the precise full size of the compressed block. + maxDecompressedSize : is the size of destination buffer, which must be already allocated. + return : the number of bytes decompressed into destination buffer (necessarily <= maxDecompressedSize) + If destination buffer is not large enough, decoding will stop and output an error code (<0). + If the source stream is detected malformed, the function will stop decoding and return a negative result. + This function is protected against buffer overflow exploits, including malicious data packets. + It never writes outside output buffer, nor reads outside input buffer. +*/ +LIZARDDLIB_API int Lizard_decompress_safe (const char* source, char* dest, int compressedSize, int maxDecompressedSize); + + + +/*! +Lizard_decompress_safe_partial() : + This function decompress a compressed block of size 'compressedSize' at position 'source' + into destination buffer 'dest' of size 'maxDecompressedSize'. + The function tries to stop decompressing operation as soon as 'targetOutputSize' has been reached, + reducing decompression time. + return : the number of bytes decoded in the destination buffer (necessarily <= maxDecompressedSize) + Note : this number can be < 'targetOutputSize' should the compressed block to decode be smaller. + Always control how many bytes were decoded. + If the source stream is detected malformed, the function will stop decoding and return a negative result. + This function never writes outside of output buffer, and never reads outside of input buffer. It is therefore protected against malicious data packets +*/ +LIZARDDLIB_API int Lizard_decompress_safe_partial (const char* source, char* dest, int compressedSize, int targetOutputSize, int maxDecompressedSize); + + + +/*-********************************************** +* Streaming Decompression Functions +************************************************/ +typedef struct Lizard_streamDecode_s Lizard_streamDecode_t; + +/* + * Lizard_streamDecode_t + * information structure to track an Lizard stream. + * init this structure content using Lizard_setStreamDecode or memset() before first use ! + * + * In the context of a DLL (liblizard) please prefer usage of construction methods below. + * They are more future proof, in case of a change of Lizard_streamDecode_t size in the future. + * Lizard_createStreamDecode will allocate and initialize an Lizard_streamDecode_t structure + * Lizard_freeStreamDecode releases its memory. + */ +LIZARDDLIB_API Lizard_streamDecode_t* Lizard_createStreamDecode(void); +LIZARDDLIB_API int Lizard_freeStreamDecode (Lizard_streamDecode_t* Lizard_stream); + +/*! Lizard_setStreamDecode() : + * Use this function to instruct where to find the dictionary. + * Setting a size of 0 is allowed (same effect as reset). + * @return : 1 if OK, 0 if error + */ +LIZARDDLIB_API int Lizard_setStreamDecode (Lizard_streamDecode_t* Lizard_streamDecode, const char* dictionary, int dictSize); + +/* +*_continue() : + These decoding functions allow decompression of multiple blocks in "streaming" mode. + Previously decoded blocks *must* remain available at the memory position where they were decoded (up to LIZARD_DICT_SIZE) + In the case of a ring buffers, decoding buffer must be either : + - Exactly same size as encoding buffer, with same update rule (block boundaries at same positions) + In which case, the decoding & encoding ring buffer can have any size, including small ones ( < LIZARD_DICT_SIZE). + - Larger than encoding buffer, by a minimum of maxBlockSize more bytes. + maxBlockSize is implementation dependent. It's the maximum size you intend to compress into a single block. + In which case, encoding and decoding buffers do not need to be synchronized, + and encoding ring buffer can have any size, including small ones ( < LIZARD_DICT_SIZE). + - _At least_ LIZARD_DICT_SIZE + 8 bytes + maxBlockSize. + In which case, encoding and decoding buffers do not need to be synchronized, + and encoding ring buffer can have any size, including larger than decoding buffer. + Whenever these conditions are not possible, save the last LIZARD_DICT_SIZE of decoded data into a safe buffer, + and indicate where it is saved using Lizard_setStreamDecode() +*/ +LIZARDDLIB_API int Lizard_decompress_safe_continue (Lizard_streamDecode_t* Lizard_streamDecode, const char* source, char* dest, int compressedSize, int maxDecompressedSize); + + +/* +Advanced decoding functions : +*_usingDict() : + These decoding functions work the same as + a combination of Lizard_setStreamDecode() followed by Lizard_decompress_x_continue() + They are stand-alone. They don't need nor update an Lizard_streamDecode_t structure. +*/ +LIZARDDLIB_API int Lizard_decompress_safe_usingDict (const char* source, char* dest, int compressedSize, int maxDecompressedSize, const char* dictStart, int dictSize); + + +#if defined (__cplusplus) +} +#endif + +#endif /* LIZARD_DECOMPRESS_H_2983827168210 */ diff --git a/contrib/lizard/lib/lizard_decompress_liz.h b/contrib/lizard/lib/lizard_decompress_liz.h new file mode 100644 index 00000000000..7e80eb5e01b --- /dev/null +++ b/contrib/lizard/lib/lizard_decompress_liz.h @@ -0,0 +1,220 @@ +/* + [0_MMMM_LLL] - 16-bit offset, 4-bit match length (4-15+), 3-bit literal length (0-7+) + [1_MMMM_LLL] - last offset, 4-bit match length (0-15+), 3-bit literal length (0-7+) + flag 31 - 24-bit offset, match length (47+), no literal length + flag 0-30 - 24-bit offset, 31 match lengths (16-46), no literal length +*/ + +/*! Lizard_decompress_LIZv1() : + * This generic decompression function cover all use cases. + * It shall be instantiated several times, using different sets of directives + * Note that it is important this generic function is really inlined, + * in order to remove useless branches during compilation optimization. + */ +FORCE_INLINE int Lizard_decompress_LIZv1( + Lizard_dstream_t* ctx, + BYTE* const dest, + int outputSize, /* this value is the max size of Output Buffer. */ + + int partialDecoding, /* full, partial */ + int targetOutputSize, /* only used if partialDecoding==partial */ + int dict, /* noDict, withPrefix64k, usingExtDict */ + const BYTE* const lowPrefix, /* == dest if dict == noDict */ + const BYTE* const dictStart, /* only if dict==usingExtDict */ + const size_t dictSize, /* note : = 0 if noDict */ + int compressionLevel + ) +{ + /* Local Variables */ + int inputSize = (int)(ctx->flagsEnd - ctx->flagsPtr); + const BYTE* const blockBase = ctx->flagsPtr; + const BYTE* const iend = ctx->literalsEnd; + + BYTE* op = dest; + BYTE* const oend = op + outputSize; + BYTE* cpy = NULL; + BYTE* oexit = op + targetOutputSize; + const BYTE* const lowLimit = lowPrefix - dictSize; + const BYTE* const dictEnd = (const BYTE*)dictStart + dictSize; + + const int checkOffset = (dictSize < (int)(LIZARD_DICT_SIZE)); + + intptr_t last_off = ctx->last_off; + intptr_t length = 0; + (void)compressionLevel; + + /* Special cases */ + if (unlikely(outputSize==0)) return ((inputSize==1) && (*ctx->flagsPtr==0)) ? 0 : -1; /* Empty output buffer */ + + /* Main Loop : decode sequences */ + while (ctx->flagsPtr < ctx->flagsEnd) { + unsigned token; + const BYTE* match; + // intptr_t litLength; + + if ((partialDecoding) && (op >= oexit)) return (int) (op-dest); + + /* get literal length */ + token = *ctx->flagsPtr++; + + if (token >= 32) + { + if ((length=(token & MAX_SHORT_LITLEN)) == MAX_SHORT_LITLEN) { + if (unlikely(ctx->literalsPtr > iend - 1)) { LIZARD_LOG_DECOMPRESS_LIZv1("1"); goto _output_error; } + length = *ctx->literalsPtr; + if unlikely(length >= 254) { + if (length == 254) { + length = MEM_readLE16(ctx->literalsPtr+1); + ctx->literalsPtr += 2; + } else { + length = MEM_readLE24(ctx->literalsPtr+1); + ctx->literalsPtr += 3; + } + } + length += MAX_SHORT_LITLEN; + ctx->literalsPtr++; + if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LIZv1("2"); goto _output_error; } /* overflow detection */ + if (unlikely((size_t)(ctx->literalsPtr+length)<(size_t)(ctx->literalsPtr))) { LIZARD_LOG_DECOMPRESS_LIZv1("3"); goto _output_error; } /* overflow detection */ + } + + /* copy literals */ + cpy = op + length; + if (unlikely(cpy > oend - WILDCOPYLENGTH || ctx->literalsPtr > iend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LIZv1("offset outside buffers\n"); goto _output_error; } /* Error : offset outside buffers */ + #if 1 + Lizard_wildCopy16(op, ctx->literalsPtr, cpy); + op = cpy; + ctx->literalsPtr += length; + #else + Lizard_copy8(op, ctx->literalsPtr); + Lizard_copy8(op+8, ctx->literalsPtr+8); + if (length > 16) + Lizard_wildCopy16(op + 16, ctx->literalsPtr + 16, cpy); + op = cpy; + ctx->literalsPtr += length; + #endif + + /* get offset */ + if (unlikely(ctx->offset16Ptr > ctx->offset16End)) { LIZARD_LOG_DECOMPRESS_LIZv1("(ctx->offset16Ptr > ctx->offset16End\n"); goto _output_error; } +#if 1 + { /* branchless */ + intptr_t new_off = MEM_readLE16(ctx->offset16Ptr); + uintptr_t not_repCode = (uintptr_t)(token >> ML_RUN_BITS) - 1; + last_off ^= not_repCode & (last_off ^ -new_off); + ctx->offset16Ptr = (BYTE*)((uintptr_t)ctx->offset16Ptr + (not_repCode & 2)); + } +#else + if ((token >> ML_RUN_BITS) == 0) + { + last_off = -(intptr_t)MEM_readLE16(ctx->offset16Ptr); + ctx->offset16Ptr += 2; + } +#endif + + /* get matchlength */ + length = (token >> RUN_BITS_LIZv1) & MAX_SHORT_MATCHLEN; + if (length == MAX_SHORT_MATCHLEN) { + if (unlikely(ctx->literalsPtr > iend - 1)) { LIZARD_LOG_DECOMPRESS_LIZv1("6"); goto _output_error; } + length = *ctx->literalsPtr; + if unlikely(length >= 254) { + if (length == 254) { + length = MEM_readLE16(ctx->literalsPtr+1); + ctx->literalsPtr += 2; + } else { + length = MEM_readLE24(ctx->literalsPtr+1); + ctx->literalsPtr += 3; + } + } + length += MAX_SHORT_MATCHLEN; + ctx->literalsPtr++; + if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LIZv1("7"); goto _output_error; } /* overflow detection */ + } + + DECOMPLOG_CODEWORDS_LIZv1("T32+ literal=%u match=%u offset=%d ipos=%d opos=%d\n", (U32)litLength, (U32)length, (int)-last_off, (U32)(ctx->flagsPtr-blockBase), (U32)(op-dest)); + } + else + if (token < LIZARD_LAST_LONG_OFF) + { + if (unlikely(ctx->offset24Ptr > ctx->offset24End - 3)) { LIZARD_LOG_DECOMPRESS_LIZv1("8"); goto _output_error; } + length = token + MM_LONGOFF; + last_off = -(intptr_t)MEM_readLE24(ctx->offset24Ptr); + ctx->offset24Ptr += 3; + DECOMPLOG_CODEWORDS_LIZv1("T0-30 literal=%u match=%u offset=%d\n", 0, (U32)length, (int)-last_off); + } + else + { + if (unlikely(ctx->literalsPtr > iend - 1)) { LIZARD_LOG_DECOMPRESS_LIZv1("9"); goto _output_error; } + length = *ctx->literalsPtr; + if unlikely(length >= 254) { + if (length == 254) { + length = MEM_readLE16(ctx->literalsPtr+1); + ctx->literalsPtr += 2; + } else { + length = MEM_readLE24(ctx->literalsPtr+1); + ctx->literalsPtr += 3; + } + } + ctx->literalsPtr++; + length += LIZARD_LAST_LONG_OFF + MM_LONGOFF; + + if (unlikely(ctx->offset24Ptr > ctx->offset24End - 3)) { LIZARD_LOG_DECOMPRESS_LIZv1("10"); goto _output_error; } + last_off = -(intptr_t)MEM_readLE24(ctx->offset24Ptr); + ctx->offset24Ptr += 3; + } + + + match = op + last_off; + if ((checkOffset) && ((unlikely((uintptr_t)(-last_off) > (uintptr_t)op) || (match < lowLimit)))) { LIZARD_LOG_DECOMPRESS_LIZv1("lowPrefix[%p]-dictSize[%d]=lowLimit[%p] match[%p]=op[%p]-last_off[%d]\n", lowPrefix, (int)dictSize, lowLimit, match, op, (int)last_off); goto _output_error; } /* Error : offset outside buffers */ + + /* check external dictionary */ + if ((dict==usingExtDict) && (match < lowPrefix)) { + if (unlikely(op + length > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LIZv1("12"); goto _output_error; } /* doesn't respect parsing restriction */ + + if (length <= (intptr_t)(lowPrefix - match)) { + /* match can be copied as a single segment from external dictionary */ + memmove(op, dictEnd - (lowPrefix-match), length); + op += length; + } else { + /* match encompass external dictionary and current block */ + size_t const copySize = (size_t)(lowPrefix-match); + size_t const restSize = length - copySize; + memcpy(op, dictEnd - copySize, copySize); + op += copySize; + if (restSize > (size_t)(op-lowPrefix)) { /* overlap copy */ + BYTE* const endOfMatch = op + restSize; + const BYTE* copyFrom = lowPrefix; + while (op < endOfMatch) *op++ = *copyFrom++; + } else { + memcpy(op, lowPrefix, restSize); + op += restSize; + } } + continue; + } + + /* copy match within block */ + cpy = op + length; + if (unlikely(cpy > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LIZv1("13match=%p lowLimit=%p\n", match, lowLimit); goto _output_error; } /* Error : offset outside buffers */ + Lizard_copy8(op, match); + Lizard_copy8(op+8, match+8); + if (length > 16) + Lizard_wildCopy16(op + 16, match + 16, cpy); + op = cpy; + } + + /* last literals */ + length = ctx->literalsEnd - ctx->literalsPtr; + cpy = op + length; + if ((length < 0) || (ctx->literalsPtr+length != iend) || (cpy > oend)) { LIZARD_LOG_DECOMPRESS_LIZv1("14"); goto _output_error; } /* Error : input must be consumed */ + memcpy(op, ctx->literalsPtr, length); + ctx->literalsPtr += length; + op += length; + + /* end of decoding */ + ctx->last_off = last_off; + return (int) (op-dest); /* Nb of output bytes decoded */ + + /* Overflow error detected */ +_output_error: + LIZARD_LOG_DECOMPRESS_LIZv1("_output_error=%d ctx->flagsPtr=%p blockBase=%p\n", (int) (-(ctx->flagsPtr-blockBase))-1, ctx->flagsPtr, blockBase); + LIZARD_LOG_DECOMPRESS_LIZv1("cpy=%p oend=%p ctx->literalsPtr+length[%d]=%p iend=%p\n", cpy, oend, (int)length, ctx->literalsPtr+length, iend); + return (int) (-(ctx->flagsPtr-blockBase))-1; +} diff --git a/contrib/lizard/lib/lizard_decompress_lz4.h b/contrib/lizard/lib/lizard_decompress_lz4.h new file mode 100644 index 00000000000..ffbef1cd976 --- /dev/null +++ b/contrib/lizard/lib/lizard_decompress_lz4.h @@ -0,0 +1,163 @@ +/*! Lizard_decompress_LZ4() : + * This generic decompression function cover all use cases. + * It shall be instantiated several times, using different sets of directives + * Note that it is important this generic function is really inlined, + * in order to remove useless branches during compilation optimization. + */ +FORCE_INLINE int Lizard_decompress_LZ4( + Lizard_dstream_t* ctx, + BYTE* const dest, + int outputSize, /* this value is the max size of Output Buffer. */ + + int partialDecoding, /* full, partial */ + int targetOutputSize, /* only used if partialDecoding==partial */ + int dict, /* noDict, withPrefix64k, usingExtDict */ + const BYTE* const lowPrefix, /* == dest if dict == noDict */ + const BYTE* const dictStart, /* only if dict==usingExtDict */ + const size_t dictSize, /* note : = 0 if noDict */ + int compressionLevel + ) +{ + /* Local Variables */ + int inputSize = (int)(ctx->flagsEnd - ctx->flagsPtr); + const BYTE* const blockBase = ctx->flagsPtr; + const BYTE* const iend = ctx->literalsEnd; + BYTE* op = dest; + BYTE* const oend = op + outputSize; + BYTE* cpy = NULL; + BYTE* oexit = op + targetOutputSize; + const BYTE* const lowLimit = lowPrefix - dictSize; + const BYTE* const dictEnd = (const BYTE*)dictStart + dictSize; + + const int checkOffset = (dictSize < (int)(LIZARD_DICT_SIZE)); + + intptr_t length = 0; + (void)compressionLevel; + + /* Special cases */ + if (unlikely(outputSize==0)) return ((inputSize==1) && (*ctx->flagsPtr==0)) ? 0 : -1; /* Empty output buffer */ + + /* Main Loop : decode sequences */ + while (ctx->flagsPtr < ctx->flagsEnd) { + unsigned token; + const BYTE* match; + size_t offset; + + /* get literal length */ + token = *ctx->flagsPtr++; + if ((length=(token & RUN_MASK_LZ4)) == RUN_MASK_LZ4) { + if (unlikely(ctx->literalsPtr > iend - 5)) { LIZARD_LOG_DECOMPRESS_LZ4("0"); goto _output_error; } + length = *ctx->literalsPtr; + if unlikely(length >= 254) { + if (length == 254) { + length = MEM_readLE16(ctx->literalsPtr+1); + ctx->literalsPtr += 2; + } else { + length = MEM_readLE24(ctx->literalsPtr+1); + ctx->literalsPtr += 3; + } + } + length += RUN_MASK_LZ4; + ctx->literalsPtr++; + if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LZ4("1"); goto _output_error; } /* overflow detection */ + if (unlikely((size_t)(ctx->literalsPtr+length)<(size_t)(ctx->literalsPtr))) { LIZARD_LOG_DECOMPRESS_LZ4("2"); goto _output_error; } /* overflow detection */ + } + + /* copy literals */ + cpy = op + length; + if (unlikely(cpy > oend - WILDCOPYLENGTH || ctx->literalsPtr + length > iend - (2 + WILDCOPYLENGTH))) { LIZARD_LOG_DECOMPRESS_LZ4("offset outside buffers\n"); goto _output_error; } /* Error : offset outside buffers */ + +#if 1 + Lizard_wildCopy16(op, ctx->literalsPtr, cpy); + op = cpy; + ctx->literalsPtr += length; +#else + Lizard_copy8(op, ctx->literalsPtr); + Lizard_copy8(op+8, ctx->literalsPtr+8); + if (length > 16) + Lizard_wildCopy16(op + 16, ctx->literalsPtr + 16, cpy); + op = cpy; + ctx->literalsPtr += length; +#endif + if ((partialDecoding) && (op >= oexit)) return (int) (op-dest); + + /* get offset */ + offset = MEM_readLE16(ctx->literalsPtr); + ctx->literalsPtr += 2; + + match = op - offset; + if ((checkOffset) && (unlikely(match < lowLimit))) { LIZARD_LOG_DECOMPRESS_LZ4("lowPrefix[%p]-dictSize[%d]=lowLimit[%p] match[%p]=op[%p]-offset[%d]\n", lowPrefix, (int)dictSize, lowLimit, match, op, (int)offset); goto _output_error; } /* Error : offset outside buffers */ + + /* get matchlength */ + length = token >> RUN_BITS_LZ4; + if (length == ML_MASK_LZ4) { + if (unlikely(ctx->literalsPtr > iend - 5)) { LIZARD_LOG_DECOMPRESS_LZ4("4"); goto _output_error; } + length = *ctx->literalsPtr; + if unlikely(length >= 254) { + if (length == 254) { + length = MEM_readLE16(ctx->literalsPtr+1); + ctx->literalsPtr += 2; + } else { + length = MEM_readLE24(ctx->literalsPtr+1); + ctx->literalsPtr += 3; + } + } + length += ML_MASK_LZ4; + ctx->literalsPtr++; + if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LZ4("5"); goto _output_error; } /* overflow detection */ + } + length += MINMATCH; + + /* check external dictionary */ + if ((dict==usingExtDict) && (match < lowPrefix)) { + if (unlikely(op + length > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LZ4("6"); goto _output_error; } /* doesn't respect parsing restriction */ + + if (length <= (intptr_t)(lowPrefix - match)) { + /* match can be copied as a single segment from external dictionary */ + memmove(op, dictEnd - (lowPrefix-match), length); + op += length; + } else { + /* match encompass external dictionary and current block */ + size_t const copySize = (size_t)(lowPrefix-match); + size_t const restSize = length - copySize; + memcpy(op, dictEnd - copySize, copySize); + op += copySize; + if (restSize > (size_t)(op-lowPrefix)) { /* overlap copy */ + BYTE* const endOfMatch = op + restSize; + const BYTE* copyFrom = lowPrefix; + while (op < endOfMatch) *op++ = *copyFrom++; + } else { + memcpy(op, lowPrefix, restSize); + op += restSize; + } } + continue; + } + + /* copy match within block */ + cpy = op + length; + if (unlikely(cpy > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LZ4("1match=%p lowLimit=%p\n", match, lowLimit); goto _output_error; } /* Error : offset outside buffers */ + Lizard_copy8(op, match); + Lizard_copy8(op+8, match+8); + if (length > 16) + Lizard_wildCopy16(op + 16, match + 16, cpy); + op = cpy; + if ((partialDecoding) && (op >= oexit)) return (int) (op-dest); + } + + /* last literals */ + length = ctx->literalsEnd - ctx->literalsPtr; + cpy = op + length; + if ((length < 0) || (ctx->literalsPtr+length != iend) || (cpy > oend)) { LIZARD_LOG_DECOMPRESS_LZ4("9"); goto _output_error; } /* Error : input must be consumed */ + memcpy(op, ctx->literalsPtr, length); + ctx->literalsPtr += length; + op += length; + + /* end of decoding */ + return (int) (op-dest); /* Nb of output bytes decoded */ + + /* Overflow error detected */ +_output_error: + LIZARD_LOG_DECOMPRESS_LZ4("_output_error=%d ctx->flagsPtr=%p blockBase=%p\n", (int) (-(ctx->flagsPtr-blockBase))-1, ctx->flagsPtr, blockBase); + LIZARD_LOG_DECOMPRESS_LZ4("cpy=%p oend=%p ctx->literalsPtr+length[%d]=%p iend=%p\n", cpy, oend, (int)length, ctx->literalsPtr+length, iend); + return (int) (-(ctx->flagsPtr-blockBase))-1; +} diff --git a/contrib/lizard/lib/lizard_frame.c b/contrib/lizard/lib/lizard_frame.c new file mode 100644 index 00000000000..f4afbb94e0a --- /dev/null +++ b/contrib/lizard/lib/lizard_frame.c @@ -0,0 +1,1362 @@ +/* +Lizard auto-framing library +Copyright (C) 2011-2016, Yann Collet +Copyright (C) 2016-2017, Przemyslaw Skibinski + +BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +You can contact the author at : +- Lizard source repository : https://github.com/inikep/lizard +*/ + +/* LizardF is a stand-alone API to create Lizard-compressed Frames +* in full conformance with specification v1.5.0 +* All related operations, including memory management, are handled by the library. +* */ + + +/*-************************************ +* Compiler Options +**************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +#endif + + + +/*-************************************ +* Includes +**************************************/ +#include "lizard_frame_static.h" +#include "lizard_compress.h" +#include "lizard_decompress.h" +#include "lizard_common.h" /* LIZARD_DICT_SIZE */ +#define XXH_STATIC_LINKING_ONLY +#include "xxhash/xxhash.h" +#include + + + +/* unoptimized version; solves endianess & alignment issues */ +static U32 LizardF_readLE32 (const void* src) +{ + const BYTE* const srcPtr = (const BYTE*)src; + U32 value32 = srcPtr[0]; + value32 += (srcPtr[1]<<8); + value32 += (srcPtr[2]<<16); + value32 += ((U32)srcPtr[3])<<24; + return value32; +} + +static void LizardF_writeLE32 (BYTE* dstPtr, U32 value32) +{ + dstPtr[0] = (BYTE)value32; + dstPtr[1] = (BYTE)(value32 >> 8); + dstPtr[2] = (BYTE)(value32 >> 16); + dstPtr[3] = (BYTE)(value32 >> 24); +} + +static U64 LizardF_readLE64 (const BYTE* srcPtr) +{ + U64 value64 = srcPtr[0]; + value64 += ((U64)srcPtr[1]<<8); + value64 += ((U64)srcPtr[2]<<16); + value64 += ((U64)srcPtr[3]<<24); + value64 += ((U64)srcPtr[4]<<32); + value64 += ((U64)srcPtr[5]<<40); + value64 += ((U64)srcPtr[6]<<48); + value64 += ((U64)srcPtr[7]<<56); + return value64; +} + +static void LizardF_writeLE64 (BYTE* dstPtr, U64 value64) +{ + dstPtr[0] = (BYTE)value64; + dstPtr[1] = (BYTE)(value64 >> 8); + dstPtr[2] = (BYTE)(value64 >> 16); + dstPtr[3] = (BYTE)(value64 >> 24); + dstPtr[4] = (BYTE)(value64 >> 32); + dstPtr[5] = (BYTE)(value64 >> 40); + dstPtr[6] = (BYTE)(value64 >> 48); + dstPtr[7] = (BYTE)(value64 >> 56); +} + + +/*-************************************ +* Constants +**************************************/ +#define _1BIT 0x01 +#define _2BITS 0x03 +#define _3BITS 0x07 +#define _4BITS 0x0F +#define _8BITS 0xFF + +#define LIZARDF_MAGIC_SKIPPABLE_START 0x184D2A50U +#define LIZARDF_MAGICNUMBER 0x184D2206U +#define LIZARDF_BLOCKUNCOMPRESSED_FLAG 0x80000000U +#define LIZARDF_BLOCKSIZEID_DEFAULT LizardF_max128KB + +static const size_t minFHSize = 7; +static const size_t maxFHSize = 15; +static const size_t BHSize = 4; + + +/*-************************************ +* Structures and local types +**************************************/ +typedef struct LizardF_cctx_s +{ + LizardF_preferences_t prefs; + U32 version; + U32 cStage; + size_t maxBlockSize; + size_t maxBufferSize; + BYTE* tmpBuff; + BYTE* tmpIn; + size_t tmpInSize; + U64 totalInSize; + XXH32_state_t xxh; + Lizard_stream_t* lizardCtxPtr; + U32 lizardCtxLevel; /* 0: unallocated; 1: Lizard_stream_t; */ +} LizardF_cctx_t; + +typedef struct LizardF_dctx_s +{ + LizardF_frameInfo_t frameInfo; + U32 version; + U32 dStage; + U64 frameRemainingSize; + size_t maxBlockSize; + size_t maxBufferSize; + const BYTE* srcExpect; + BYTE* tmpIn; + size_t tmpInSize; + size_t tmpInTarget; + BYTE* tmpOutBuffer; + const BYTE* dict; + size_t dictSize; + BYTE* tmpOut; + size_t tmpOutSize; + size_t tmpOutStart; + XXH32_state_t xxh; + BYTE header[16]; +} LizardF_dctx_t; + + +/*-************************************ +* Error management +**************************************/ +#define LIZARDF_GENERATE_STRING(STRING) #STRING, +static const char* LizardF_errorStrings[] = { LIZARDF_LIST_ERRORS(LIZARDF_GENERATE_STRING) }; + + +unsigned LizardF_isError(LizardF_errorCode_t code) +{ + return (code > (LizardF_errorCode_t)(-LizardF_ERROR_maxCode)); +} + +const char* LizardF_getErrorName(LizardF_errorCode_t code) +{ + static const char* codeError = "Unspecified error code"; + if (LizardF_isError(code)) return LizardF_errorStrings[-(int)(code)]; + return codeError; +} + + +/*-************************************ +* Private functions +**************************************/ +static size_t LizardF_getBlockSize(unsigned blockSizeID) +{ + static const size_t blockSizes[7] = { 128 KB, 256 KB, 1 MB, 4 MB, 16 MB, 64 MB, 256 MB }; + + if (blockSizeID == 0) blockSizeID = LIZARDF_BLOCKSIZEID_DEFAULT; + blockSizeID -= 1; + if (blockSizeID >= 7) return (size_t)-LizardF_ERROR_maxBlockSize_invalid; + + return blockSizes[blockSizeID]; +} + +static LizardF_blockSizeID_t LizardF_optimalBSID(const LizardF_blockSizeID_t requestedBSID, const size_t srcSize) +{ + LizardF_blockSizeID_t proposedBSID = LizardF_max128KB; + size_t maxBlockSize; + + while (requestedBSID > proposedBSID) + { + maxBlockSize = LizardF_getBlockSize(proposedBSID); + if (srcSize <= maxBlockSize) { + return proposedBSID; + } + proposedBSID = (LizardF_blockSizeID_t)((int)proposedBSID + 1); + } + return requestedBSID; +} + + +static BYTE LizardF_headerChecksum (const void* header, size_t length) +{ + U32 xxh = XXH32(header, length, 0); + return (BYTE)(xxh >> 8); +} + + +/*-************************************ +* Simple compression functions +**************************************/ + +size_t LizardF_compressFrameBound(size_t srcSize, const LizardF_preferences_t* preferencesPtr) +{ + LizardF_preferences_t prefs; + size_t headerSize; + size_t streamSize; + + if (preferencesPtr!=NULL) prefs = *preferencesPtr; + else memset(&prefs, 0, sizeof(prefs)); + + prefs.frameInfo.blockSizeID = LizardF_optimalBSID(prefs.frameInfo.blockSizeID, srcSize); + prefs.autoFlush = 1; + + headerSize = maxFHSize; /* header size, including magic number and frame content size*/ + streamSize = LizardF_compressBound(srcSize, &prefs); + + return headerSize + streamSize; +} + + + +/*! LizardF_compressFrame() : +* Compress an entire srcBuffer into a valid Lizard frame, as defined by specification v1.5.0, in a single step. +* The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. +* You can get the minimum value of dstMaxSize by using LizardF_compressFrameBound() +* If this condition is not respected, LizardF_compressFrame() will fail (result is an errorCode) +* The LizardF_preferences_t structure is optional : you can provide NULL as argument. All preferences will then be set to default. +* The result of the function is the number of bytes written into dstBuffer. +* The function outputs an error code if it fails (can be tested using LizardF_isError()) +*/ +size_t LizardF_compressFrame(void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_preferences_t* preferencesPtr) +{ + LizardF_cctx_t cctxI; + LizardF_preferences_t prefs; + LizardF_compressOptions_t options; + LizardF_errorCode_t errorCode; + BYTE* const dstStart = (BYTE*) dstBuffer; + BYTE* dstPtr = dstStart; + BYTE* const dstEnd = dstStart + dstMaxSize; + + memset(&cctxI, 0, sizeof(cctxI)); /* works because no allocation */ + memset(&options, 0, sizeof(options)); + + cctxI.version = LIZARDF_VERSION; + cctxI.maxBufferSize = 5 MB; /* mess with real buffer size to prevent allocation; works because autoflush==1 & stableSrc==1 */ + + if (preferencesPtr!=NULL) + prefs = *preferencesPtr; + else + memset(&prefs, 0, sizeof(prefs)); + if (prefs.frameInfo.contentSize != 0) + prefs.frameInfo.contentSize = (U64)srcSize; /* auto-correct content size if selected (!=0) */ + + prefs.frameInfo.blockSizeID = LizardF_optimalBSID(prefs.frameInfo.blockSizeID, srcSize); + prefs.autoFlush = 1; + if (srcSize <= LizardF_getBlockSize(prefs.frameInfo.blockSizeID)) + prefs.frameInfo.blockMode = LizardF_blockIndependent; /* no need for linked blocks */ + + options.stableSrc = 1; + + if (dstMaxSize < LizardF_compressFrameBound(srcSize, &prefs)) + return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; + + errorCode = LizardF_compressBegin(&cctxI, dstBuffer, dstMaxSize, &prefs); /* write header */ + if (LizardF_isError(errorCode)) goto error; + dstPtr += errorCode; /* header size */ + + errorCode = LizardF_compressUpdate(&cctxI, dstPtr, dstEnd-dstPtr, srcBuffer, srcSize, &options); + if (LizardF_isError(errorCode)) goto error; + dstPtr += errorCode; + + errorCode = LizardF_compressEnd(&cctxI, dstPtr, dstEnd-dstPtr, &options); /* flush last block, and generate suffix */ + if (LizardF_isError(errorCode)) goto error; + dstPtr += errorCode; + + Lizard_freeStream(cctxI.lizardCtxPtr); + FREEMEM(cctxI.tmpBuff); + return (dstPtr - dstStart); +error: + Lizard_freeStream(cctxI.lizardCtxPtr); + FREEMEM(cctxI.tmpBuff); + return errorCode; +} + + +/*-********************************* +* Advanced compression functions +***********************************/ + +/* LizardF_createCompressionContext() : +* The first thing to do is to create a compressionContext object, which will be used in all compression operations. +* This is achieved using LizardF_createCompressionContext(), which takes as argument a version and an LizardF_preferences_t structure. +* The version provided MUST be LIZARDF_VERSION. It is intended to track potential version differences between different binaries. +* The function will provide a pointer to an allocated LizardF_compressionContext_t object. +* If the result LizardF_errorCode_t is not OK_NoError, there was an error during context creation. +* Object can release its memory using LizardF_freeCompressionContext(); +*/ +LizardF_errorCode_t LizardF_createCompressionContext(LizardF_compressionContext_t* LizardF_compressionContextPtr, unsigned version) +{ + LizardF_cctx_t* cctxPtr; + + cctxPtr = (LizardF_cctx_t*)ALLOCATOR(1, sizeof(LizardF_cctx_t)); + if (cctxPtr==NULL) return (LizardF_errorCode_t)(-LizardF_ERROR_allocation_failed); + + cctxPtr->version = version; + cctxPtr->cStage = 0; /* Next stage : write header */ + + *LizardF_compressionContextPtr = (LizardF_compressionContext_t)cctxPtr; + + return LizardF_OK_NoError; +} + + +LizardF_errorCode_t LizardF_freeCompressionContext(LizardF_compressionContext_t LizardF_compressionContext) +{ + LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)LizardF_compressionContext; + + if (cctxPtr != NULL) { /* null pointers can be safely provided to this function, like free() */ + Lizard_freeStream(cctxPtr->lizardCtxPtr); + FREEMEM(cctxPtr->tmpBuff); + FREEMEM(LizardF_compressionContext); + } + + return LizardF_OK_NoError; +} + + +/*! LizardF_compressBegin() : +* will write the frame header into dstBuffer. +* dstBuffer must be large enough to accommodate a header (dstMaxSize). Maximum header size is LizardF_MAXHEADERFRAME_SIZE bytes. +* The result of the function is the number of bytes written into dstBuffer for the header +* or an error code (can be tested using LizardF_isError()) +*/ +size_t LizardF_compressBegin(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const LizardF_preferences_t* preferencesPtr) +{ + LizardF_preferences_t prefNull; + LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; + BYTE* const dstStart = (BYTE*)dstBuffer; + BYTE* dstPtr = dstStart; + BYTE* headerStart; + size_t requiredBuffSize; + + if (dstMaxSize < maxFHSize) return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; + if (cctxPtr->cStage != 0) return (size_t)-LizardF_ERROR_GENERIC; + memset(&prefNull, 0, sizeof(prefNull)); + if (preferencesPtr == NULL) preferencesPtr = &prefNull; + cctxPtr->prefs = *preferencesPtr; + + /* ctx Management */ + if (cctxPtr->lizardCtxLevel == 0) { + cctxPtr->lizardCtxPtr = Lizard_createStream(cctxPtr->prefs.compressionLevel); + cctxPtr->lizardCtxLevel = 1; + } + + /* Buffer Management */ + if (cctxPtr->prefs.frameInfo.blockSizeID == 0) cctxPtr->prefs.frameInfo.blockSizeID = LIZARDF_BLOCKSIZEID_DEFAULT; + cctxPtr->maxBlockSize = LizardF_getBlockSize(cctxPtr->prefs.frameInfo.blockSizeID); + requiredBuffSize = cctxPtr->maxBlockSize + ((cctxPtr->prefs.frameInfo.blockMode == LizardF_blockLinked) * 2 * LIZARD_DICT_SIZE); + + if (preferencesPtr->autoFlush) + requiredBuffSize = (cctxPtr->prefs.frameInfo.blockMode == LizardF_blockLinked) * LIZARD_DICT_SIZE; /* just needs dict */ + + if (cctxPtr->maxBufferSize < requiredBuffSize) { + cctxPtr->maxBufferSize = requiredBuffSize; + FREEMEM(cctxPtr->tmpBuff); + cctxPtr->tmpBuff = (BYTE*)ALLOCATOR(1, requiredBuffSize); + if (cctxPtr->tmpBuff == NULL) { printf("ERROR in LizardF_compressBegin: Cannot allocate %d MB\n", (int)(requiredBuffSize>>20)); return (size_t)-LizardF_ERROR_allocation_failed; } + } + cctxPtr->tmpIn = cctxPtr->tmpBuff; + cctxPtr->tmpInSize = 0; + XXH32_reset(&(cctxPtr->xxh), 0); + cctxPtr->lizardCtxPtr = Lizard_resetStream((Lizard_stream_t*)(cctxPtr->lizardCtxPtr), cctxPtr->prefs.compressionLevel); + if (!cctxPtr->lizardCtxPtr) return (size_t)-LizardF_ERROR_allocation_failed; + + /* Magic Number */ + LizardF_writeLE32(dstPtr, LIZARDF_MAGICNUMBER); + dstPtr += 4; + headerStart = dstPtr; + + /* FLG Byte */ + *dstPtr++ = (BYTE)(((1 & _2BITS) << 6) /* Version('01') */ + + ((cctxPtr->prefs.frameInfo.blockMode & _1BIT ) << 5) /* Block mode */ + + ((cctxPtr->prefs.frameInfo.contentChecksumFlag & _1BIT ) << 2) /* Frame checksum */ + + ((cctxPtr->prefs.frameInfo.contentSize > 0) << 3)); /* Frame content size */ + /* BD Byte */ + *dstPtr++ = (BYTE)((cctxPtr->prefs.frameInfo.blockSizeID & _3BITS) << 4); + /* Optional Frame content size field */ + if (cctxPtr->prefs.frameInfo.contentSize) { + LizardF_writeLE64(dstPtr, cctxPtr->prefs.frameInfo.contentSize); + dstPtr += 8; + cctxPtr->totalInSize = 0; + } + /* CRC Byte */ + *dstPtr = LizardF_headerChecksum(headerStart, dstPtr - headerStart); + dstPtr++; + + cctxPtr->cStage = 1; /* header written, now request input data block */ + + return (dstPtr - dstStart); +} + + +/* LizardF_compressBound() : gives the size of Dst buffer given a srcSize to handle worst case situations. +* The LizardF_frameInfo_t structure is optional : +* you can provide NULL as argument, preferences will then be set to cover worst case situations. +* */ +size_t LizardF_compressBound(size_t srcSize, const LizardF_preferences_t* preferencesPtr) +{ + LizardF_preferences_t prefsNull; + memset(&prefsNull, 0, sizeof(prefsNull)); + prefsNull.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; /* worst case */ + { const LizardF_preferences_t* prefsPtr = (preferencesPtr==NULL) ? &prefsNull : preferencesPtr; + LizardF_blockSizeID_t bid = prefsPtr->frameInfo.blockSizeID; + size_t blockSize = LizardF_getBlockSize(bid); + unsigned nbBlocks = (unsigned)(srcSize / blockSize) + 1; + size_t lastBlockSize = prefsPtr->autoFlush ? srcSize % blockSize : blockSize; + size_t blockInfo = 4; /* default, without block CRC option */ + size_t frameEnd = 4 + (prefsPtr->frameInfo.contentChecksumFlag*4); + + return (blockInfo * nbBlocks) + (blockSize * (nbBlocks-1)) + lastBlockSize + frameEnd;; + } +} + + +typedef int (*compressFunc_t)(void* ctx, const char* src, char* dst, int srcSize, int dstSize, int level); + +static size_t LizardF_compressBlock(void* dst, const void* src, size_t srcSize, compressFunc_t compress, void* lizardctx, int level) +{ + /* compress one block */ + BYTE* cSizePtr = (BYTE*)dst; + U32 cSize; + cSize = (U32)compress(lizardctx, (const char*)src, (char*)(cSizePtr+4), (int)(srcSize), (int)(srcSize-1), level); + LizardF_writeLE32(cSizePtr, cSize); + if (cSize == 0) { /* compression failed */ + cSize = (U32)srcSize; + LizardF_writeLE32(cSizePtr, cSize + LIZARDF_BLOCKUNCOMPRESSED_FLAG); + memcpy(cSizePtr+4, src, srcSize); + } + return cSize + 4; +} + + + +static int LizardF_localLizard_compress_continue(void* ctx, const char* src, char* dst, int srcSize, int dstSize, int level) +{ + (void)level; + return Lizard_compress_continue((Lizard_stream_t*)ctx, src, dst, srcSize, dstSize); +} + +static compressFunc_t LizardF_selectCompression(LizardF_blockMode_t blockMode) +{ + if (blockMode == LizardF_blockIndependent) return Lizard_compress_extState; + return LizardF_localLizard_compress_continue; +} + +static int LizardF_localSaveDict(LizardF_cctx_t* cctxPtr) +{ + return Lizard_saveDict ((Lizard_stream_t*)(cctxPtr->lizardCtxPtr), (char*)(cctxPtr->tmpBuff), LIZARD_DICT_SIZE); +} + +typedef enum { notDone, fromTmpBuffer, fromSrcBuffer } LizardF_lastBlockStatus; + +/*! LizardF_compressUpdate() : +* LizardF_compressUpdate() can be called repetitively to compress as much data as necessary. +* The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. +* If this condition is not respected, LizardF_compress() will fail (result is an errorCode) +* You can get the minimum value of dstMaxSize by using LizardF_compressBound() +* The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. +* The result of the function is the number of bytes written into dstBuffer : it can be zero, meaning input data was just buffered. +* The function outputs an error code if it fails (can be tested using LizardF_isError()) +*/ +size_t LizardF_compressUpdate(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_compressOptions_t* compressOptionsPtr) +{ + LizardF_compressOptions_t cOptionsNull; + LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; + size_t blockSize = cctxPtr->maxBlockSize; + const BYTE* srcPtr = (const BYTE*)srcBuffer; + const BYTE* const srcEnd = srcPtr + srcSize; + BYTE* const dstStart = (BYTE*)dstBuffer; + BYTE* dstPtr = dstStart; + LizardF_lastBlockStatus lastBlockCompressed = notDone; + compressFunc_t compress; + + + if (cctxPtr->cStage != 1) return (size_t)-LizardF_ERROR_GENERIC; + if (dstMaxSize < LizardF_compressBound(srcSize, &(cctxPtr->prefs))) return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; + memset(&cOptionsNull, 0, sizeof(cOptionsNull)); + if (compressOptionsPtr == NULL) compressOptionsPtr = &cOptionsNull; + + /* select compression function */ + compress = LizardF_selectCompression(cctxPtr->prefs.frameInfo.blockMode); + + /* complete tmp buffer */ + if (cctxPtr->tmpInSize > 0) { /* some data already within tmp buffer */ + size_t sizeToCopy = blockSize - cctxPtr->tmpInSize; + if (sizeToCopy > srcSize) { + /* add src to tmpIn buffer */ + memcpy(cctxPtr->tmpIn + cctxPtr->tmpInSize, srcBuffer, srcSize); + srcPtr = srcEnd; + cctxPtr->tmpInSize += srcSize; + /* still needs some CRC */ + } else { + /* complete tmpIn block and then compress it */ + lastBlockCompressed = fromTmpBuffer; + memcpy(cctxPtr->tmpIn + cctxPtr->tmpInSize, srcBuffer, sizeToCopy); + srcPtr += sizeToCopy; + + dstPtr += LizardF_compressBlock(dstPtr, cctxPtr->tmpIn, blockSize, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); + + if (cctxPtr->prefs.frameInfo.blockMode==LizardF_blockLinked) cctxPtr->tmpIn += blockSize; + cctxPtr->tmpInSize = 0; + } + } + + while ((size_t)(srcEnd - srcPtr) >= blockSize) { + /* compress full block */ + lastBlockCompressed = fromSrcBuffer; + dstPtr += LizardF_compressBlock(dstPtr, srcPtr, blockSize, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); + srcPtr += blockSize; + } + + if ((cctxPtr->prefs.autoFlush) && (srcPtr < srcEnd)) { + /* compress remaining input < blockSize */ + lastBlockCompressed = fromSrcBuffer; + dstPtr += LizardF_compressBlock(dstPtr, srcPtr, srcEnd - srcPtr, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); + srcPtr = srcEnd; + } + + /* preserve dictionary if necessary */ + if ((cctxPtr->prefs.frameInfo.blockMode==LizardF_blockLinked) && (lastBlockCompressed==fromSrcBuffer)) { + if (compressOptionsPtr->stableSrc) { + cctxPtr->tmpIn = cctxPtr->tmpBuff; + } else { + int realDictSize = LizardF_localSaveDict(cctxPtr); + if (realDictSize==0) return (size_t)-LizardF_ERROR_GENERIC; + cctxPtr->tmpIn = cctxPtr->tmpBuff + realDictSize; + } + } + + /* keep tmpIn within limits */ + if ((cctxPtr->tmpIn + blockSize) > (cctxPtr->tmpBuff + cctxPtr->maxBufferSize) /* necessarily LizardF_blockLinked && lastBlockCompressed==fromTmpBuffer */ + && !(cctxPtr->prefs.autoFlush)) + { + int realDictSize = LizardF_localSaveDict(cctxPtr); + cctxPtr->tmpIn = cctxPtr->tmpBuff + realDictSize; + } + + /* some input data left, necessarily < blockSize */ + if (srcPtr < srcEnd) { + /* fill tmp buffer */ + size_t sizeToCopy = srcEnd - srcPtr; + memcpy(cctxPtr->tmpIn, srcPtr, sizeToCopy); + cctxPtr->tmpInSize = sizeToCopy; + } + + if (cctxPtr->prefs.frameInfo.contentChecksumFlag == LizardF_contentChecksumEnabled) + XXH32_update(&(cctxPtr->xxh), srcBuffer, srcSize); + + cctxPtr->totalInSize += srcSize; + return dstPtr - dstStart; +} + + +/*! LizardF_flush() : +* Should you need to create compressed data immediately, without waiting for a block to be filled, +* you can call Lizard_flush(), which will immediately compress any remaining data stored within compressionContext. +* The result of the function is the number of bytes written into dstBuffer +* (it can be zero, this means there was no data left within compressionContext) +* The function outputs an error code if it fails (can be tested using LizardF_isError()) +* The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. +*/ +size_t LizardF_flush(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* compressOptionsPtr) +{ + LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; + BYTE* const dstStart = (BYTE*)dstBuffer; + BYTE* dstPtr = dstStart; + compressFunc_t compress; + + + if (cctxPtr->tmpInSize == 0) return 0; /* nothing to flush */ + if (cctxPtr->cStage != 1) return (size_t)-LizardF_ERROR_GENERIC; + if (dstMaxSize < (cctxPtr->tmpInSize + 8)) return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; /* +8 : block header(4) + block checksum(4) */ + (void)compressOptionsPtr; /* not yet useful */ + + /* select compression function */ + compress = LizardF_selectCompression(cctxPtr->prefs.frameInfo.blockMode); + + /* compress tmp buffer */ + dstPtr += LizardF_compressBlock(dstPtr, cctxPtr->tmpIn, cctxPtr->tmpInSize, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); + if (cctxPtr->prefs.frameInfo.blockMode==LizardF_blockLinked) cctxPtr->tmpIn += cctxPtr->tmpInSize; + cctxPtr->tmpInSize = 0; + + /* keep tmpIn within limits */ + if ((cctxPtr->tmpIn + cctxPtr->maxBlockSize) > (cctxPtr->tmpBuff + cctxPtr->maxBufferSize)) { /* necessarily LizardF_blockLinked */ + int realDictSize = LizardF_localSaveDict(cctxPtr); + cctxPtr->tmpIn = cctxPtr->tmpBuff + realDictSize; + } + + return dstPtr - dstStart; +} + + +/*! LizardF_compressEnd() : +* When you want to properly finish the compressed frame, just call LizardF_compressEnd(). +* It will flush whatever data remained within compressionContext (like Lizard_flush()) +* but also properly finalize the frame, with an endMark and a checksum. +* The result of the function is the number of bytes written into dstBuffer (necessarily >= 4 (endMark size)) +* The function outputs an error code if it fails (can be tested using LizardF_isError()) +* The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. +* compressionContext can then be used again, starting with LizardF_compressBegin(). The preferences will remain the same. +*/ +size_t LizardF_compressEnd(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* compressOptionsPtr) +{ + LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; + BYTE* const dstStart = (BYTE*)dstBuffer; + BYTE* dstPtr = dstStart; + size_t errorCode; + + errorCode = LizardF_flush(compressionContext, dstBuffer, dstMaxSize, compressOptionsPtr); + if (LizardF_isError(errorCode)) return errorCode; + dstPtr += errorCode; + + LizardF_writeLE32(dstPtr, 0); + dstPtr+=4; /* endMark */ + + if (cctxPtr->prefs.frameInfo.contentChecksumFlag == LizardF_contentChecksumEnabled) { + U32 xxh = XXH32_digest(&(cctxPtr->xxh)); + LizardF_writeLE32(dstPtr, xxh); + dstPtr+=4; /* content Checksum */ + } + + cctxPtr->cStage = 0; /* state is now re-usable (with identical preferences) */ + cctxPtr->maxBufferSize = 0; /* reuse HC context */ + + if (cctxPtr->prefs.frameInfo.contentSize) { + if (cctxPtr->prefs.frameInfo.contentSize != cctxPtr->totalInSize) + return (size_t)-LizardF_ERROR_frameSize_wrong; + } + + return dstPtr - dstStart; +} + + +/*-*************************************************** +* Frame Decompression +*****************************************************/ + +/* Resource management */ + +/*! LizardF_createDecompressionContext() : +* Create a decompressionContext object, which will track all decompression operations. +* Provides a pointer to a fully allocated and initialized LizardF_decompressionContext object. +* Object can later be released using LizardF_freeDecompressionContext(). +* @return : if != 0, there was an error during context creation. +*/ +LizardF_errorCode_t LizardF_createDecompressionContext(LizardF_decompressionContext_t* LizardF_decompressionContextPtr, unsigned versionNumber) +{ + LizardF_dctx_t* const dctxPtr = (LizardF_dctx_t*)ALLOCATOR(1, sizeof(LizardF_dctx_t)); + if (dctxPtr==NULL) return (LizardF_errorCode_t)-LizardF_ERROR_GENERIC; + + dctxPtr->version = versionNumber; + *LizardF_decompressionContextPtr = (LizardF_decompressionContext_t)dctxPtr; + return LizardF_OK_NoError; +} + +LizardF_errorCode_t LizardF_freeDecompressionContext(LizardF_decompressionContext_t LizardF_decompressionContext) +{ + LizardF_errorCode_t result = LizardF_OK_NoError; + LizardF_dctx_t* const dctxPtr = (LizardF_dctx_t*)LizardF_decompressionContext; + if (dctxPtr != NULL) { /* can accept NULL input, like free() */ + result = (LizardF_errorCode_t)dctxPtr->dStage; + FREEMEM(dctxPtr->tmpIn); + FREEMEM(dctxPtr->tmpOutBuffer); + FREEMEM(dctxPtr); + } + return result; +} + + +/* ******************************************************************** */ +/* ********************* Decompression ******************************** */ +/* ******************************************************************** */ + +typedef enum { dstage_getHeader=0, dstage_storeHeader, + dstage_getCBlockSize, dstage_storeCBlockSize, + dstage_copyDirect, + dstage_getCBlock, dstage_storeCBlock, + dstage_decodeCBlock, dstage_decodeCBlock_intoDst, + dstage_decodeCBlock_intoTmp, dstage_flushOut, + dstage_getSuffix, dstage_storeSuffix, + dstage_getSFrameSize, dstage_storeSFrameSize, + dstage_skipSkippable +} dStage_t; + + +/*! LizardF_headerSize() : +* @return : size of frame header +* or an error code, which can be tested using LizardF_isError() +*/ +static size_t LizardF_headerSize(const void* src, size_t srcSize) +{ + /* minimal srcSize to determine header size */ + if (srcSize < 5) return (size_t)-LizardF_ERROR_frameHeader_incomplete; + + /* special case : skippable frames */ + if ((LizardF_readLE32(src) & 0xFFFFFFF0U) == LIZARDF_MAGIC_SKIPPABLE_START) return 8; + + /* control magic number */ + if (LizardF_readLE32(src) != LIZARDF_MAGICNUMBER) return (size_t)-LizardF_ERROR_frameType_unknown; + + /* Frame Header Size */ + { BYTE const FLG = ((const BYTE*)src)[4]; + U32 const contentSizeFlag = (FLG>>3) & _1BIT; + return contentSizeFlag ? maxFHSize : minFHSize; + } +} + + +/*! LizardF_decodeHeader() : + input : `srcVoidPtr` points at the **beginning of the frame** + output : set internal values of dctx, such as + dctxPtr->frameInfo and dctxPtr->dStage. + Also allocates internal buffers. + @return : nb Bytes read from srcVoidPtr (necessarily <= srcSize) + or an error code (testable with LizardF_isError()) +*/ +static size_t LizardF_decodeHeader(LizardF_dctx_t* dctxPtr, const void* srcVoidPtr, size_t srcSize) +{ + BYTE FLG, BD, HC; + unsigned version, blockMode, blockChecksumFlag, contentSizeFlag, contentChecksumFlag, blockSizeID; + size_t bufferNeeded, currentBlockSize; + size_t frameHeaderSize; + const BYTE* srcPtr = (const BYTE*)srcVoidPtr; + + /* need to decode header to get frameInfo */ + if (srcSize < minFHSize) return (size_t)-LizardF_ERROR_frameHeader_incomplete; /* minimal frame header size */ + memset(&(dctxPtr->frameInfo), 0, sizeof(dctxPtr->frameInfo)); + + /* special case : skippable frames */ + if ((LizardF_readLE32(srcPtr) & 0xFFFFFFF0U) == LIZARDF_MAGIC_SKIPPABLE_START) { + dctxPtr->frameInfo.frameType = LizardF_skippableFrame; + if (srcVoidPtr == (void*)(dctxPtr->header)) { + dctxPtr->tmpInSize = srcSize; + dctxPtr->tmpInTarget = 8; + dctxPtr->dStage = dstage_storeSFrameSize; + return srcSize; + } else { + dctxPtr->dStage = dstage_getSFrameSize; + return 4; + } + } + + /* control magic number */ + if (LizardF_readLE32(srcPtr) != LIZARDF_MAGICNUMBER) return (size_t)-LizardF_ERROR_frameType_unknown; + dctxPtr->frameInfo.frameType = LizardF_frame; + + /* Flags */ + FLG = srcPtr[4]; + version = (FLG>>6) & _2BITS; + blockMode = (FLG>>5) & _1BIT; + blockChecksumFlag = (FLG>>4) & _1BIT; + contentSizeFlag = (FLG>>3) & _1BIT; + contentChecksumFlag = (FLG>>2) & _1BIT; + + /* Frame Header Size */ + frameHeaderSize = contentSizeFlag ? maxFHSize : minFHSize; + + if (srcSize < frameHeaderSize) { + /* not enough input to fully decode frame header */ + if (srcPtr != dctxPtr->header) + memcpy(dctxPtr->header, srcPtr, srcSize); + dctxPtr->tmpInSize = srcSize; + dctxPtr->tmpInTarget = frameHeaderSize; + dctxPtr->dStage = dstage_storeHeader; + return srcSize; + } + + BD = srcPtr[5]; + blockSizeID = (BD>>4) & _3BITS; + + /* validate */ + if (version != 1) return (size_t)-LizardF_ERROR_headerVersion_wrong; /* Version Number, only supported value */ + if (blockChecksumFlag != 0) return (size_t)-LizardF_ERROR_blockChecksum_unsupported; /* Not supported for the time being */ + if (((FLG>>0)&_2BITS) != 0) return (size_t)-LizardF_ERROR_reservedFlag_set; /* Reserved bits */ + if (((BD>>7)&_1BIT) != 0) return (size_t)-LizardF_ERROR_reservedFlag_set; /* Reserved bit */ + if (blockSizeID < 1) return (size_t)-LizardF_ERROR_maxBlockSize_invalid; /* 1-7 only supported values for the time being */ + if (((BD>>0)&_4BITS) != 0) return (size_t)-LizardF_ERROR_reservedFlag_set; /* Reserved bits */ + + /* check */ + HC = LizardF_headerChecksum(srcPtr+4, frameHeaderSize-5); + if (HC != srcPtr[frameHeaderSize-1]) return (size_t)-LizardF_ERROR_headerChecksum_invalid; /* Bad header checksum error */ + + /* save */ + dctxPtr->frameInfo.blockMode = (LizardF_blockMode_t)blockMode; + dctxPtr->frameInfo.contentChecksumFlag = (LizardF_contentChecksum_t)contentChecksumFlag; + dctxPtr->frameInfo.blockSizeID = (LizardF_blockSizeID_t)blockSizeID; + currentBlockSize = dctxPtr->maxBlockSize; + dctxPtr->maxBlockSize = LizardF_getBlockSize(blockSizeID); + if (contentSizeFlag) + dctxPtr->frameRemainingSize = dctxPtr->frameInfo.contentSize = LizardF_readLE64(srcPtr+6); + + /* init */ + if (contentChecksumFlag) XXH32_reset(&(dctxPtr->xxh), 0); + + /* alloc */ + bufferNeeded = dctxPtr->maxBlockSize + ((dctxPtr->frameInfo.blockMode==LizardF_blockLinked) * 2 * LIZARD_DICT_SIZE); + if (bufferNeeded > dctxPtr->maxBufferSize || dctxPtr->maxBlockSize > currentBlockSize) { /* tmp buffers too small */ + FREEMEM(dctxPtr->tmpIn); + FREEMEM(dctxPtr->tmpOutBuffer); + dctxPtr->maxBufferSize = 0; + dctxPtr->tmpIn = (BYTE*)ALLOCATOR(1, dctxPtr->maxBlockSize); + if (dctxPtr->tmpIn == NULL) return (size_t)-LizardF_ERROR_GENERIC; + dctxPtr->tmpOutBuffer= (BYTE*)ALLOCATOR(1, bufferNeeded); + if (dctxPtr->tmpOutBuffer== NULL) return (size_t)-LizardF_ERROR_GENERIC; + dctxPtr->maxBufferSize = bufferNeeded; + } + dctxPtr->tmpInSize = 0; + dctxPtr->tmpInTarget = 0; + dctxPtr->dict = dctxPtr->tmpOutBuffer; + dctxPtr->dictSize = 0; + dctxPtr->tmpOut = dctxPtr->tmpOutBuffer; + dctxPtr->tmpOutStart = 0; + dctxPtr->tmpOutSize = 0; + + dctxPtr->dStage = dstage_getCBlockSize; + + return frameHeaderSize; +} + + +/*! LizardF_getFrameInfo() : +* Decodes frame header information, such as blockSize. +* It is optional : you could start by calling directly LizardF_decompress() instead. +* The objective is to extract header information without starting decompression, typically for allocation purposes. +* LizardF_getFrameInfo() can also be used *after* starting decompression, on a valid LizardF_decompressionContext_t. +* The number of bytes read from srcBuffer will be provided within *srcSizePtr (necessarily <= original value). +* You are expected to resume decompression from where it stopped (srcBuffer + *srcSizePtr) +* @return : hint of the better `srcSize` to use for next call to LizardF_decompress, +* or an error code which can be tested using LizardF_isError(). +*/ +LizardF_errorCode_t LizardF_getFrameInfo(LizardF_decompressionContext_t dCtx, LizardF_frameInfo_t* frameInfoPtr, + const void* srcBuffer, size_t* srcSizePtr) +{ + LizardF_dctx_t* dctxPtr = (LizardF_dctx_t*)dCtx; + + if (dctxPtr->dStage > dstage_storeHeader) { /* note : requires dstage_* header related to be at beginning of enum */ + /* frameInfo already decoded */ + size_t o=0, i=0; + *srcSizePtr = 0; + *frameInfoPtr = dctxPtr->frameInfo; + return LizardF_decompress(dCtx, NULL, &o, NULL, &i, NULL); /* returns : recommended nb of bytes for LizardF_decompress() */ + } else { + size_t nextSrcSize, o=0; + size_t const hSize = LizardF_headerSize(srcBuffer, *srcSizePtr); + if (LizardF_isError(hSize)) { *srcSizePtr=0; return hSize; } + if (*srcSizePtr < hSize) { *srcSizePtr=0; return (size_t)-LizardF_ERROR_frameHeader_incomplete; } + + *srcSizePtr = hSize; + nextSrcSize = LizardF_decompress(dCtx, NULL, &o, srcBuffer, srcSizePtr, NULL); + if (dctxPtr->dStage <= dstage_storeHeader) return (size_t)-LizardF_ERROR_frameHeader_incomplete; /* should not happen, already checked */ + *frameInfoPtr = dctxPtr->frameInfo; + return nextSrcSize; + } +} + + +/* trivial redirector, for common prototype */ +static int LizardF_decompress_safe (const char* source, char* dest, int compressedSize, int maxDecompressedSize, const char* dictStart, int dictSize) +{ + (void)dictStart; (void)dictSize; + return Lizard_decompress_safe (source, dest, compressedSize, maxDecompressedSize); +} + + +static void LizardF_updateDict(LizardF_dctx_t* dctxPtr, const BYTE* dstPtr, size_t dstSize, const BYTE* dstPtr0, unsigned withinTmp) +{ + if (dctxPtr->dictSize==0) + dctxPtr->dict = (const BYTE*)dstPtr; /* priority to dictionary continuity */ + + if (dctxPtr->dict + dctxPtr->dictSize == dstPtr) { /* dictionary continuity */ + dctxPtr->dictSize += dstSize; + return; + } + + if (dstPtr - dstPtr0 + dstSize >= LIZARD_DICT_SIZE) { /* dstBuffer large enough to become dictionary */ + dctxPtr->dict = (const BYTE*)dstPtr0; + dctxPtr->dictSize = dstPtr - dstPtr0 + dstSize; + return; + } + + if ((withinTmp) && (dctxPtr->dict == dctxPtr->tmpOutBuffer)) { + /* assumption : dctxPtr->dict + dctxPtr->dictSize == dctxPtr->tmpOut + dctxPtr->tmpOutStart */ + dctxPtr->dictSize += dstSize; + return; + } + + if (withinTmp) { /* copy relevant dict portion in front of tmpOut within tmpOutBuffer */ + size_t preserveSize = dctxPtr->tmpOut - dctxPtr->tmpOutBuffer; + size_t copySize = LIZARD_DICT_SIZE - dctxPtr->tmpOutSize; + const BYTE* oldDictEnd = dctxPtr->dict + dctxPtr->dictSize - dctxPtr->tmpOutStart; + if (dctxPtr->tmpOutSize > LIZARD_DICT_SIZE) copySize = 0; + if (copySize > preserveSize) copySize = preserveSize; + + memcpy(dctxPtr->tmpOutBuffer + preserveSize - copySize, oldDictEnd - copySize, copySize); + + dctxPtr->dict = dctxPtr->tmpOutBuffer; + dctxPtr->dictSize = preserveSize + dctxPtr->tmpOutStart + dstSize; + return; + } + + if (dctxPtr->dict == dctxPtr->tmpOutBuffer) { /* copy dst into tmp to complete dict */ + if (dctxPtr->dictSize + dstSize > dctxPtr->maxBufferSize) { /* tmp buffer not large enough */ + size_t preserveSize = LIZARD_DICT_SIZE - dstSize; /* note : dstSize < LIZARD_DICT_SIZE */ + memcpy(dctxPtr->tmpOutBuffer, dctxPtr->dict + dctxPtr->dictSize - preserveSize, preserveSize); + dctxPtr->dictSize = preserveSize; + } + memcpy(dctxPtr->tmpOutBuffer + dctxPtr->dictSize, dstPtr, dstSize); + dctxPtr->dictSize += dstSize; + return; + } + + /* join dict & dest into tmp */ + { size_t preserveSize = LIZARD_DICT_SIZE - dstSize; /* note : dstSize < LIZARD_DICT_SIZE */ + if (preserveSize > dctxPtr->dictSize) preserveSize = dctxPtr->dictSize; + memcpy(dctxPtr->tmpOutBuffer, dctxPtr->dict + dctxPtr->dictSize - preserveSize, preserveSize); + memcpy(dctxPtr->tmpOutBuffer + preserveSize, dstPtr, dstSize); + dctxPtr->dict = dctxPtr->tmpOutBuffer; + dctxPtr->dictSize = preserveSize + dstSize; + } +} + + + +/*! LizardF_decompress() : +* Call this function repetitively to regenerate data compressed within srcBuffer. +* The function will attempt to decode *srcSizePtr from srcBuffer, into dstBuffer of maximum size *dstSizePtr. +* +* The number of bytes regenerated into dstBuffer will be provided within *dstSizePtr (necessarily <= original value). +* +* The number of bytes effectively read from srcBuffer will be provided within *srcSizePtr (necessarily <= original value). +* If the number of bytes read is < number of bytes provided, then the decompression operation is not complete. +* You will have to call it again, continuing from where it stopped. +* +* The function result is an hint of the better srcSize to use for next call to LizardF_decompress. +* Basically, it's the size of the current (or remaining) compressed block + header of next block. +* Respecting the hint provides some boost to performance, since it allows less buffer shuffling. +* Note that this is just a hint, you can always provide any srcSize you want. +* When a frame is fully decoded, the function result will be 0. +* If decompression failed, function result is an error code which can be tested using LizardF_isError(). +*/ +size_t LizardF_decompress(LizardF_decompressionContext_t decompressionContext, + void* dstBuffer, size_t* dstSizePtr, + const void* srcBuffer, size_t* srcSizePtr, + const LizardF_decompressOptions_t* decompressOptionsPtr) +{ + LizardF_dctx_t* dctxPtr = (LizardF_dctx_t*)decompressionContext; + LizardF_decompressOptions_t optionsNull; + const BYTE* const srcStart = (const BYTE*)srcBuffer; + const BYTE* const srcEnd = srcStart + *srcSizePtr; + const BYTE* srcPtr = srcStart; + BYTE* const dstStart = (BYTE*)dstBuffer; + BYTE* const dstEnd = dstStart + *dstSizePtr; + BYTE* dstPtr = dstStart; + const BYTE* selectedIn = NULL; + unsigned doAnotherStage = 1; + size_t nextSrcSizeHint = 1; + + + memset(&optionsNull, 0, sizeof(optionsNull)); + if (decompressOptionsPtr==NULL) decompressOptionsPtr = &optionsNull; + *srcSizePtr = 0; + *dstSizePtr = 0; + + /* expect to continue decoding src buffer where it left previously */ + if (dctxPtr->srcExpect != NULL) { + if (srcStart != dctxPtr->srcExpect) return (size_t)-LizardF_ERROR_srcPtr_wrong; + } + + /* programmed as a state machine */ + + while (doAnotherStage) { + + switch(dctxPtr->dStage) + { + + case dstage_getHeader: + if ((size_t)(srcEnd-srcPtr) >= maxFHSize) { /* enough to decode - shortcut */ + LizardF_errorCode_t const hSize = LizardF_decodeHeader(dctxPtr, srcPtr, srcEnd-srcPtr); + if (LizardF_isError(hSize)) return hSize; + srcPtr += hSize; + break; + } + dctxPtr->tmpInSize = 0; + dctxPtr->tmpInTarget = minFHSize; /* minimum to attempt decode */ + dctxPtr->dStage = dstage_storeHeader; + /* fallthrough */ + + case dstage_storeHeader: + { size_t sizeToCopy = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; + if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; + memcpy(dctxPtr->header + dctxPtr->tmpInSize, srcPtr, sizeToCopy); + dctxPtr->tmpInSize += sizeToCopy; + srcPtr += sizeToCopy; + if (dctxPtr->tmpInSize < dctxPtr->tmpInTarget) { + nextSrcSizeHint = (dctxPtr->tmpInTarget - dctxPtr->tmpInSize) + BHSize; /* rest of header + nextBlockHeader */ + doAnotherStage = 0; /* not enough src data, ask for some more */ + break; + } + { LizardF_errorCode_t const hSize = LizardF_decodeHeader(dctxPtr, dctxPtr->header, dctxPtr->tmpInTarget); + if (LizardF_isError(hSize)) return hSize; + } + break; + } + + case dstage_getCBlockSize: + if ((size_t)(srcEnd - srcPtr) >= BHSize) { + selectedIn = srcPtr; + srcPtr += BHSize; + } else { + /* not enough input to read cBlockSize field */ + dctxPtr->tmpInSize = 0; + dctxPtr->dStage = dstage_storeCBlockSize; + } + + if (dctxPtr->dStage == dstage_storeCBlockSize) /* can be skipped */ + case dstage_storeCBlockSize: + { + size_t sizeToCopy = BHSize - dctxPtr->tmpInSize; + if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; + memcpy(dctxPtr->tmpIn + dctxPtr->tmpInSize, srcPtr, sizeToCopy); + srcPtr += sizeToCopy; + dctxPtr->tmpInSize += sizeToCopy; + if (dctxPtr->tmpInSize < BHSize) { /* not enough input to get full cBlockSize; wait for more */ + nextSrcSizeHint = BHSize - dctxPtr->tmpInSize; + doAnotherStage = 0; + break; + } + selectedIn = dctxPtr->tmpIn; + } + + /* case dstage_decodeCBlockSize: */ /* no more direct access, to prevent scan-build warning */ + { size_t const nextCBlockSize = LizardF_readLE32(selectedIn) & 0x7FFFFFFFU; + if (nextCBlockSize==0) { /* frameEnd signal, no more CBlock */ + dctxPtr->dStage = dstage_getSuffix; + break; + } + if (nextCBlockSize > dctxPtr->maxBlockSize) return (size_t)-LizardF_ERROR_GENERIC; /* invalid cBlockSize */ + dctxPtr->tmpInTarget = nextCBlockSize; + if (LizardF_readLE32(selectedIn) & LIZARDF_BLOCKUNCOMPRESSED_FLAG) { + dctxPtr->dStage = dstage_copyDirect; + break; + } + dctxPtr->dStage = dstage_getCBlock; + if (dstPtr==dstEnd) { + nextSrcSizeHint = nextCBlockSize + BHSize; + doAnotherStage = 0; + } + break; + } + + case dstage_copyDirect: /* uncompressed block */ + { size_t sizeToCopy = dctxPtr->tmpInTarget; + if ((size_t)(srcEnd-srcPtr) < sizeToCopy) sizeToCopy = srcEnd - srcPtr; /* not enough input to read full block */ + if ((size_t)(dstEnd-dstPtr) < sizeToCopy) sizeToCopy = dstEnd - dstPtr; + memcpy(dstPtr, srcPtr, sizeToCopy); + if (dctxPtr->frameInfo.contentChecksumFlag) XXH32_update(&(dctxPtr->xxh), srcPtr, sizeToCopy); + if (dctxPtr->frameInfo.contentSize) dctxPtr->frameRemainingSize -= sizeToCopy; + + /* dictionary management */ + if (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) + LizardF_updateDict(dctxPtr, dstPtr, sizeToCopy, dstStart, 0); + + srcPtr += sizeToCopy; + dstPtr += sizeToCopy; + if (sizeToCopy == dctxPtr->tmpInTarget) { /* all copied */ + dctxPtr->dStage = dstage_getCBlockSize; + break; + } + dctxPtr->tmpInTarget -= sizeToCopy; /* still need to copy more */ + nextSrcSizeHint = dctxPtr->tmpInTarget + BHSize; + doAnotherStage = 0; + break; + } + + case dstage_getCBlock: /* entry from dstage_decodeCBlockSize */ + if ((size_t)(srcEnd-srcPtr) < dctxPtr->tmpInTarget) { + dctxPtr->tmpInSize = 0; + dctxPtr->dStage = dstage_storeCBlock; + break; + } + selectedIn = srcPtr; + srcPtr += dctxPtr->tmpInTarget; + dctxPtr->dStage = dstage_decodeCBlock; + break; + + case dstage_storeCBlock: + { size_t sizeToCopy = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; + if (sizeToCopy > (size_t)(srcEnd-srcPtr)) sizeToCopy = srcEnd-srcPtr; + memcpy(dctxPtr->tmpIn + dctxPtr->tmpInSize, srcPtr, sizeToCopy); + dctxPtr->tmpInSize += sizeToCopy; + srcPtr += sizeToCopy; + if (dctxPtr->tmpInSize < dctxPtr->tmpInTarget) { /* need more input */ + nextSrcSizeHint = (dctxPtr->tmpInTarget - dctxPtr->tmpInSize) + BHSize; + doAnotherStage=0; + break; + } + selectedIn = dctxPtr->tmpIn; + dctxPtr->dStage = dstage_decodeCBlock; + } + /* fallthrough */ + + case dstage_decodeCBlock: + if ((size_t)(dstEnd-dstPtr) < dctxPtr->maxBlockSize) /* not enough place into dst : decode into tmpOut */ + dctxPtr->dStage = dstage_decodeCBlock_intoTmp; + else + dctxPtr->dStage = dstage_decodeCBlock_intoDst; + break; + + case dstage_decodeCBlock_intoDst: + { int (*decoder)(const char*, char*, int, int, const char*, int); + int decodedSize; + + if (dctxPtr->frameInfo.blockMode == LizardF_blockLinked) + decoder = Lizard_decompress_safe_usingDict; + else + decoder = LizardF_decompress_safe; + + decodedSize = decoder((const char*)selectedIn, (char*)dstPtr, (int)dctxPtr->tmpInTarget, (int)dctxPtr->maxBlockSize, (const char*)dctxPtr->dict, (int)dctxPtr->dictSize); + if (decodedSize < 0) return (size_t)-LizardF_ERROR_GENERIC; /* decompression failed */ + if (dctxPtr->frameInfo.contentChecksumFlag) XXH32_update(&(dctxPtr->xxh), dstPtr, decodedSize); + if (dctxPtr->frameInfo.contentSize) dctxPtr->frameRemainingSize -= decodedSize; + + /* dictionary management */ + if (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) + LizardF_updateDict(dctxPtr, dstPtr, decodedSize, dstStart, 0); + + dstPtr += decodedSize; + dctxPtr->dStage = dstage_getCBlockSize; + break; + } + + case dstage_decodeCBlock_intoTmp: + /* not enough place into dst : decode into tmpOut */ + { int (*decoder)(const char*, char*, int, int, const char*, int); + int decodedSize; + + if (dctxPtr->frameInfo.blockMode == LizardF_blockLinked) + decoder = Lizard_decompress_safe_usingDict; + else + decoder = LizardF_decompress_safe; + + /* ensure enough place for tmpOut */ + if (dctxPtr->frameInfo.blockMode == LizardF_blockLinked) { + if (dctxPtr->dict == dctxPtr->tmpOutBuffer) { + if (dctxPtr->dictSize > 2 * LIZARD_DICT_SIZE) { + memcpy(dctxPtr->tmpOutBuffer, dctxPtr->dict + dctxPtr->dictSize - LIZARD_DICT_SIZE, LIZARD_DICT_SIZE); + dctxPtr->dictSize = LIZARD_DICT_SIZE; + } + dctxPtr->tmpOut = dctxPtr->tmpOutBuffer + dctxPtr->dictSize; + } else { /* dict not within tmp */ + size_t reservedDictSpace = dctxPtr->dictSize; + if (reservedDictSpace > LIZARD_DICT_SIZE) reservedDictSpace = LIZARD_DICT_SIZE; + dctxPtr->tmpOut = dctxPtr->tmpOutBuffer + reservedDictSpace; + } + } + + /* Decode */ + decodedSize = decoder((const char*)selectedIn, (char*)dctxPtr->tmpOut, (int)dctxPtr->tmpInTarget, (int)dctxPtr->maxBlockSize, (const char*)dctxPtr->dict, (int)dctxPtr->dictSize); + if (decodedSize < 0) return (size_t)-LizardF_ERROR_decompressionFailed; /* decompression failed */ + if (dctxPtr->frameInfo.contentChecksumFlag) XXH32_update(&(dctxPtr->xxh), dctxPtr->tmpOut, decodedSize); + if (dctxPtr->frameInfo.contentSize) dctxPtr->frameRemainingSize -= decodedSize; + dctxPtr->tmpOutSize = decodedSize; + dctxPtr->tmpOutStart = 0; + dctxPtr->dStage = dstage_flushOut; + break; + } + + case dstage_flushOut: /* flush decoded data from tmpOut to dstBuffer */ + { size_t sizeToCopy = dctxPtr->tmpOutSize - dctxPtr->tmpOutStart; + if (sizeToCopy > (size_t)(dstEnd-dstPtr)) sizeToCopy = dstEnd-dstPtr; + memcpy(dstPtr, dctxPtr->tmpOut + dctxPtr->tmpOutStart, sizeToCopy); + + /* dictionary management */ + if (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) + LizardF_updateDict(dctxPtr, dstPtr, sizeToCopy, dstStart, 1); + + dctxPtr->tmpOutStart += sizeToCopy; + dstPtr += sizeToCopy; + + /* end of flush ? */ + if (dctxPtr->tmpOutStart == dctxPtr->tmpOutSize) { + dctxPtr->dStage = dstage_getCBlockSize; + break; + } + nextSrcSizeHint = BHSize; + doAnotherStage = 0; /* still some data to flush */ + break; + } + + case dstage_getSuffix: + { size_t const suffixSize = dctxPtr->frameInfo.contentChecksumFlag * 4; + if (dctxPtr->frameRemainingSize) return (size_t)-LizardF_ERROR_frameSize_wrong; /* incorrect frame size decoded */ + if (suffixSize == 0) { /* frame completed */ + nextSrcSizeHint = 0; + dctxPtr->dStage = dstage_getHeader; + doAnotherStage = 0; + break; + } + if ((srcEnd - srcPtr) < 4) { /* not enough size for entire CRC */ + dctxPtr->tmpInSize = 0; + dctxPtr->dStage = dstage_storeSuffix; + } else { + selectedIn = srcPtr; + srcPtr += 4; + } + } + + if (dctxPtr->dStage == dstage_storeSuffix) /* can be skipped */ + case dstage_storeSuffix: + { + size_t sizeToCopy = 4 - dctxPtr->tmpInSize; + if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; + memcpy(dctxPtr->tmpIn + dctxPtr->tmpInSize, srcPtr, sizeToCopy); + srcPtr += sizeToCopy; + dctxPtr->tmpInSize += sizeToCopy; + if (dctxPtr->tmpInSize < 4) { /* not enough input to read complete suffix */ + nextSrcSizeHint = 4 - dctxPtr->tmpInSize; + doAnotherStage=0; + break; + } + selectedIn = dctxPtr->tmpIn; + } + + /* case dstage_checkSuffix: */ /* no direct call, to avoid scan-build warning */ + { U32 const readCRC = LizardF_readLE32(selectedIn); + U32 const resultCRC = XXH32_digest(&(dctxPtr->xxh)); + if (readCRC != resultCRC) return (size_t)-LizardF_ERROR_contentChecksum_invalid; + nextSrcSizeHint = 0; + dctxPtr->dStage = dstage_getHeader; + doAnotherStage = 0; + break; + } + + case dstage_getSFrameSize: + if ((srcEnd - srcPtr) >= 4) { + selectedIn = srcPtr; + srcPtr += 4; + } else { + /* not enough input to read cBlockSize field */ + dctxPtr->tmpInSize = 4; + dctxPtr->tmpInTarget = 8; + dctxPtr->dStage = dstage_storeSFrameSize; + } + + if (dctxPtr->dStage == dstage_storeSFrameSize) + case dstage_storeSFrameSize: + { + size_t sizeToCopy = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; + if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; + memcpy(dctxPtr->header + dctxPtr->tmpInSize, srcPtr, sizeToCopy); + srcPtr += sizeToCopy; + dctxPtr->tmpInSize += sizeToCopy; + if (dctxPtr->tmpInSize < dctxPtr->tmpInTarget) { /* not enough input to get full sBlockSize; wait for more */ + nextSrcSizeHint = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; + doAnotherStage = 0; + break; + } + selectedIn = dctxPtr->header + 4; + } + + /* case dstage_decodeSFrameSize: */ /* no direct access */ + { size_t const SFrameSize = LizardF_readLE32(selectedIn); + dctxPtr->frameInfo.contentSize = SFrameSize; + dctxPtr->tmpInTarget = SFrameSize; + dctxPtr->dStage = dstage_skipSkippable; + break; + } + + case dstage_skipSkippable: + { size_t skipSize = dctxPtr->tmpInTarget; + if (skipSize > (size_t)(srcEnd-srcPtr)) skipSize = srcEnd-srcPtr; + srcPtr += skipSize; + dctxPtr->tmpInTarget -= skipSize; + doAnotherStage = 0; + nextSrcSizeHint = dctxPtr->tmpInTarget; + if (nextSrcSizeHint) break; + dctxPtr->dStage = dstage_getHeader; + break; + } + } + } + + /* preserve dictionary within tmp if necessary */ + if ( (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) + &&(dctxPtr->dict != dctxPtr->tmpOutBuffer) + &&(!decompressOptionsPtr->stableDst) + &&((unsigned)(dctxPtr->dStage-1) < (unsigned)(dstage_getSuffix-1)) + ) + { + if (dctxPtr->dStage == dstage_flushOut) { + size_t preserveSize = dctxPtr->tmpOut - dctxPtr->tmpOutBuffer; + size_t copySize = LIZARD_DICT_SIZE - dctxPtr->tmpOutSize; + const BYTE* oldDictEnd = dctxPtr->dict + dctxPtr->dictSize - dctxPtr->tmpOutStart; + if (dctxPtr->tmpOutSize > LIZARD_DICT_SIZE) copySize = 0; + if (copySize > preserveSize) copySize = preserveSize; + + memcpy(dctxPtr->tmpOutBuffer + preserveSize - copySize, oldDictEnd - copySize, copySize); + + dctxPtr->dict = dctxPtr->tmpOutBuffer; + dctxPtr->dictSize = preserveSize + dctxPtr->tmpOutStart; + } else { + size_t newDictSize = dctxPtr->dictSize; + const BYTE* oldDictEnd = dctxPtr->dict + dctxPtr->dictSize; + if ((newDictSize) > LIZARD_DICT_SIZE) newDictSize = LIZARD_DICT_SIZE; + + memcpy(dctxPtr->tmpOutBuffer, oldDictEnd - newDictSize, newDictSize); + + dctxPtr->dict = dctxPtr->tmpOutBuffer; + dctxPtr->dictSize = newDictSize; + dctxPtr->tmpOut = dctxPtr->tmpOutBuffer + newDictSize; + } + } + + /* require function to be called again from position where it stopped */ + if (srcPtrsrcExpect = srcPtr; + else + dctxPtr->srcExpect = NULL; + + *srcSizePtr = (srcPtr - srcStart); + *dstSizePtr = (dstPtr - dstStart); + return nextSrcSizeHint; +} diff --git a/contrib/lizard/lib/lizard_frame.h b/contrib/lizard/lib/lizard_frame.h new file mode 100644 index 00000000000..7cdd20063c5 --- /dev/null +++ b/contrib/lizard/lib/lizard_frame.h @@ -0,0 +1,303 @@ +/* + Lizard auto-framing library + Header File + Copyright (C) 2011-2015, Yann Collet + Copyright (C) 2016-2017, Przemyslaw Skibinski + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +/* LizardF is a stand-alone API to create Lizard-compressed frames + * conformant with specification v1.5.1. + * All related operations, including memory management, are handled internally by the library. + * You don't need lizard_compress.h when using lizard_frame.h. + * */ + +#pragma once + +#if defined (__cplusplus) +extern "C" { +#endif + +/*-************************************ +* Includes +**************************************/ +#include /* size_t */ + + +/*-************************************ +* Error management +**************************************/ +typedef size_t LizardF_errorCode_t; + +unsigned LizardF_isError(LizardF_errorCode_t code); +const char* LizardF_getErrorName(LizardF_errorCode_t code); /* return error code string; useful for debugging */ + + +/*-************************************ +* Frame compression types +**************************************/ +//#define LIZARDF_DISABLE_OBSOLETE_ENUMS +#ifndef LIZARDF_DISABLE_OBSOLETE_ENUMS +# define LIZARDF_OBSOLETE_ENUM(x) ,x +#else +# define LIZARDF_OBSOLETE_ENUM(x) +#endif + +typedef enum { + LizardF_default=0, + LizardF_max128KB=1, + LizardF_max256KB=2, + LizardF_max1MB=3, + LizardF_max4MB=4, + LizardF_max16MB=5, + LizardF_max64MB=6, + LizardF_max256MB=7 +} LizardF_blockSizeID_t; + +typedef enum { + LizardF_blockLinked=0, + LizardF_blockIndependent + LIZARDF_OBSOLETE_ENUM(blockLinked = LizardF_blockLinked) + LIZARDF_OBSOLETE_ENUM(blockIndependent = LizardF_blockIndependent) +} LizardF_blockMode_t; + +typedef enum { + LizardF_noContentChecksum=0, + LizardF_contentChecksumEnabled + LIZARDF_OBSOLETE_ENUM(noContentChecksum = LizardF_noContentChecksum) + LIZARDF_OBSOLETE_ENUM(contentChecksumEnabled = LizardF_contentChecksumEnabled) +} LizardF_contentChecksum_t; + +typedef enum { + LizardF_frame=0, + LizardF_skippableFrame + LIZARDF_OBSOLETE_ENUM(skippableFrame = LizardF_skippableFrame) +} LizardF_frameType_t; + +#ifndef LIZARDF_DISABLE_OBSOLETE_ENUMS +typedef LizardF_blockSizeID_t blockSizeID_t; +typedef LizardF_blockMode_t blockMode_t; +typedef LizardF_frameType_t frameType_t; +typedef LizardF_contentChecksum_t contentChecksum_t; +#endif + +typedef struct { + LizardF_blockSizeID_t blockSizeID; /* max64KB, max256KB, max1MB, max4MB ; 0 == default */ + LizardF_blockMode_t blockMode; /* blockLinked, blockIndependent ; 0 == default */ + LizardF_contentChecksum_t contentChecksumFlag; /* noContentChecksum, contentChecksumEnabled ; 0 == default */ + LizardF_frameType_t frameType; /* LizardF_frame, skippableFrame ; 0 == default */ + unsigned long long contentSize; /* Size of uncompressed (original) content ; 0 == unknown */ + unsigned reserved[2]; /* must be zero for forward compatibility */ +} LizardF_frameInfo_t; + +typedef struct { + LizardF_frameInfo_t frameInfo; + int compressionLevel; /* 0 == default (fast mode); values above 16 count as 16; values below 0 count as 0 */ + unsigned autoFlush; /* 1 == always flush (reduce need for tmp buffer) */ + unsigned reserved[4]; /* must be zero for forward compatibility */ +} LizardF_preferences_t; + + +/*-********************************* +* Simple compression function +***********************************/ +size_t LizardF_compressFrameBound(size_t srcSize, const LizardF_preferences_t* preferencesPtr); + +/*!LizardF_compressFrame() : + * Compress an entire srcBuffer into a valid Lizard frame, as defined by specification v1.5.1 + * The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. + * You can get the minimum value of dstMaxSize by using LizardF_compressFrameBound() + * If this condition is not respected, LizardF_compressFrame() will fail (result is an errorCode) + * The LizardF_preferences_t structure is optional : you can provide NULL as argument. All preferences will be set to default. + * The result of the function is the number of bytes written into dstBuffer. + * The function outputs an error code if it fails (can be tested using LizardF_isError()) + */ +size_t LizardF_compressFrame(void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_preferences_t* preferencesPtr); + + + +/*-*********************************** +* Advanced compression functions +*************************************/ +typedef struct LizardF_cctx_s* LizardF_compressionContext_t; /* must be aligned on 8-bytes */ + +typedef struct { + unsigned stableSrc; /* 1 == src content will remain available on future calls to LizardF_compress(); avoid saving src content within tmp buffer as future dictionary */ + unsigned reserved[3]; +} LizardF_compressOptions_t; + +/* Resource Management */ + +#define LIZARDF_VERSION 100 +LizardF_errorCode_t LizardF_createCompressionContext(LizardF_compressionContext_t* cctxPtr, unsigned version); +LizardF_errorCode_t LizardF_freeCompressionContext(LizardF_compressionContext_t cctx); +/* LizardF_createCompressionContext() : + * The first thing to do is to create a compressionContext object, which will be used in all compression operations. + * This is achieved using LizardF_createCompressionContext(), which takes as argument a version and an LizardF_preferences_t structure. + * The version provided MUST be LIZARDF_VERSION. It is intended to track potential version differences between different binaries. + * The function will provide a pointer to a fully allocated LizardF_compressionContext_t object. + * If the result LizardF_errorCode_t is not zero, there was an error during context creation. + * Object can release its memory using LizardF_freeCompressionContext(); + */ + + +/* Compression */ + +size_t LizardF_compressBegin(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const LizardF_preferences_t* prefsPtr); +/* LizardF_compressBegin() : + * will write the frame header into dstBuffer. + * dstBuffer must be large enough to accommodate a header (dstMaxSize). Maximum header size is 15 bytes. + * The LizardF_preferences_t structure is optional : you can provide NULL as argument, all preferences will then be set to default. + * The result of the function is the number of bytes written into dstBuffer for the header + * or an error code (can be tested using LizardF_isError()) + */ + +size_t LizardF_compressBound(size_t srcSize, const LizardF_preferences_t* prefsPtr); +/* LizardF_compressBound() : + * Provides the minimum size of Dst buffer given srcSize to handle worst case situations. + * Different preferences can produce different results. + * prefsPtr is optional : you can provide NULL as argument, all preferences will then be set to cover worst case. + * This function includes frame termination cost (4 bytes, or 8 if frame checksum is enabled) + */ + +size_t LizardF_compressUpdate(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_compressOptions_t* cOptPtr); +/* LizardF_compressUpdate() + * LizardF_compressUpdate() can be called repetitively to compress as much data as necessary. + * The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. + * You can get the minimum value of dstMaxSize by using LizardF_compressBound(). + * If this condition is not respected, LizardF_compress() will fail (result is an errorCode). + * LizardF_compressUpdate() doesn't guarantee error recovery, so you have to reset compression context when an error occurs. + * The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. + * The result of the function is the number of bytes written into dstBuffer : it can be zero, meaning input data was just buffered. + * The function outputs an error code if it fails (can be tested using LizardF_isError()) + */ + +size_t LizardF_flush(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* cOptPtr); +/* LizardF_flush() + * Should you need to generate compressed data immediately, without waiting for the current block to be filled, + * you can call Lizard_flush(), which will immediately compress any remaining data buffered within cctx. + * Note that dstMaxSize must be large enough to ensure the operation will be successful. + * LizardF_compressOptions_t structure is optional : you can provide NULL as argument. + * The result of the function is the number of bytes written into dstBuffer + * (it can be zero, this means there was no data left within cctx) + * The function outputs an error code if it fails (can be tested using LizardF_isError()) + */ + +size_t LizardF_compressEnd(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* cOptPtr); +/* LizardF_compressEnd() + * When you want to properly finish the compressed frame, just call LizardF_compressEnd(). + * It will flush whatever data remained within compressionContext (like Lizard_flush()) + * but also properly finalize the frame, with an endMark and a checksum. + * The result of the function is the number of bytes written into dstBuffer (necessarily >= 4 (endMark), or 8 if optional frame checksum is enabled) + * The function outputs an error code if it fails (can be tested using LizardF_isError()) + * The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. + * A successful call to LizardF_compressEnd() makes cctx available again for next compression task. + */ + + +/*-********************************* +* Decompression functions +***********************************/ + +typedef struct LizardF_dctx_s* LizardF_decompressionContext_t; /* must be aligned on 8-bytes */ + +typedef struct { + unsigned stableDst; /* guarantee that decompressed data will still be there on next function calls (avoid storage into tmp buffers) */ + unsigned reserved[3]; +} LizardF_decompressOptions_t; + + +/* Resource management */ + +/*!LizardF_createDecompressionContext() : + * Create an LizardF_decompressionContext_t object, which will be used to track all decompression operations. + * The version provided MUST be LIZARDF_VERSION. It is intended to track potential breaking differences between different versions. + * The function will provide a pointer to a fully allocated and initialized LizardF_decompressionContext_t object. + * The result is an errorCode, which can be tested using LizardF_isError(). + * dctx memory can be released using LizardF_freeDecompressionContext(); + * The result of LizardF_freeDecompressionContext() is indicative of the current state of decompressionContext when being released. + * That is, it should be == 0 if decompression has been completed fully and correctly. + */ +LizardF_errorCode_t LizardF_createDecompressionContext(LizardF_decompressionContext_t* dctxPtr, unsigned version); +LizardF_errorCode_t LizardF_freeDecompressionContext(LizardF_decompressionContext_t dctx); + + +/*====== Decompression ======*/ + +/*!LizardF_getFrameInfo() : + * This function decodes frame header information (such as max blockSize, frame checksum, etc.). + * Its usage is optional. The objective is to extract frame header information, typically for allocation purposes. + * A header size is variable and can be from 7 to 15 bytes. It's also possible to input more bytes than that. + * The number of bytes read from srcBuffer will be updated within *srcSizePtr (necessarily <= original value). + * (note that LizardF_getFrameInfo() can also be used anytime *after* starting decompression, in this case 0 input byte is enough) + * Frame header info is *copied into* an already allocated LizardF_frameInfo_t structure. + * The function result is an hint about how many srcSize bytes LizardF_decompress() expects for next call, + * or an error code which can be tested using LizardF_isError() + * (typically, when there is not enough src bytes to fully decode the frame header) + * Decompression is expected to resume from where it stopped (srcBuffer + *srcSizePtr) + */ +size_t LizardF_getFrameInfo(LizardF_decompressionContext_t dctx, + LizardF_frameInfo_t* frameInfoPtr, + const void* srcBuffer, size_t* srcSizePtr); + +/*!LizardF_decompress() : + * Call this function repetitively to regenerate data compressed within srcBuffer. + * The function will attempt to decode *srcSizePtr bytes from srcBuffer, into dstBuffer of maximum size *dstSizePtr. + * + * The number of bytes regenerated into dstBuffer will be provided within *dstSizePtr (necessarily <= original value). + * + * The number of bytes read from srcBuffer will be provided within *srcSizePtr (necessarily <= original value). + * If number of bytes read is < number of bytes provided, then decompression operation is not completed. + * It typically happens when dstBuffer is not large enough to contain all decoded data. + * LizardF_decompress() must be called again, starting from where it stopped (srcBuffer + *srcSizePtr) + * The function will check this condition, and refuse to continue if it is not respected. + * + * `dstBuffer` is expected to be flushed between each call to the function, its content will be overwritten. + * `dst` arguments can be changed at will at each consecutive call to the function. + * + * The function result is an hint of how many `srcSize` bytes LizardF_decompress() expects for next call. + * Schematically, it's the size of the current (or remaining) compressed block + header of next block. + * Respecting the hint provides some boost to performance, since it does skip intermediate buffers. + * This is just a hint though, it's always possible to provide any srcSize. + * When a frame is fully decoded, the function result will be 0 (no more data expected). + * If decompression failed, function result is an error code, which can be tested using LizardF_isError(). + * + * After a frame is fully decoded, dctx can be used again to decompress another frame. + */ +size_t LizardF_decompress(LizardF_decompressionContext_t dctx, + void* dstBuffer, size_t* dstSizePtr, + const void* srcBuffer, size_t* srcSizePtr, + const LizardF_decompressOptions_t* dOptPtr); + + + +#if defined (__cplusplus) +} +#endif diff --git a/contrib/lizard/lib/lizard_frame_static.h b/contrib/lizard/lib/lizard_frame_static.h new file mode 100644 index 00000000000..da80756057b --- /dev/null +++ b/contrib/lizard/lib/lizard_frame_static.h @@ -0,0 +1,81 @@ +/* + Lizard auto-framing library + Header File for static linking only + Copyright (C) 2011-2015, Yann Collet. + Copyright (C) 2016-2017, Przemyslaw Skibinski + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +#pragma once + +#if defined (__cplusplus) +extern "C" { +#endif + +/* lizard_frame_static.h should be used solely in the context of static linking. + * It contains definitions which may still change overtime. + * Never use it in the context of DLL linking. + * */ + + +/************************************** +* Includes +**************************************/ +#include "lizard_frame.h" + + +/************************************** + * Error management + * ************************************/ +#define LIZARDF_LIST_ERRORS(ITEM) \ + ITEM(OK_NoError) ITEM(ERROR_GENERIC) \ + ITEM(ERROR_maxBlockSize_invalid) ITEM(ERROR_blockMode_invalid) ITEM(ERROR_contentChecksumFlag_invalid) \ + ITEM(ERROR_compressionLevel_invalid) \ + ITEM(ERROR_headerVersion_wrong) ITEM(ERROR_blockChecksum_unsupported) ITEM(ERROR_reservedFlag_set) \ + ITEM(ERROR_allocation_failed) \ + ITEM(ERROR_srcSize_tooLarge) ITEM(ERROR_dstMaxSize_tooSmall) \ + ITEM(ERROR_frameHeader_incomplete) ITEM(ERROR_frameType_unknown) ITEM(ERROR_frameSize_wrong) \ + ITEM(ERROR_srcPtr_wrong) \ + ITEM(ERROR_decompressionFailed) \ + ITEM(ERROR_headerChecksum_invalid) ITEM(ERROR_contentChecksum_invalid) \ + ITEM(ERROR_maxCode) + +//#define LIZARDF_DISABLE_OLD_ENUMS +#ifndef LIZARDF_DISABLE_OLD_ENUMS +#define LIZARDF_GENERATE_ENUM(ENUM) LizardF_##ENUM, ENUM = LizardF_##ENUM, +#else +#define LIZARDF_GENERATE_ENUM(ENUM) LizardF_##ENUM, +#endif +typedef enum { LIZARDF_LIST_ERRORS(LIZARDF_GENERATE_ENUM) } LizardF_errorCodes; /* enum is exposed, to handle specific errors; compare function result to -enum value */ + + +#if defined (__cplusplus) +} +#endif diff --git a/contrib/lizard/lib/lizard_parser_fast.h b/contrib/lizard/lib/lizard_parser_fast.h new file mode 100644 index 00000000000..f9e54b7f475 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_fast.h @@ -0,0 +1,196 @@ +#define LIZARD_FAST_MIN_OFFSET 8 +#define LIZARD_FAST_LONGOFF_MM 0 /* not used with offsets > 1<<16 */ + +/************************************** +* Hash Functions +**************************************/ +static size_t Lizard_hashPosition(const void* p) +{ + if (MEM_64bits()) + return Lizard_hash5Ptr(p, LIZARD_HASHLOG_LZ4); + return Lizard_hash4Ptr(p, LIZARD_HASHLOG_LZ4); +} + +static void Lizard_putPositionOnHash(const BYTE* p, size_t h, U32* hashTable, const BYTE* srcBase) +{ + hashTable[h] = (U32)(p-srcBase); +} + +static void Lizard_putPosition(const BYTE* p, U32* hashTable, const BYTE* srcBase) +{ + size_t const h = Lizard_hashPosition(p); + Lizard_putPositionOnHash(p, h, hashTable, srcBase); +} + +static U32 Lizard_getPositionOnHash(size_t h, U32* hashTable) +{ + return hashTable[h]; +} + +static U32 Lizard_getPosition(const BYTE* p, U32* hashTable) +{ + size_t const h = Lizard_hashPosition(p); + return Lizard_getPositionOnHash(h, hashTable); +} + + +static const U32 Lizard_skipTrigger = 6; /* Increase this value ==> compression run slower on incompressible data */ +static const U32 Lizard_minLength = (MFLIMIT+1); + + +FORCE_INLINE int Lizard_compress_fast( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const U32 acceleration = 1; + const BYTE* base = ctx->base; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictBase = ctx->dictBase; + const BYTE* const dictEnd = dictBase + dictLimit; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = iend - LASTLITERALS; + const BYTE* anchor = ip; + + size_t forwardH, matchIndex; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 lowLimit = (ctx->lowLimit + maxDistance >= (U32)(ip - base)) ? ctx->lowLimit : (U32)(ip - base) - maxDistance; + + /* Init conditions */ + if ((U32)(iend-ip) > (U32)LIZARD_MAX_INPUT_SIZE) goto _output_error; /* Unsupported inputSize, too large (or negative) */ + + if ((U32)(iend-ip) < Lizard_minLength) goto _last_literals; /* Input too small, no compression (all literals) */ + + /* First Byte */ + Lizard_putPosition(ip, ctx->hashTable, base); + ip++; forwardH = Lizard_hashPosition(ip); + + /* Main Loop */ + for ( ; ; ) { + const BYTE* match; + size_t matchLength; + + /* Find a match */ + { const BYTE* forwardIp = ip; + unsigned step = 1; + unsigned searchMatchNb = acceleration << Lizard_skipTrigger; + while (1) { + size_t const h = forwardH; + ip = forwardIp; + forwardIp += step; + step = (searchMatchNb++ >> Lizard_skipTrigger); + + if (unlikely(forwardIp > mflimit)) goto _last_literals; + + matchIndex = Lizard_getPositionOnHash(h, ctx->hashTable); + forwardH = Lizard_hashPosition(forwardIp); + Lizard_putPositionOnHash(ip, h, ctx->hashTable, base); + + if ((matchIndex < lowLimit) || (matchIndex >= (U32)(ip - base)) || (base + matchIndex + maxDistance < ip)) continue; + + if (matchIndex >= dictLimit) { + match = base + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) +#endif + if (MEM_read32(match) == MEM_read32(ip)) + { + int back = 0; + matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); + + while ((ip+back > anchor) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + matchLength -= back; +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + { + ip += back; + match += back; + break; + } + } + } else { + match = dictBase + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + const U32 newLowLimit = (lowLimit + maxDistance >= (U32)(ip-base)) ? lowLimit : (U32)(ip - base) - maxDistance; + int back = 0; + matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); + + while ((ip+back > anchor) && (matchIndex+back > newLowLimit) && (ip[back-1] == match[back-1])) back--; + matchLength -= back; + match = base + matchIndex + back; +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + { + ip += back; + break; + } + } + } + } // while (1) + } + +_next_match: + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, matchLength+MINMATCH, match)) goto _output_error; + + /* Test end of chunk */ + if (ip > mflimit) break; + + /* Fill table */ + Lizard_putPosition(ip-2, ctx->hashTable, base); + + /* Test next position */ + matchIndex = Lizard_getPosition(ip, ctx->hashTable); + Lizard_putPosition(ip, ctx->hashTable, base); + if ((matchIndex >= lowLimit) && (matchIndex < (U32)(ip - base)) && (base + matchIndex + maxDistance >= ip)) + { + if (matchIndex >= dictLimit) { + match = base + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) +#endif + if (MEM_read32(match) == MEM_read32(ip)) + { + matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + goto _next_match; + } + } else { + match = dictBase + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); + match = base + matchIndex; +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + goto _next_match; + } + } + } + + /* Prepare next loop */ + forwardH = Lizard_hashPosition(++ip); + } + +_last_literals: + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} diff --git a/contrib/lizard/lib/lizard_parser_fastbig.h b/contrib/lizard/lib/lizard_parser_fastbig.h new file mode 100644 index 00000000000..30c464fcb90 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_fastbig.h @@ -0,0 +1,175 @@ +#define LIZARD_FASTBIG_LONGOFF_MM MM_LONGOFF + +/************************************** +* Hash Functions +**************************************/ +static size_t Lizard_hashPositionHLog(const void* p, int hashLog) +{ + if (MEM_64bits()) + return Lizard_hash5Ptr(p, hashLog); + return Lizard_hash4Ptr(p, hashLog); +} + +static void Lizard_putPositionOnHashHLog(const BYTE* p, size_t h, U32* hashTable, const BYTE* srcBase) +{ + hashTable[h] = (U32)(p-srcBase); +} + +static void Lizard_putPositionHLog(const BYTE* p, U32* hashTable, const BYTE* srcBase, int hashLog) +{ + size_t const h = Lizard_hashPositionHLog(p, hashLog); + Lizard_putPositionOnHashHLog(p, h, hashTable, srcBase); +} + +static U32 Lizard_getPositionOnHashHLog(size_t h, U32* hashTable) +{ + return hashTable[h]; +} + +static U32 Lizard_getPositionHLog(const BYTE* p, U32* hashTable, int hashLog) +{ + size_t const h = Lizard_hashPositionHLog(p, hashLog); + return Lizard_getPositionOnHashHLog(h, hashTable); +} + +FORCE_INLINE int Lizard_compress_fastBig( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const U32 acceleration = 1; + const BYTE* base = ctx->base; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictBase = ctx->dictBase; + const BYTE* const dictEnd = dictBase + dictLimit; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = iend - LASTLITERALS; + const BYTE* anchor = ip; + + size_t forwardH, matchIndex; + const int hashLog = ctx->params.hashLog; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 lowLimit = (ctx->lowLimit + maxDistance >= (U32)(ip - base)) ? ctx->lowLimit : (U32)(ip - base) - maxDistance; + + /* Init conditions */ + if ((U32)(iend-ip) > (U32)LIZARD_MAX_INPUT_SIZE) goto _output_error; /* Unsupported inputSize, too large (or negative) */ + + if ((U32)(iend-ip) < Lizard_minLength) goto _last_literals; /* Input too small, no compression (all literals) */ + + /* First Byte */ + Lizard_putPositionHLog(ip, ctx->hashTable, base, hashLog); + ip++; forwardH = Lizard_hashPositionHLog(ip, hashLog); + + /* Main Loop */ + for ( ; ; ) { + const BYTE* match; + size_t matchLength; + + /* Find a match */ + { const BYTE* forwardIp = ip; + unsigned step = 1; + unsigned searchMatchNb = acceleration << Lizard_skipTrigger; + while (1) { + size_t const h = forwardH; + ip = forwardIp; + forwardIp += step; + step = (searchMatchNb++ >> Lizard_skipTrigger); + + if (unlikely(forwardIp > mflimit)) goto _last_literals; + + matchIndex = Lizard_getPositionOnHashHLog(h, ctx->hashTable); + forwardH = Lizard_hashPositionHLog(forwardIp, hashLog); + Lizard_putPositionOnHashHLog(ip, h, ctx->hashTable, base); + + if ((matchIndex < lowLimit) || (matchIndex >= (U32)(ip - base)) || (base + matchIndex + maxDistance < ip)) continue; + + if (matchIndex >= dictLimit) { + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) + if (MEM_read32(match) == MEM_read32(ip)) + { + int back = 0; + matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); + + while ((ip+back > anchor) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + matchLength -= back; + if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + { + ip += back; + match += back; + break; + } + } + } else { + match = dictBase + matchIndex; + if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + const U32 newLowLimit = (lowLimit + maxDistance >= (U32)(ip-base)) ? lowLimit : (U32)(ip - base) - maxDistance; + int back = 0; + matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); + + while ((ip+back > anchor) && (matchIndex+back > newLowLimit) && (ip[back-1] == match[back-1])) back--; + matchLength -= back; + match = base + matchIndex + back; + if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + { + ip += back; + break; + } + } + } + } // while (1) + } + +_next_match: + if (Lizard_encodeSequence_LIZv1(ctx, &ip, &anchor, matchLength+MINMATCH, match)) goto _output_error; + + /* Test end of chunk */ + if (ip > mflimit) break; + + /* Fill table */ + Lizard_putPositionHLog(ip-2, ctx->hashTable, base, hashLog); + + /* Test next position */ + matchIndex = Lizard_getPositionHLog(ip, ctx->hashTable, hashLog); + Lizard_putPositionHLog(ip, ctx->hashTable, base, hashLog); + if ((matchIndex >= lowLimit) && (matchIndex < (U32)(ip - base)) && (base + matchIndex + maxDistance >= ip)) + { + if (matchIndex >= dictLimit) { + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) + if (MEM_read32(match) == MEM_read32(ip)) + { + matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); + if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + goto _next_match; + } + } else { + match = dictBase + matchIndex; + if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); + match = base + matchIndex; + if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + goto _next_match; + } + } + } + + /* Prepare next loop */ + forwardH = Lizard_hashPositionHLog(++ip, hashLog); + } + +_last_literals: + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LIZv1(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} diff --git a/contrib/lizard/lib/lizard_parser_fastsmall.h b/contrib/lizard/lib/lizard_parser_fastsmall.h new file mode 100644 index 00000000000..550edc5b201 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_fastsmall.h @@ -0,0 +1,189 @@ +/************************************** +* Hash Functions +**************************************/ +static size_t Lizard_hashPositionSmall(const void* p) +{ + if (MEM_64bits()) + return Lizard_hash5Ptr(p, LIZARD_HASHLOG_LZ4SM); + return Lizard_hash4Ptr(p, LIZARD_HASHLOG_LZ4SM); +} + +static void Lizard_putPositionOnHashSmall(const BYTE* p, size_t h, U32* hashTable, const BYTE* srcBase) +{ + hashTable[h] = (U32)(p-srcBase); +} + +static void Lizard_putPositionSmall(const BYTE* p, U32* hashTable, const BYTE* srcBase) +{ + size_t const h = Lizard_hashPositionSmall(p); + Lizard_putPositionOnHashSmall(p, h, hashTable, srcBase); +} + +static U32 Lizard_getPositionOnHashSmall(size_t h, U32* hashTable) +{ + return hashTable[h]; +} + +static U32 Lizard_getPositionSmall(const BYTE* p, U32* hashTable) +{ + size_t const h = Lizard_hashPositionSmall(p); + return Lizard_getPositionOnHashSmall(h, hashTable); +} + + +FORCE_INLINE int Lizard_compress_fastSmall( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const U32 acceleration = 1; + const BYTE* base = ctx->base; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictBase = ctx->dictBase; + const BYTE* const dictEnd = dictBase + dictLimit; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = iend - LASTLITERALS; + const BYTE* anchor = ip; + + size_t forwardH, matchIndex; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 lowLimit = (ctx->lowLimit + maxDistance >= (U32)(ip - base)) ? ctx->lowLimit : (U32)(ip - base) - maxDistance; + + /* Init conditions */ + if ((U32)(iend-ip) > (U32)LIZARD_MAX_INPUT_SIZE) goto _output_error; /* Unsupported inputSize, too large (or negative) */ + + if ((U32)(iend-ip) < Lizard_minLength) goto _last_literals; /* Input too small, no compression (all literals) */ + + /* First Byte */ + Lizard_putPositionSmall(ip, ctx->hashTable, base); + ip++; forwardH = Lizard_hashPositionSmall(ip); + + /* Main Loop */ + for ( ; ; ) { + const BYTE* match; + size_t matchLength; + + /* Find a match */ + { const BYTE* forwardIp = ip; + unsigned step = 1; + unsigned searchMatchNb = acceleration << Lizard_skipTrigger; + while (1) { + size_t const h = forwardH; + ip = forwardIp; + forwardIp += step; + step = (searchMatchNb++ >> Lizard_skipTrigger); + + if (unlikely(forwardIp > mflimit)) goto _last_literals; + + matchIndex = Lizard_getPositionOnHashSmall(h, ctx->hashTable); + forwardH = Lizard_hashPositionSmall(forwardIp); + Lizard_putPositionOnHashSmall(ip, h, ctx->hashTable, base); + + if ((matchIndex < lowLimit) || (matchIndex >= (U32)(ip - base)) || (base + matchIndex + maxDistance < ip)) continue; + + if (matchIndex >= dictLimit) { + match = base + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) +#endif + if (MEM_read32(match) == MEM_read32(ip)) + { + int back = 0; + matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); + + while ((ip+back > anchor) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + matchLength -= back; +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + { + ip += back; + match += back; + break; + } + } + } else { + match = dictBase + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + const U32 newLowLimit = (lowLimit + maxDistance >= (U32)(ip-base)) ? lowLimit : (U32)(ip - base) - maxDistance; + int back = 0; + matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); + + while ((ip+back > anchor) && (matchIndex+back > newLowLimit) && (ip[back-1] == match[back-1])) back--; + matchLength -= back; + match = base + matchIndex + back; +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + { + ip += back; + break; + } + } + } + } // while (1) + } + +_next_match: + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, matchLength+MINMATCH, match)) goto _output_error; + + /* Test end of chunk */ + if (ip > mflimit) break; + + /* Fill table */ + Lizard_putPositionSmall(ip-2, ctx->hashTable, base); + + /* Test next position */ + matchIndex = Lizard_getPositionSmall(ip, ctx->hashTable); + Lizard_putPositionSmall(ip, ctx->hashTable, base); + if ((matchIndex >= lowLimit) && (matchIndex < (U32)(ip - base)) && (base + matchIndex + maxDistance >= ip)) + { + if (matchIndex >= dictLimit) { + match = base + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) +#endif + if (MEM_read32(match) == MEM_read32(ip)) + { + matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + goto _next_match; + } + } else { + match = dictBase + matchIndex; +#if LIZARD_FAST_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); + match = base + matchIndex; +#if LIZARD_FAST_LONGOFF_MM > 0 + if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + goto _next_match; + } + } + } + + /* Prepare next loop */ + forwardH = Lizard_hashPositionSmall(++ip); + } + +_last_literals: + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} diff --git a/contrib/lizard/lib/lizard_parser_hashchain.h b/contrib/lizard/lib/lizard_parser_hashchain.h new file mode 100644 index 00000000000..64b9d76c451 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_hashchain.h @@ -0,0 +1,369 @@ +#define LIZARD_HC_MIN_OFFSET 8 +#define LIZARD_HC_LONGOFF_MM 0 /* not used with offsets > 1<<16 */ +#define OPTIMAL_ML (int)((ML_MASK_LZ4-1)+MINMATCH) +#define GET_MINMATCH(offset) (MINMATCH) + +#if 1 + #define LIZARD_HC_HASH_FUNCTION(ip, hashLog) Lizard_hashPtr(ip, hashLog, ctx->params.searchLength) +#else + #define LIZARD_HC_HASH_FUNCTION(ip, hashLog) Lizard_hash5Ptr(ip, hashLog) +#endif + +/* Update chains up to ip (excluded) */ +FORCE_INLINE void Lizard_Insert (Lizard_stream_t* ctx, const BYTE* ip) +{ + U32* const chainTable = ctx->chainTable; + U32* const hashTable = ctx->hashTable; +#if MINMATCH == 3 + U32* HashTable3 = ctx->hashTable3; +#endif + const BYTE* const base = ctx->base; + U32 const target = (U32)(ip - base); + U32 idx = ctx->nextToUpdate; + const int hashLog = ctx->params.hashLog; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + + while (idx < target) { + size_t const h = Lizard_hashPtr(base+idx, hashLog, ctx->params.searchLength); + size_t delta = idx - hashTable[h]; + if (delta>maxDistance) delta = maxDistance; + DELTANEXT(idx) = (U32)delta; + if ((hashTable[h] >= idx) || (idx >= hashTable[h] + LIZARD_HC_MIN_OFFSET)) + hashTable[h] = idx; +#if MINMATCH == 3 + HashTable3[Lizard_hash3Ptr(base+idx, ctx->params.hashLog3)] = idx; +#endif + idx++; + } + + ctx->nextToUpdate = target; +} + + + +FORCE_INLINE int Lizard_InsertAndFindBestMatch (Lizard_stream_t* ctx, /* Index table will be updated */ + const BYTE* ip, const BYTE* const iLimit, + const BYTE** matchpos) +{ + U32* const chainTable = ctx->chainTable; + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + U32 matchIndex, delta; + const BYTE* match; + int nbAttempts=ctx->params.searchNum; + size_t ml=0; + const int hashLog = ctx->params.hashLog; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 current = (U32)(ip - base); + const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; + + /* HC4 match finder */ + Lizard_Insert(ctx, ip); + matchIndex = HashTable[LIZARD_HC_HASH_FUNCTION(ip, hashLog)]; + + while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { + nbAttempts--; + if (matchIndex >= dictLimit) { + match = base + matchIndex; +#if LIZARD_HC_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_HC_MIN_OFFSET) +#endif + if (*(match+ml) == *(ip+ml) + && (MEM_read32(match) == MEM_read32(ip))) + { + size_t const mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; +#if LIZARD_HC_LONGOFF_MM > 0 + if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + if (mlt > ml) { ml = mlt; *matchpos = match; } + } + } else { + match = dictBase + matchIndex; +#if LIZARD_HC_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_HC_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; +#if LIZARD_HC_LONGOFF_MM > 0 + if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - (base + matchIndex)) < LIZARD_MAX_16BIT_OFFSET)) +#endif + if (mlt > ml) { ml = mlt; *matchpos = base + matchIndex; } /* virtual matchpos */ + } + } + delta = DELTANEXT(matchIndex); + if (delta > matchIndex) break; + matchIndex -= delta; + } + + return (int)ml; +} + + +FORCE_INLINE int Lizard_InsertAndGetWiderMatch ( + Lizard_stream_t* ctx, + const BYTE* const ip, + const BYTE* const iLowLimit, + const BYTE* const iHighLimit, + int longest, + const BYTE** matchpos, + const BYTE** startpos) +{ + U32* const chainTable = ctx->chainTable; + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictBase = ctx->dictBase; + const BYTE* const dictEnd = dictBase + dictLimit; + U32 matchIndex, delta; + int nbAttempts = ctx->params.searchNum; + int LLdelta = (int)(ip-iLowLimit); + const int hashLog = ctx->params.hashLog; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 current = (U32)(ip - base); + const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; + + /* First Match */ + Lizard_Insert(ctx, ip); + matchIndex = HashTable[LIZARD_HC_HASH_FUNCTION(ip, hashLog)]; + + while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { + nbAttempts--; + if (matchIndex >= dictLimit) { + const BYTE* match = base + matchIndex; +#if LIZARD_HC_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_HC_MIN_OFFSET) +#endif + if (*(iLowLimit + longest) == *(match - LLdelta + longest)) { + if (MEM_read32(match) == MEM_read32(ip)) { + int mlt = MINMATCH + Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit); + int back = 0; + while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + +#if LIZARD_HC_LONGOFF_MM > 0 + if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) +#endif + if (mlt > longest) { + longest = (int)mlt; + *matchpos = match+back; + *startpos = ip+back; + } + } + } + } else { + const BYTE* match = dictBase + matchIndex; +#if LIZARD_HC_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_HC_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + int back=0; + size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; + while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == match[back-1])) back--; + mlt -= back; +#if LIZARD_HC_LONGOFF_MM > 0 + if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - (base + matchIndex)) < LIZARD_MAX_16BIT_OFFSET)) +#endif + if ((int)mlt > longest) { longest = (int)mlt; *matchpos = base + matchIndex + back; *startpos = ip+back; } + } + } + delta = DELTANEXT(matchIndex); + if (delta > matchIndex) break; + matchIndex -= delta; + } + + return longest; +} + + +FORCE_INLINE int Lizard_compress_hashChain ( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const BYTE* anchor = ip; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = (iend - LASTLITERALS); + + int ml, ml2, ml3, ml0; + const BYTE* ref = NULL; + const BYTE* start2 = NULL; + const BYTE* ref2 = NULL; + const BYTE* start3 = NULL; + const BYTE* ref3 = NULL; + const BYTE* start0; + const BYTE* ref0; + + /* init */ + ip++; + + /* Main Loop */ + while (ip < mflimit) { + ml = Lizard_InsertAndFindBestMatch (ctx, ip, matchlimit, (&ref)); + if (!ml) { ip++; continue; } + + /* saved, in case we would skip too much */ + start0 = ip; + ref0 = ref; + ml0 = ml; + +_Search2: + if (ip+ml < mflimit) + ml2 = Lizard_InsertAndGetWiderMatch(ctx, ip + ml - 2, ip + 1, matchlimit, ml, &ref2, &start2); + else ml2 = ml; + + if (ml2 == ml) { /* No better match */ + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + continue; + } + + if (start0 < ip) { + if (start2 < ip + ml0) { /* empirical */ + ip = start0; + ref = ref0; + ml = ml0; + } + } + + /* Here, start0==ip */ + if ((start2 - ip) < 3) { /* First Match too small : removed */ + ml = ml2; + ip = start2; + ref =ref2; + goto _Search2; + } + +_Search3: + /* + * Currently we have : + * ml2 > ml1, and + * ip1+3 <= ip2 (usually < ip1+ml1) + */ + if ((start2 - ip) < OPTIMAL_ML) { + int correction; + int new_ml = ml; + if (new_ml > OPTIMAL_ML) new_ml = OPTIMAL_ML; + if (ip+new_ml > start2 + ml2 - GET_MINMATCH((U32)(start2 - ref2))) { + new_ml = (int)(start2 - ip) + ml2 - GET_MINMATCH((U32)(start2 - ref2)); + if (new_ml < GET_MINMATCH((U32)(ip - ref))) { // match2 doesn't fit + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + continue; + } + } + correction = new_ml - (int)(start2 - ip); + if (correction > 0) { + start2 += correction; + ref2 += correction; + ml2 -= correction; + } + } + /* Now, we have start2 = ip+new_ml, with new_ml = min(ml, OPTIMAL_ML=18) */ + + if (start2 + ml2 < mflimit) + ml3 = Lizard_InsertAndGetWiderMatch(ctx, start2 + ml2 - 3, start2, matchlimit, ml2, &ref3, &start3); + else ml3 = ml2; + + if (ml3 == ml2) { /* No better match : 2 sequences to encode */ + /* ip & ref are known; Now for ml */ + if (start2 < ip+ml) ml = (int)(start2 - ip); + /* Now, encode 2 sequences */ + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + ip = start2; + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml2, ref2)) return 0; + continue; + } + + if (start3 < ip+ml+3) { /* Not enough space for match 2 : remove it */ + if (start3 >= (ip+ml)) { /* can write Seq1 immediately ==> Seq2 is removed, so Seq3 becomes Seq1 */ + if (start2 < ip+ml) { + int correction = (int)(ip+ml - start2); + start2 += correction; + ref2 += correction; + ml2 -= correction; + if (ml2 < GET_MINMATCH((U32)(start2 - ref2))) { + start2 = start3; + ref2 = ref3; + ml2 = ml3; + } + } + + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + ip = start3; + ref = ref3; + ml = ml3; + + start0 = start2; + ref0 = ref2; + ml0 = ml2; + goto _Search2; + } + + start2 = start3; + ref2 = ref3; + ml2 = ml3; + goto _Search3; + } + + /* + * OK, now we have 3 ascending matches; let's write at least the first one + * ip & ref are known; Now for ml + */ + if (start2 < ip+ml) { + if ((start2 - ip) < (int)ML_MASK_LZ4) { + int correction; + if (ml > OPTIMAL_ML) ml = OPTIMAL_ML; + if (ip + ml > start2 + ml2 - GET_MINMATCH((U32)(start2 - ref2))) { + ml = (int)(start2 - ip) + ml2 - GET_MINMATCH((U32)(start2 - ref2)); + if (ml < GET_MINMATCH((U32)(ip - ref))) { // match2 doesn't fit, remove it + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + ip = start3; + ref = ref3; + ml = ml3; + + start0 = start2; + ref0 = ref2; + ml0 = ml2; + goto _Search2; + } + } + correction = ml - (int)(start2 - ip); + if (correction > 0) { + start2 += correction; + ref2 += correction; + ml2 -= correction; + } + } else { + ml = (int)(start2 - ip); + } + } + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + + ip = start2; + ref = ref2; + ml = ml2; + + start2 = start3; + ref2 = ref3; + ml2 = ml3; + + goto _Search3; + } + + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} diff --git a/contrib/lizard/lib/lizard_parser_lowestprice.h b/contrib/lizard/lib/lizard_parser_lowestprice.h new file mode 100644 index 00000000000..f8b95dad4ea --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_lowestprice.h @@ -0,0 +1,376 @@ +#define LIZARD_LOWESTPRICE_MIN_OFFSET 8 + + +FORCE_INLINE size_t Lizard_more_profitable(Lizard_stream_t* const ctx, const BYTE *best_ip, size_t best_off, size_t best_common, const BYTE *ip, size_t off, size_t common, size_t literals, int last_off) +{ + size_t sum; + + if (literals > 0) + sum = MAX(common + literals, best_common); + else + sum = MAX(common, best_common - literals); + + if ((int)off == last_off) off = 0; // rep code + if ((int)best_off == last_off) best_off = 0; + + return Lizard_get_price_LIZv1(ctx, last_off, ip, ctx->off24pos, sum - common, (U32)off, common) <= Lizard_get_price_LIZv1(ctx, last_off, best_ip, ctx->off24pos, sum - best_common, (U32)best_off, best_common); +} + + +FORCE_INLINE size_t Lizard_better_price(Lizard_stream_t* const ctx, const BYTE *best_ip, size_t best_off, size_t best_common, const BYTE *ip, size_t off, size_t common, int last_off) +{ + if ((int)off == last_off) off = 0; // rep code + if ((int)best_off == last_off) best_off = 0; + + return Lizard_get_price_LIZv1(ctx, last_off, ip, ctx->off24pos, 0, (U32)off, common) < Lizard_get_price_LIZv1(ctx, last_off, best_ip, ctx->off24pos, common - best_common, (U32)best_off, best_common); +} + + +FORCE_INLINE int Lizard_FindMatchLowestPrice (Lizard_stream_t* ctx, /* Index table will be updated */ + const BYTE* ip, const BYTE* const iLimit, + const BYTE** matchpos) +{ + U32* const chainTable = ctx->chainTable; + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const intptr_t dictLimit = ctx->dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; + const intptr_t current = (ip - base); + const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; + const BYTE* const lowPrefixPtr = base + dictLimit; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + intptr_t matchIndex; + const BYTE* match, *matchDict; + int nbAttempts=ctx->params.searchNum; + size_t ml=0, mlt; + + matchIndex = HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; + + if (ctx->last_off >= LIZARD_LOWESTPRICE_MIN_OFFSET) { + intptr_t matchIndexLO = (ip - ctx->last_off) - base; + if (matchIndexLO >= lowLimit) { + if (matchIndexLO >= dictLimit) { + match = base + matchIndexLO; + mlt = Lizard_count(ip, match, iLimit);// + MINMATCH; + // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) + if (mlt > REPMINMATCH) { + *matchpos = match; + return (int)mlt; + } + } else { + match = dictBase + matchIndexLO; + if ((U32)((dictLimit-1) - matchIndexLO) >= 3) { /* intentional overflow */ + mlt = Lizard_count_2segments(ip, match, iLimit, dictEnd, lowPrefixPtr); + // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) + if (mlt > REPMINMATCH) { + *matchpos = base + matchIndexLO; /* virtual matchpos */ + return (int)mlt; + } + } + } + } + } + + +#if MINMATCH == 3 + { + U32 matchIndex3 = ctx->hashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; + if (matchIndex3 < current && matchIndex3 >= lowLimit) + { + size_t offset = (size_t)current - matchIndex3; + if (offset < LIZARD_MAX_8BIT_OFFSET) + { + match = ip - offset; + if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) + { + ml = 3;//Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + *matchpos = match; + } + } + } + } +#endif + while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { + nbAttempts--; + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_LOWESTPRICE_MIN_OFFSET) { + if (matchIndex >= dictLimit) { + if (*(match+ml) == *(ip+ml) && (MEM_read32(match) == MEM_read32(ip))) { + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (!ml || (mlt > ml && Lizard_better_price(ctx, ip, (ip - *matchpos), ml, ip, (ip - match), mlt, ctx->last_off))) + { ml = mlt; *matchpos = match; } + } + } else { + matchDict = dictBase + matchIndex; + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(matchDict) == MEM_read32(ip)) { + mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (!ml || (mlt > ml && Lizard_better_price(ctx, ip, (ip - *matchpos), ml, ip, (U32)(ip - match), mlt, ctx->last_off))) + { ml = mlt; *matchpos = match; } /* virtual matchpos */ + } + } + } + matchIndex -= chainTable[matchIndex & contentMask]; + } + + return (int)ml; +} + + +FORCE_INLINE size_t Lizard_GetWiderMatch ( + Lizard_stream_t* ctx, + const BYTE* const ip, + const BYTE* const iLowLimit, + const BYTE* const iHighLimit, + size_t longest, + const BYTE** matchpos, + const BYTE** startpos) +{ + U32* const chainTable = ctx->chainTable; + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const intptr_t dictLimit = ctx->dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; + const intptr_t current = (ip - base); + const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; + const BYTE* const lowPrefixPtr = base + dictLimit; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const BYTE* match, *matchDict; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + intptr_t matchIndex; + int nbAttempts = ctx->params.searchNum; + size_t mlt; + + /* First Match */ + matchIndex = HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; + + if (ctx->last_off >= LIZARD_LOWESTPRICE_MIN_OFFSET) { + intptr_t matchIndexLO = (ip - ctx->last_off) - base; + if (matchIndexLO >= lowLimit) { + if (matchIndexLO >= dictLimit) { + match = base + matchIndexLO; + if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { + int back = 0; + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit) + MINMATCH; + while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + + if (mlt > longest) + if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) { + *matchpos = match+back; + *startpos = ip+back; + longest = mlt; + } + } + } else { + match = dictBase + matchIndexLO; + if ((U32)((dictLimit-1) - matchIndexLO) >= 3) /* intentional overflow */ + if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { + int back=0; + mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; + while ((ip+back > iLowLimit) && (matchIndexLO+back > lowLimit) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + + if (mlt > longest) + if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) { + *matchpos = base + matchIndexLO + back; /* virtual matchpos */ + *startpos = ip+back; + longest = mlt; + } + } + } + } + } + +#if MINMATCH == 3 + { + U32 matchIndex3 = ctx->hashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; + if (matchIndex3 < current && matchIndex3 >= lowLimit) { + size_t offset = (size_t)current - matchIndex3; + if (offset < LIZARD_MAX_8BIT_OFFSET) { + match = ip - offset; + if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) { + mlt = Lizard_count(ip + MINMATCH, match + MINMATCH, iHighLimit) + MINMATCH; + + int back = 0; + while ((ip + back > iLowLimit) && (match + back > lowPrefixPtr) && (ip[back - 1] == match[back - 1])) back--; + mlt -= back; + + if (!longest || (mlt > longest && Lizard_better_price(ctx, *startpos, (*startpos - *matchpos), longest, ip, (ip - match), mlt, ctx->last_off))) { + *matchpos = match + back; + *startpos = ip + back; + longest = mlt; + } + } + } + } + } +#endif + + while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { + nbAttempts--; + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_LOWESTPRICE_MIN_OFFSET) { + if (matchIndex >= dictLimit) { + if (MEM_read32(match) == MEM_read32(ip)) { + int back = 0; + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit) + MINMATCH; + while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (!longest || (mlt > longest && Lizard_better_price(ctx, *startpos, (*startpos - *matchpos), longest, ip, (ip - match), mlt, ctx->last_off))) + { longest = mlt; *startpos = ip+back; *matchpos = match+back; } + } + } else { + matchDict = dictBase + matchIndex; + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(matchDict) == MEM_read32(ip)) { + int back=0; + mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; + while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == matchDict[back-1])) back--; + mlt -= back; + + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (!longest || (mlt > longest && Lizard_better_price(ctx, *startpos, (*startpos - *matchpos), longest, ip, (U32)(ip - match), mlt, ctx->last_off))) + { longest = mlt; *startpos = ip+back; *matchpos = match+back; } /* virtual matchpos */ + } + } + } + matchIndex -= chainTable[matchIndex & contentMask]; + } + + return longest; +} + + + + +FORCE_INLINE int Lizard_compress_lowestPrice( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const BYTE* anchor = ip; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = (iend - LASTLITERALS); + + size_t ml, ml2, ml0; + const BYTE* ref=NULL; + const BYTE* start2=NULL; + const BYTE* ref2=NULL; + const BYTE* start0; + const BYTE* ref0; + const BYTE* lowPrefixPtr = ctx->base + ctx->dictLimit; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + const size_t sufficient_len = ctx->params.sufficientLength; + + /* Main Loop */ + while (ip < mflimit) + { + Lizard_Insert(ctx, ip); + ml = Lizard_FindMatchLowestPrice (ctx, ip, matchlimit, (&ref)); + if (!ml) { ip++; continue; } + + { + int back = 0; + while ((ip + back > anchor) && (ref + back > lowPrefixPtr) && (ip[back - 1] == ref[back - 1])) back--; + ml -= back; + ip += back; + ref += back; + } + + /* saved, in case we would skip too much */ + start0 = ip; + ref0 = ref; + ml0 = ml; + // goto _Encode; + +_Search: + if (ip+ml >= mflimit) { goto _Encode; } + if (ml >= sufficient_len) { goto _Encode; } + + Lizard_Insert(ctx, ip); + ml2 = (int)Lizard_GetWiderMatch(ctx, ip + ml - 2, anchor, matchlimit, 0, &ref2, &start2); + if (!ml2) goto _Encode; + + { + U64 price, best_price; + int off0=0, off1=0; + const BYTE *pos, *best_pos; + + // find the lowest price for encoding ml bytes + best_pos = ip; + best_price = LIZARD_MAX_PRICE; + off0 = (int)(ip - ref); + off1 = (int)(start2 - ref2); + + for (pos = ip + ml; pos >= start2; pos--) + { + int common0 = (int)(pos - ip); + if (common0 >= MINMATCH) { + price = (int)Lizard_get_price_LIZv1(ctx, ctx->last_off, ip, ctx->off24pos, ip - anchor, (off0 == ctx->last_off) ? 0 : off0, common0); + + { + int common1 = (int)(start2 + ml2 - pos); + if (common1 >= MINMATCH) + price += Lizard_get_price_LIZv1(ctx, ctx->last_off, pos, ctx->off24pos, 0, (off1 == off0) ? 0 : (off1), common1); + else + price += Lizard_get_price_LIZv1(ctx, ctx->last_off, pos, ctx->off24pos, common1, 0, 0); + } + + if (price < best_price) { + best_price = price; + best_pos = pos; + } + } else { + price = Lizard_get_price_LIZv1(ctx, ctx->last_off, ip, ctx->off24pos, start2 - anchor, (off1 == ctx->last_off) ? 0 : off1, ml2); + + if (price < best_price) + best_pos = pos; + break; + } + } + ml = (int)(best_pos - ip); + } + + + if ((ml < MINMATCH) || ((ml < minMatchLongOff) && ((U32)(ip-ref) >= LIZARD_MAX_16BIT_OFFSET))) + { + ip = start2; + ref = ref2; + ml = ml2; + goto _Search; + } + +_Encode: + if (start0 < ip) + { + if (Lizard_more_profitable(ctx, ip, (ip - ref), ml, start0, (start0 - ref0), ml0, (ref0 - ref), ctx->last_off)) + { + ip = start0; + ref = ref0; + ml = ml0; + } + } + + if (Lizard_encodeSequence_LIZv1(ctx, &ip, &anchor, ml, ((ip - ref == ctx->last_off) ? ip : ref))) return 0; + } + + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LIZv1(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} + diff --git a/contrib/lizard/lib/lizard_parser_nochain.h b/contrib/lizard/lib/lizard_parser_nochain.h new file mode 100644 index 00000000000..e8abb586ff4 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_nochain.h @@ -0,0 +1,318 @@ +#define OPTIMAL_ML (int)((ML_MASK_LZ4-1)+MINMATCH) + +//#define LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog) Lizard_hashPtr(ip, hashLog, ctx->params.searchLength) +#define LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog) Lizard_hash5Ptr(ip, hashLog) +#define LIZARD_NOCHAIN_MIN_OFFSET 8 + +/* Update chains up to ip (excluded) */ +FORCE_INLINE void Lizard_InsertNoChain (Lizard_stream_t* ctx, const BYTE* ip) +{ + U32* const hashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + U32 const target = (U32)(ip - base); + U32 idx = ctx->nextToUpdate; + const int hashLog = ctx->params.hashLog; + + while (idx < target) { + size_t const h = LIZARD_NOCHAIN_HASH_FUNCTION(base+idx, hashLog); + if ((hashTable[h] >= idx) || (idx >= hashTable[h] + LIZARD_NOCHAIN_MIN_OFFSET)) + hashTable[h] = idx; + idx++; + } + + ctx->nextToUpdate = target; +} + + +FORCE_INLINE int Lizard_InsertAndFindBestMatchNoChain (Lizard_stream_t* ctx, /* Index table will be updated */ + const BYTE* ip, const BYTE* const iLimit, + const BYTE** matchpos) +{ + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + U32 matchIndex; + const BYTE* match; + size_t ml=0; + const int hashLog = ctx->params.hashLog; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 current = (U32)(ip - base); + const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; + + /* HC4 match finder */ + Lizard_InsertNoChain(ctx, ip); + matchIndex = HashTable[LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog)]; + + if ((matchIndex < current) && (matchIndex >= lowLimit)) { + if (matchIndex >= dictLimit) { + match = base + matchIndex; +#if LIZARD_NOCHAIN_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_NOCHAIN_MIN_OFFSET) +#endif + if (*(match+ml) == *(ip+ml) && (MEM_read32(match) == MEM_read32(ip))) + { + size_t const mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + if (mlt > ml) { ml = mlt; *matchpos = match; } + } + } else { + match = dictBase + matchIndex; +#if LIZARD_NOCHAIN_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_NOCHAIN_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; + if (mlt > ml) { ml = mlt; *matchpos = base + matchIndex; } /* virtual matchpos */ + } + } + } + + return (int)ml; +} + + +FORCE_INLINE int Lizard_InsertAndGetWiderMatchNoChain ( + Lizard_stream_t* ctx, + const BYTE* const ip, + const BYTE* const iLowLimit, + const BYTE* const iHighLimit, + int longest, + const BYTE** matchpos, + const BYTE** startpos) +{ + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const BYTE* const dictBase = ctx->dictBase; + const BYTE* const dictEnd = dictBase + dictLimit; + U32 matchIndex; + int LLdelta = (int)(ip-iLowLimit); + const int hashLog = ctx->params.hashLog; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 current = (U32)(ip - base); + const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; + + /* First Match */ + Lizard_InsertNoChain(ctx, ip); + matchIndex = HashTable[LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog)]; + + if ((matchIndex < current) && (matchIndex >= lowLimit)) { + if (matchIndex >= dictLimit) { + const BYTE* match = base + matchIndex; +#if LIZARD_NOCHAIN_MIN_OFFSET > 0 + if ((U32)(ip - match) >= LIZARD_NOCHAIN_MIN_OFFSET) +#endif + if (*(iLowLimit + longest) == *(match - LLdelta + longest)) { + if (MEM_read32(match) == MEM_read32(ip)) { + int mlt = MINMATCH + Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit); + int back = 0; + while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + + if (mlt > longest) { + longest = (int)mlt; + *matchpos = match+back; + *startpos = ip+back; + } + } + } + } else { + const BYTE* match = dictBase + matchIndex; +#if LIZARD_NOCHAIN_MIN_OFFSET > 0 + if ((U32)(ip - (base + matchIndex)) >= LIZARD_NOCHAIN_MIN_OFFSET) +#endif + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(match) == MEM_read32(ip)) { + int back=0; + size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; + while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + if ((int)mlt > longest) { longest = (int)mlt; *matchpos = base + matchIndex + back; *startpos = ip+back; } + } + } + } + + return longest; +} + + +FORCE_INLINE int Lizard_compress_noChain ( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const BYTE* anchor = ip; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = (iend - LASTLITERALS); + + int ml, ml2, ml3, ml0; + const BYTE* ref = NULL; + const BYTE* start2 = NULL; + const BYTE* ref2 = NULL; + const BYTE* start3 = NULL; + const BYTE* ref3 = NULL; + const BYTE* start0; + const BYTE* ref0; + + /* init */ + ip++; + + /* Main Loop */ + while (ip < mflimit) { + ml = Lizard_InsertAndFindBestMatchNoChain (ctx, ip, matchlimit, (&ref)); + if (!ml) { ip++; continue; } + + /* saved, in case we would skip too much */ + start0 = ip; + ref0 = ref; + ml0 = ml; + +_Search2: + if (ip+ml < mflimit) + ml2 = Lizard_InsertAndGetWiderMatchNoChain(ctx, ip + ml - 2, ip + 1, matchlimit, ml, &ref2, &start2); + else ml2 = ml; + + if (ml2 == ml) { /* No better match */ + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + continue; + } + + if (start0 < ip) { + if (start2 < ip + ml0) { /* empirical */ + ip = start0; + ref = ref0; + ml = ml0; + } + } + + /* Here, start0==ip */ + if ((start2 - ip) < 3) { /* First Match too small : removed */ + ml = ml2; + ip = start2; + ref =ref2; + goto _Search2; + } + +_Search3: + /* + * Currently we have : + * ml2 > ml1, and + * ip1+3 <= ip2 (usually < ip1+ml1) + */ + if ((start2 - ip) < OPTIMAL_ML) { + int correction; + int new_ml = ml; + if (new_ml > OPTIMAL_ML) new_ml = OPTIMAL_ML; + if (ip+new_ml > start2 + ml2 - MINMATCH) new_ml = (int)(start2 - ip) + ml2 - MINMATCH; + correction = new_ml - (int)(start2 - ip); + if (correction > 0) { + start2 += correction; + ref2 += correction; + ml2 -= correction; + } + } + /* Now, we have start2 = ip+new_ml, with new_ml = min(ml, OPTIMAL_ML=18) */ + + if (start2 + ml2 < mflimit) + ml3 = Lizard_InsertAndGetWiderMatchNoChain(ctx, start2 + ml2 - 3, start2, matchlimit, ml2, &ref3, &start3); + else ml3 = ml2; + + if (ml3 == ml2) { /* No better match : 2 sequences to encode */ + /* ip & ref are known; Now for ml */ + if (start2 < ip+ml) ml = (int)(start2 - ip); + /* Now, encode 2 sequences */ + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + ip = start2; + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml2, ref2)) return 0; + continue; + } + + if (start3 < ip+ml+3) { /* Not enough space for match 2 : remove it */ + if (start3 >= (ip+ml)) { /* can write Seq1 immediately ==> Seq2 is removed, so Seq3 becomes Seq1 */ + if (start2 < ip+ml) { + int correction = (int)(ip+ml - start2); + start2 += correction; + ref2 += correction; + ml2 -= correction; + if (ml2 < MINMATCH) { + start2 = start3; + ref2 = ref3; + ml2 = ml3; + } + } + + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + ip = start3; + ref = ref3; + ml = ml3; + + start0 = start2; + ref0 = ref2; + ml0 = ml2; + goto _Search2; + } + + start2 = start3; + ref2 = ref3; + ml2 = ml3; + goto _Search3; + } + + /* + * OK, now we have 3 ascending matches; let's write at least the first one + * ip & ref are known; Now for ml + */ + if (start2 < ip+ml) { + if ((start2 - ip) < (int)ML_MASK_LZ4) { + int correction; + if (ml > OPTIMAL_ML) ml = OPTIMAL_ML; + if (ip + ml > start2 + ml2 - MINMATCH) { + ml = (int)(start2 - ip) + ml2 - MINMATCH; + if (ml < MINMATCH) { // match2 doesn't fit, remove it + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + ip = start3; + ref = ref3; + ml = ml3; + + start0 = start2; + ref0 = ref2; + ml0 = ml2; + goto _Search2; + } + } + correction = ml - (int)(start2 - ip); + if (correction > 0) { + start2 += correction; + ref2 += correction; + ml2 -= correction; + } + } else { + ml = (int)(start2 - ip); + } + } + if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; + + ip = start2; + ref = ref2; + ml = ml2; + + start2 = start3; + ref2 = ref3; + ml2 = ml3; + + goto _Search3; + } + + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} diff --git a/contrib/lizard/lib/lizard_parser_optimal.h b/contrib/lizard/lib/lizard_parser_optimal.h new file mode 100644 index 00000000000..8c9b83e9590 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_optimal.h @@ -0,0 +1,679 @@ +#define LIZARD_LOG_PARSER(fmt, ...) //printf(fmt, __VA_ARGS__) +#define LIZARD_LOG_PRICE(fmt, ...) //printf(fmt, __VA_ARGS__) +#define LIZARD_LOG_ENCODE(fmt, ...) //printf(fmt, __VA_ARGS__) + +#define LIZARD_OPTIMAL_MIN_OFFSET 8 +#define LIZARD_OPT_NUM (1<<12) +#define REPMINMATCH 1 + + +FORCE_INLINE size_t Lizard_get_price(Lizard_stream_t* const ctx, int rep, const BYTE *ip, const BYTE *off24pos, size_t litLength, U32 offset, size_t matchLength) +{ + if (ctx->params.decompressType == Lizard_coderwords_LZ4) + return Lizard_get_price_LZ4(ctx, ip, litLength, offset, matchLength); + + return Lizard_get_price_LIZv1(ctx, rep, ip, off24pos, litLength, offset, matchLength); +} + + + +typedef struct +{ + int off; + int len; + int back; +} Lizard_match_t; + +typedef struct +{ + int price; + int off; + int mlen; + int litlen; + int rep; + const BYTE* off24pos; +} Lizard_optimal_t; + + +/* Update chains up to ip (excluded) */ +FORCE_INLINE void Lizard_BinTree_Insert(Lizard_stream_t* ctx, const BYTE* ip) +{ +#if MINMATCH == 3 + U32* HashTable3 = ctx->hashTable3; + const BYTE* const base = ctx->base; + const U32 target = (U32)(ip - base); + U32 idx = ctx->nextToUpdate; + + while(idx < target) { + HashTable3[Lizard_hash3Ptr(base+idx, ctx->params.hashLog3)] = idx; + idx++; + } + + ctx->nextToUpdate = target; +#else + (void)ctx; (void)ip; +#endif +} + + + +FORCE_INLINE int Lizard_GetAllMatches ( + Lizard_stream_t* ctx, + const BYTE* const ip, + const BYTE* const iLowLimit, + const BYTE* const iHighLimit, + size_t best_mlen, + Lizard_match_t* matches) +{ + U32* const chainTable = ctx->chainTable; + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const intptr_t dictLimit = ctx->dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; + const intptr_t current = (ip - base); + const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const BYTE* match, *matchDict; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + intptr_t matchIndex; + int nbAttempts = ctx->params.searchNum; + // bool fullSearch = (ctx->params.fullSearch >= 2); + int mnum = 0; + U32* HashPos; + size_t mlt; + + if (ip + MINMATCH > iHighLimit) return 0; + + /* First Match */ + HashPos = &HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; + matchIndex = *HashPos; +#if MINMATCH == 3 + { + U32* const HashTable3 = ctx->hashTable3; + U32* HashPos3 = &HashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; + + if ((*HashPos3 < current) && (*HashPos3 >= lowLimit)) { + size_t offset = current - *HashPos3; + if (offset < LIZARD_MAX_8BIT_OFFSET) { + match = ip - offset; + if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) { + size_t mlt = Lizard_count(ip + MINMATCH, match + MINMATCH, iHighLimit) + MINMATCH; + + int back = 0; + while ((ip + back > iLowLimit) && (match + back > lowPrefixPtr) && (ip[back - 1] == match[back - 1])) back--; + mlt -= back; + + matches[mnum].off = (int)offset; + matches[mnum].len = (int)mlt; + matches[mnum].back = -back; + mnum++; + } + } + } + + *HashPos3 = current; + } +#endif + + chainTable[current & contentMask] = (U32)(current - matchIndex); + *HashPos = (U32)current; + ctx->nextToUpdate++; + + if (best_mlen < MINMATCH-1) best_mlen = MINMATCH-1; + + while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { + nbAttempts--; + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_OPTIMAL_MIN_OFFSET) { + if (matchIndex >= dictLimit) { + if ((/*fullSearch ||*/ ip[best_mlen] == match[best_mlen]) && (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip))) { + int back = 0; + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit) + MINMATCH; + while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; + mlt -= back; + + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (mlt > best_mlen) { + best_mlen = mlt; + matches[mnum].off = (int)(ip - match); + matches[mnum].len = (int)mlt; + matches[mnum].back = -back; + mnum++; + + if (best_mlen > LIZARD_OPT_NUM) break; + } + } + } else { + matchDict = dictBase + matchIndex; + // fprintf(stderr, "dictBase[%p]+matchIndex[%d]=match[%p] dictLimit=%d base=%p ip=%p iLimit=%p off=%d\n", dictBase, matchIndex, match, dictLimit, base, ip, iLimit, (U32)(ip-match)); + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_readMINMATCH(matchDict) == MEM_readMINMATCH(ip)) { + int back=0; + mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; + while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == matchDict[back-1])) back--; + mlt -= back; + + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (mlt > best_mlen) { + best_mlen = mlt; + matches[mnum].off = (int)(ip - match); + matches[mnum].len = (int)mlt; + matches[mnum].back = -back; + mnum++; + + if (best_mlen > LIZARD_OPT_NUM) break; + } + } + } + } + matchIndex -= chainTable[matchIndex & contentMask]; + } + + return mnum; +} + + + + +FORCE_INLINE int Lizard_BinTree_GetAllMatches ( + Lizard_stream_t* ctx, + const BYTE* const ip, + const BYTE* const iHighLimit, + size_t best_mlen, + Lizard_match_t* matches) +{ + U32* const chainTable = ctx->chainTable; + U32* const HashTable = ctx->hashTable; + const BYTE* const base = ctx->base; + const intptr_t dictLimit = ctx->dictLimit; + const BYTE* const dictBase = ctx->dictBase; + const BYTE* const dictEnd = dictBase + dictLimit; + const U32 contentMask = (1 << ctx->params.contentLog) - 1; + const BYTE* const lowPrefixPtr = base + dictLimit; + const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; + const intptr_t current = (ip - base); + const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; + const BYTE* match; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + int nbAttempts = ctx->params.searchNum; + int mnum = 0; + U32 *ptr0, *ptr1, delta0, delta1; + intptr_t matchIndex; + size_t mlt = 0; + U32* HashPos; + + if (ip + MINMATCH > iHighLimit) return 0; + + /* First Match */ + HashPos = &HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; + matchIndex = *HashPos; + + +#if MINMATCH == 3 + { + U32* HashPos3 = &ctx->hashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; + + if ((*HashPos3 < current) && (*HashPos3 >= lowLimit)) { + size_t offset = current - *HashPos3; + if (offset < LIZARD_MAX_8BIT_OFFSET) { + match = ip - offset; + if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) + { + mlt = Lizard_count(ip + MINMATCH, match + MINMATCH, iHighLimit) + MINMATCH; + + matches[mnum].off = (int)offset; + matches[mnum].len = (int)mlt; + matches[mnum].back = 0; + mnum++; + } + } + *HashPos3 = current; + } + } +#endif + + *HashPos = (U32)current; + ctx->nextToUpdate++; + + // check rest of matches + ptr0 = &chainTable[(current*2+1) & contentMask]; + ptr1 = &chainTable[(current*2) & contentMask]; + delta0 = delta1 = (U32)(current - matchIndex); + + if (best_mlen < MINMATCH-1) best_mlen = MINMATCH-1; + + while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { + nbAttempts--; + if (matchIndex >= dictLimit) { + match = base + matchIndex; + // if (ip[mlt] == match[mlt]) + mlt = Lizard_count(ip, match, iHighLimit); + } else { + match = dictBase + matchIndex; + mlt = Lizard_count_2segments(ip, match, iHighLimit, dictEnd, lowPrefixPtr); + if (matchIndex + (int)mlt >= dictLimit) + match = base + matchIndex; /* to prepare for next usage of match[mlt] */ + } + + if ((U32)(current - matchIndex) >= LIZARD_OPTIMAL_MIN_OFFSET) { + if ((mlt >= minMatchLongOff) || ((U32)(current - matchIndex) < LIZARD_MAX_16BIT_OFFSET)) + if (mlt > best_mlen) { + best_mlen = mlt; + matches[mnum].off = (int)(current - matchIndex); + matches[mnum].len = (int)mlt; + matches[mnum].back = 0; + mnum++; + + if (mlt > LIZARD_OPT_NUM) break; + if (ip + mlt >= iHighLimit) break; + } + } else { +#if 1 + intptr_t newMatchIndex; + size_t newml = 0, newoff = 0; + do { + newoff += (int)(current - matchIndex); + } while (newoff < LIZARD_OPTIMAL_MIN_OFFSET); + newMatchIndex = current - newoff; + if (newMatchIndex >= dictLimit) newml = Lizard_count(ip, base + newMatchIndex, iHighLimit); + + // printf("%d: off=%d mlt=%d\n", (U32)current, (U32)(current - matchIndex), (int)mlt); + // printf("%d: newoff=%d newml=%d\n", (U32)current, (int)newoff, (int)newml); + + if ((newml >= minMatchLongOff) && (newml > best_mlen)) { + best_mlen = newml; + matches[mnum].off = (int)newoff; + matches[mnum].len = (int)newml; + matches[mnum].back = 0; + mnum++; + + if (newml > LIZARD_OPT_NUM) break; + if (ip + newml >= iHighLimit) break; + } +#endif + } + + if (ip[mlt] < match[mlt]) { + *ptr0 = delta0; + ptr0 = &chainTable[(matchIndex*2) & contentMask]; + if (*ptr0 == (U32)-1) break; + delta0 = *ptr0; + delta1 += delta0; + matchIndex -= delta0; + } else { + *ptr1 = delta1; + ptr1 = &chainTable[(matchIndex*2+1) & contentMask]; + if (*ptr1 == (U32)-1) break; + delta1 = *ptr1; + delta0 += delta1; + matchIndex -= delta1; + } + } + + *ptr0 = (U32)-1; + *ptr1 = (U32)-1; + + return mnum; +} + + +#define SET_PRICE(pos, mlen, offset, litlen, price) \ + { \ + while (last_pos < pos) { opt[last_pos+1].price = LIZARD_MAX_PRICE; last_pos++; } \ + opt[pos].mlen = (int)mlen; \ + opt[pos].off = (int)offset; \ + opt[pos].litlen = (int)litlen; \ + opt[pos].price = (int)price; \ + LIZARD_LOG_PARSER("%d: SET price[%d/%d]=%d litlen=%d len=%d off=%d\n", (int)(inr-source), pos, last_pos, opt[pos].price, opt[pos].litlen, opt[pos].mlen, opt[pos].off); \ + } + + +FORCE_INLINE int Lizard_compress_optimalPrice( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + Lizard_optimal_t opt[LIZARD_OPT_NUM + 4]; + Lizard_match_t matches[LIZARD_OPT_NUM + 1]; + const BYTE *inr; + size_t res, cur, cur2, skip_num = 0; + size_t i, llen, litlen, mlen, best_mlen, price, offset, best_off, match_num, last_pos; + + const BYTE* anchor = ip; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = (iend - LASTLITERALS); + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const intptr_t dictLimit = ctx->dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + const BYTE* const lowPrefixPtr = base + dictLimit; + const intptr_t lowLimit = ctx->lowLimit; + const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; + + const size_t sufficient_len = ctx->params.sufficientLength; + const int faster_get_matches = (ctx->params.fullSearch == 0); + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + const int lizardOptimalMinOffset = (ctx->params.decompressType == Lizard_coderwords_LZ4) ? (1<<30) : LIZARD_OPTIMAL_MIN_OFFSET; + const size_t repMinMatch = (ctx->params.decompressType == Lizard_coderwords_LZ4) ? MINMATCH : REPMINMATCH; + + /* Main Loop */ + while (ip < mflimit) { + memset(opt, 0, sizeof(Lizard_optimal_t)); + last_pos = 0; + llen = ip - anchor; + + /* check rep code */ + + if (ctx->last_off >= lizardOptimalMinOffset) { + intptr_t matchIndexLO = (ip - ctx->last_off) - base; + mlen = 0; + if ((matchIndexLO >= lowLimit) && (base + matchIndexLO + maxDistance >= ip)) { + if (matchIndexLO >= dictLimit) { + mlen = Lizard_count(ip, base + matchIndexLO, matchlimit); + } else { + mlen = Lizard_count_2segments(ip, dictBase + matchIndexLO, matchlimit, dictEnd, lowPrefixPtr); + } + } + if (mlen >= REPMINMATCH) { + if (mlen > sufficient_len || mlen >= LIZARD_OPT_NUM) { + best_mlen = mlen; best_off = 0; cur = 0; last_pos = 1; + goto encode; + } + + do + { + litlen = 0; + price = Lizard_get_price(ctx, ctx->last_off, ip, ctx->off24pos, llen, 0, mlen); + if (mlen > last_pos || price < (size_t)opt[mlen].price) + SET_PRICE(mlen, mlen, 0, litlen, price); + mlen--; + } + while (mlen >= REPMINMATCH); + } + } + + if (faster_get_matches && last_pos) + match_num = 0; + else + { + if (ctx->params.parserType == Lizard_parser_optimalPrice) { + Lizard_Insert(ctx, ip); + match_num = Lizard_GetAllMatches(ctx, ip, ip, matchlimit, last_pos, matches); + } else { + Lizard_BinTree_Insert(ctx, ip); + match_num = Lizard_BinTree_GetAllMatches(ctx, ip, matchlimit, last_pos, matches); + } + } + + LIZARD_LOG_PARSER("%d: match_num=%d last_pos=%d\n", (int)(ip-source), match_num, last_pos); + if (!last_pos && !match_num) { ip++; continue; } + + if (match_num && (size_t)matches[match_num-1].len > sufficient_len) { + best_mlen = matches[match_num-1].len; + best_off = matches[match_num-1].off; + cur = 0; + last_pos = 1; + goto encode; + } + + // set prices using matches at position = 0 + best_mlen = (last_pos > MINMATCH) ? last_pos : MINMATCH; + + for (i = 0; i < match_num; i++) { + mlen = (i>0) ? (size_t)matches[i-1].len+1 : best_mlen; + best_mlen = (matches[i].len < LIZARD_OPT_NUM) ? matches[i].len : LIZARD_OPT_NUM; + LIZARD_LOG_PARSER("%d: start Found mlen=%d off=%d best_mlen=%d last_pos=%d\n", (int)(ip-source), matches[i].len, matches[i].off, best_mlen, last_pos); + while (mlen <= best_mlen){ + litlen = 0; + price = Lizard_get_price(ctx, ctx->last_off, ip, ctx->off24pos, llen + litlen, matches[i].off, mlen); + + if ((mlen >= minMatchLongOff) || (matches[i].off < LIZARD_MAX_16BIT_OFFSET)) + if (mlen > last_pos || price < (size_t)opt[mlen].price) + SET_PRICE(mlen, mlen, matches[i].off, litlen, price); + mlen++; + } + } + + if (last_pos < repMinMatch) { ip++; continue; } + + opt[0].off24pos = ctx->off24pos; + opt[0].rep = ctx->last_off; + opt[0].mlen = 1; + opt[0].off = -1; + + // check further positions + for (skip_num = 0, cur = 1; cur <= last_pos; cur++) { + int rep; + inr = ip + cur; + + if (opt[cur-1].off == -1) { // -1 = literals, 0 = rep + litlen = opt[cur-1].litlen + 1; + + if (cur != litlen) { + price = opt[cur - litlen].price + Lizard_get_price(ctx, opt[cur-litlen].rep, inr, ctx->off24pos, litlen, 0, 0); + LIZARD_LOG_PRICE("%d: TRY1 opt[%d].price=%d price=%d cur=%d litlen=%d\n", (int)(inr-source), cur - litlen, opt[cur - litlen].price, price, cur, litlen); + } else { + price = Lizard_get_price(ctx, ctx->last_off, inr, ctx->off24pos, llen + litlen, 0, 0); + LIZARD_LOG_PRICE("%d: TRY2 price=%d cur=%d litlen=%d llen=%d\n", (int)(inr-source), price, cur, litlen, llen); + } + } else { + litlen = 1; + price = opt[cur - 1].price + Lizard_get_price(ctx, opt[cur-1].rep, inr, ctx->off24pos, litlen, 0, 0); + LIZARD_LOG_PRICE("%d: TRY3 price=%d cur=%d litlen=%d litonly=%d\n", (int)(inr-source), price, cur, litlen, Lizard_get_price(ctx, rep, inr, ctx->off24pos, litlen, 0, 0)); + } + + mlen = 1; + best_mlen = 0; + LIZARD_LOG_PARSER("%d: TRY price=%d opt[%d].price=%d\n", (int)(inr-source), price, cur, opt[cur].price); + + if (cur > last_pos || price <= (size_t)opt[cur].price) // || ((price == opt[cur].price) && (opt[cur-1].mlen == 1) && (cur != litlen))) + SET_PRICE(cur, mlen, -1, litlen, price); + + if (cur == last_pos) break; + + + + /* set rep code */ + if (opt[cur].off != -1) { + mlen = opt[cur].mlen; + offset = opt[cur].off; + if (offset < 1) { + opt[cur].rep = opt[cur-mlen].rep; + opt[cur].off24pos = opt[cur-mlen].off24pos; + LIZARD_LOG_PARSER("%d: COPYREP1 cur=%d mlen=%d rep=%d\n", (int)(inr-source), cur, mlen, opt[cur-mlen].rep); + } else { + opt[cur].rep = (int)offset; + opt[cur].off24pos = (offset >= LIZARD_MAX_16BIT_OFFSET) ? inr : opt[cur-mlen].off24pos; + LIZARD_LOG_PARSER("%d: COPYREP2 cur=%d offset=%d rep=%d\n", (int)(inr-source), cur, offset, opt[cur].rep); + } + } else { + opt[cur].rep = opt[cur-1].rep; // copy rep + opt[cur].off24pos = opt[cur-1].off24pos; + } + + rep = opt[cur].rep; + LIZARD_LOG_PARSER("%d: CURRENT price[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(inr-source), cur, last_pos, opt[cur].price, opt[cur].off, opt[cur].mlen, opt[cur].litlen, opt[cur].rep); + + + /* check rep code */ + if (opt[cur].rep >= lizardOptimalMinOffset) { + intptr_t matchIndexLO = (inr - opt[cur].rep) - base; + mlen = 0; + if ((matchIndexLO >= lowLimit) && (base + matchIndexLO + maxDistance >= inr)) { + if (matchIndexLO >= dictLimit) { + mlen = Lizard_count(inr, base + matchIndexLO, matchlimit); + } else { + mlen = Lizard_count_2segments(inr, dictBase + matchIndexLO, matchlimit, dictEnd, lowPrefixPtr); + } + } + if (mlen >= REPMINMATCH/* && mlen > best_mlen*/) { + LIZARD_LOG_PARSER("%d: try REP rep=%d mlen=%d\n", (int)(inr-source), opt[cur].rep, mlen); + LIZARD_LOG_PARSER("%d: Found REP mlen=%d off=%d rep=%d opt[%d].off=%d\n", (int)(inr-source), mlen, 0, opt[cur].rep, cur, opt[cur].off); + + if (mlen > sufficient_len || cur + mlen >= LIZARD_OPT_NUM) { + best_mlen = mlen; + best_off = 0; + LIZARD_LOG_PARSER("%d: REP sufficient_len=%d best_mlen=%d best_off=%d last_pos=%d\n", (int)(inr-source), sufficient_len, best_mlen, best_off, last_pos); + last_pos = cur + 1; + goto encode; + } + + best_mlen = mlen; + if (faster_get_matches) + skip_num = best_mlen; + + do + { + //if (opt[cur].mlen == 1) + if (opt[cur].off == -1) { + litlen = opt[cur].litlen; + + if (cur != litlen) { + price = opt[cur - litlen].price + Lizard_get_price(ctx, rep, inr, opt[cur].off24pos, litlen, 0, mlen); + LIZARD_LOG_PRICE("%d: TRY1 opt[%d].price=%d price=%d cur=%d litlen=%d\n", (int)(inr-source), cur - litlen, opt[cur - litlen].price, price, cur, litlen); + } else { + price = Lizard_get_price(ctx, rep, inr, ctx->off24pos, llen + litlen, 0, mlen); + LIZARD_LOG_PRICE("%d: TRY2 price=%d cur=%d litlen=%d llen=%d\n", (int)(inr-source), price, cur, litlen, llen); + } + } else { + litlen = 0; + price = opt[cur].price + Lizard_get_price(ctx, rep, inr, opt[cur].off24pos, litlen, 0, mlen); + LIZARD_LOG_PRICE("%d: TRY3 price=%d cur=%d litlen=%d getprice=%d\n", (int)(inr-source), price, cur, litlen, Lizard_get_price(ctx, rep, inr, opt[cur].off24pos, litlen, 0, mlen - MINMATCH)); + } + + LIZARD_LOG_PARSER("%d: Found REP mlen=%d off=%d price=%d litlen=%d price[%d]=%d\n", (int)(inr-source), mlen, 0, price, litlen, cur - litlen, opt[cur - litlen].price); + + if (cur + mlen > last_pos || price <= (size_t)opt[cur + mlen].price) // || ((price == opt[cur + mlen].price) && (opt[cur].mlen == 1) && (cur != litlen))) // at equal price prefer REP instead of MATCH + SET_PRICE(cur + mlen, mlen, 0, litlen, price); + mlen--; + } + while (mlen >= REPMINMATCH); + } + } + + if (faster_get_matches && skip_num > 0) { + skip_num--; + continue; + } + + if (ctx->params.parserType == Lizard_parser_optimalPrice) { + Lizard_Insert(ctx, inr); + match_num = Lizard_GetAllMatches(ctx, inr, ip, matchlimit, best_mlen, matches); + LIZARD_LOG_PARSER("%d: Lizard_GetAllMatches match_num=%d\n", (int)(inr-source), match_num); + } else { + Lizard_BinTree_Insert(ctx, inr); + match_num = Lizard_BinTree_GetAllMatches(ctx, inr, matchlimit, best_mlen, matches); + LIZARD_LOG_PARSER("%d: Lizard_BinTree_GetAllMatches match_num=%d\n", (int)(inr-source), match_num); + } + + + if (match_num > 0 && (size_t)matches[match_num-1].len > sufficient_len) { + cur -= matches[match_num-1].back; + best_mlen = matches[match_num-1].len; + best_off = matches[match_num-1].off; + last_pos = cur + 1; + goto encode; + } + + // set prices using matches at position = cur + best_mlen = (best_mlen > MINMATCH) ? best_mlen : MINMATCH; + + for (i = 0; i < match_num; i++) { + mlen = (i>0) ? (size_t)matches[i-1].len+1 : best_mlen; + cur2 = cur - matches[i].back; + best_mlen = (cur2 + matches[i].len < LIZARD_OPT_NUM) ? (size_t)matches[i].len : LIZARD_OPT_NUM - cur2; + LIZARD_LOG_PARSER("%d: Found1 cur=%d cur2=%d mlen=%d off=%d best_mlen=%d last_pos=%d\n", (int)(inr-source), cur, cur2, matches[i].len, matches[i].off, best_mlen, last_pos); + + if (mlen < (size_t)matches[i].back + 1) + mlen = matches[i].back + 1; + + while (mlen <= best_mlen) { + // if (opt[cur2].mlen == 1) + if (opt[cur2].off == -1) + { + litlen = opt[cur2].litlen; + + if (cur2 != litlen) + price = opt[cur2 - litlen].price + Lizard_get_price(ctx, rep, inr, opt[cur2].off24pos, litlen, matches[i].off, mlen); + else + price = Lizard_get_price(ctx, rep, inr, ctx->off24pos, llen + litlen, matches[i].off, mlen); + } else { + litlen = 0; + price = opt[cur2].price + Lizard_get_price(ctx, rep, inr, opt[cur2].off24pos, litlen, matches[i].off, mlen); + } + + LIZARD_LOG_PARSER("%d: Found2 pred=%d mlen=%d best_mlen=%d off=%d price=%d litlen=%d price[%d]=%d\n", (int)(inr-source), matches[i].back, mlen, best_mlen, matches[i].off, price, litlen, cur - litlen, opt[cur - litlen].price); + // if (cur2 + mlen > last_pos || ((matches[i].off != opt[cur2 + mlen].off) && (price < opt[cur2 + mlen].price))) + + if ((mlen >= minMatchLongOff) || (matches[i].off < LIZARD_MAX_16BIT_OFFSET)) + if (cur2 + mlen > last_pos || price < (size_t)opt[cur2 + mlen].price) + { + SET_PRICE(cur2 + mlen, mlen, matches[i].off, litlen, price); + } + + mlen++; + } + } + } // for (skip_num = 0, cur = 1; cur <= last_pos; cur++) + + + best_mlen = opt[last_pos].mlen; + best_off = opt[last_pos].off; + cur = last_pos - best_mlen; + + encode: // cur, last_pos, best_mlen, best_off have to be set + for (i = 1; i <= last_pos; i++) { + LIZARD_LOG_PARSER("%d: price[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(ip-source+i), i, last_pos, opt[i].price, opt[i].off, opt[i].mlen, opt[i].litlen, opt[i].rep); + } + + LIZARD_LOG_PARSER("%d: cur=%d/%d best_mlen=%d best_off=%d rep=%d\n", (int)(ip-source+cur), cur, last_pos, best_mlen, best_off, opt[cur].rep); + + opt[0].mlen = 1; + + while (1) { + mlen = opt[cur].mlen; + offset = opt[cur].off; + opt[cur].mlen = (int)best_mlen; + opt[cur].off = (int)best_off; + best_mlen = mlen; + best_off = offset; + if (mlen > cur) break; + cur -= mlen; + } + + for (i = 0; i <= last_pos;) { + LIZARD_LOG_PARSER("%d: price2[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(ip-source+i), i, last_pos, opt[i].price, opt[i].off, opt[i].mlen, opt[i].litlen, opt[i].rep); + i += opt[i].mlen; + } + + cur = 0; + + while (cur < last_pos) { + LIZARD_LOG_PARSER("%d: price3[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(ip-source+cur), cur, last_pos, opt[cur].price, opt[cur].off, opt[cur].mlen, opt[cur].litlen, opt[cur].rep); + mlen = opt[cur].mlen; + // if (mlen == 1) { ip++; cur++; continue; } + if (opt[cur].off == -1) { ip++; cur++; continue; } + offset = opt[cur].off; + cur += mlen; + + LIZARD_LOG_ENCODE("%d: ENCODE literals=%d off=%d mlen=%d ", (int)(ip-source), (int)(ip-anchor), (int)(offset), mlen); + res = Lizard_encodeSequence(ctx, &ip, &anchor, mlen, ip - offset); + if (res) return 0; + + LIZARD_LOG_PARSER("%d: offset=%d rep=%d\n", (int)(ip-source), offset, ctx->last_off); + } + } + + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} + diff --git a/contrib/lizard/lib/lizard_parser_pricefast.h b/contrib/lizard/lib/lizard_parser_pricefast.h new file mode 100644 index 00000000000..9818e2c4212 --- /dev/null +++ b/contrib/lizard/lib/lizard_parser_pricefast.h @@ -0,0 +1,250 @@ +#define LIZARD_PRICEFAST_MIN_OFFSET 8 + +FORCE_INLINE int Lizard_FindMatchFast(Lizard_stream_t* ctx, intptr_t matchIndex, intptr_t matchIndex3, /* Index table will be updated */ + const BYTE* ip, const BYTE* const iLimit, + const BYTE** matchpos) +{ + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const intptr_t dictLimit = ctx->dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; + const intptr_t current = (U32)(ip - base); + const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; + const BYTE* const lowPrefixPtr = base + dictLimit; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + const BYTE* match, *matchDict; + size_t ml=0, mlt; + + if (ctx->last_off >= LIZARD_PRICEFAST_MIN_OFFSET) { + intptr_t matchIndexLO = (ip - ctx->last_off) - base; + if (matchIndexLO >= lowLimit) { + if (matchIndexLO >= dictLimit) { + match = base + matchIndexLO; + if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) + { + *matchpos = match; + return (int)mlt; + } + } + } else { + match = dictBase + matchIndexLO; + if ((U32)((dictLimit-1) - matchIndexLO) >= 3) /* intentional overflow */ + if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { + mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; + // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) + { + *matchpos = base + matchIndexLO; /* virtual matchpos */ + return (int)mlt; + } + } + } + } + } + + +#if MINMATCH == 3 + if (matchIndex3 < current && matchIndex3 >= lowLimit) { + intptr_t offset = current - matchIndex3; + if (offset < LIZARD_MAX_8BIT_OFFSET) { + match = ip - offset; + if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) { + ml = 3;//Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + *matchpos = match; + } + } + } +#else + (void)matchIndex3; +#endif + + if ((matchIndex < current) && (matchIndex >= lowLimit)) { + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_PRICEFAST_MIN_OFFSET) { + if (matchIndex >= dictLimit) { + if (*(match+ml) == *(ip+ml) && (MEM_read32(match) == MEM_read32(ip))) { + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (!ml || (mlt > ml)) // && Lizard_better_price((ip - *matchpos), ml, (ip - match), mlt, ctx->last_off))) + { ml = mlt; *matchpos = match; } + } + } else { + matchDict = dictBase + matchIndex; + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(matchDict) == MEM_read32(ip)) { + mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + if (!ml || (mlt > ml)) // && Lizard_better_price((ip - *matchpos), ml, (U32)(ip - match), mlt, ctx->last_off))) + { ml = mlt; *matchpos = match; } /* virtual matchpos */ + } + } + } + } + + return (int)ml; +} + + +FORCE_INLINE int Lizard_FindMatchFaster (Lizard_stream_t* ctx, U32 matchIndex, /* Index table will be updated */ + const BYTE* ip, const BYTE* const iLimit, + const BYTE** matchpos) +{ + const BYTE* const base = ctx->base; + const BYTE* const dictBase = ctx->dictBase; + const U32 dictLimit = ctx->dictLimit; + const BYTE* const dictEnd = dictBase + dictLimit; + const U32 maxDistance = (1 << ctx->params.windowLog) - 1; + const U32 current = (U32)(ip - base); + const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; + const BYTE* const lowPrefixPtr = base + dictLimit; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + const BYTE* match, *matchDict; + size_t ml=0, mlt; + + if (matchIndex < current && matchIndex >= lowLimit) { + match = base + matchIndex; + if ((U32)(ip - match) >= LIZARD_PRICEFAST_MIN_OFFSET) { + if (matchIndex >= dictLimit) { + if (MEM_read32(match) == MEM_read32(ip)) { + mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + { ml = mlt; *matchpos = match; } + } + } else { + matchDict = dictBase + matchIndex; + if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ + if (MEM_read32(matchDict) == MEM_read32(ip)) { + mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; + if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) + { ml = mlt; *matchpos = match; } /* virtual matchpos */ + } + } + } + } + + return (int)ml; +} + + + +FORCE_INLINE int Lizard_compress_priceFast( + Lizard_stream_t* const ctx, + const BYTE* ip, + const BYTE* const iend) +{ + const BYTE* anchor = ip; + const BYTE* const mflimit = iend - MFLIMIT; + const BYTE* const matchlimit = (iend - LASTLITERALS); + + size_t ml, ml2=0; + const BYTE* ref=NULL; + const BYTE* start2=NULL; + const BYTE* ref2=NULL; + const BYTE* lowPrefixPtr = ctx->base + ctx->dictLimit; + U32* HashTable = ctx->hashTable; +#if MINMATCH == 3 + U32* HashTable3 = ctx->hashTable3; +#endif + const BYTE* const base = ctx->base; + const size_t minMatchLongOff = ctx->params.minMatchLongOff; + U32* HashPos; + + /* init */ + ip++; + + /* Main Loop */ + while (ip < mflimit) + { + HashPos = &HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; +#if MINMATCH == 3 + { + U32* HashPos3 = &HashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; + ml = Lizard_FindMatchFast (ctx, *HashPos, *HashPos3, ip, matchlimit, (&ref)); + *HashPos3 = (U32)(ip - base); + } +#else + ml = Lizard_FindMatchFast (ctx, *HashPos, 0, ip, matchlimit, (&ref)); +#endif + if ((*HashPos >= (U32)(ip - base)) || ((U32)(ip - base) >= *HashPos + LIZARD_PRICEFAST_MIN_OFFSET)) + *HashPos = (U32)(ip - base); + + if (!ml) { ip++; continue; } + if ((int)(ip - ref) == ctx->last_off) { ml2=0; ref=ip; goto _Encode; } + + { + int back = 0; + while ((ip+back>anchor) && (ref+back > lowPrefixPtr) && (ip[back-1] == ref[back-1])) back--; + ml -= back; + ip += back; + ref += back; + } + +_Search: + if (ip+ml >= mflimit) goto _Encode; + + start2 = ip + ml - 2; + HashPos = &HashTable[Lizard_hashPtr(start2, ctx->params.hashLog, ctx->params.searchLength)]; + ml2 = Lizard_FindMatchFaster(ctx, *HashPos, start2, matchlimit, (&ref2)); + if ((*HashPos >= (U32)(start2 - base)) || ((U32)(start2 - base) >= *HashPos + LIZARD_PRICEFAST_MIN_OFFSET)) + *HashPos = (U32)(start2 - base); + + if (!ml2) goto _Encode; + + { + int back = 0; + while ((start2+back>ip) && (ref2+back > lowPrefixPtr) && (start2[back-1] == ref2[back-1])) back--; + ml2 -= back; + start2 += back; + ref2 += back; + } + + if (ml2 <= ml) { ml2 = 0; goto _Encode; } + + if (start2 <= ip) + { + + ip = start2; ref = ref2; ml = ml2; + ml2 = 0; + goto _Encode; + } + + if (start2 - ip < 3) + { + ip = start2; ref = ref2; ml = ml2; + ml2 = 0; + goto _Search; + } + + if (start2 < ip + ml) + { + size_t correction = ml - (int)(start2 - ip); + start2 += correction; + ref2 += correction; + ml2 -= correction; + if (ml2 < 3) { ml2 = 0; } + if ((ml2 < minMatchLongOff) && ((U32)(start2 - ref2) >= LIZARD_MAX_16BIT_OFFSET)) { ml2 = 0; } + } + +_Encode: + if (Lizard_encodeSequence_LIZv1(ctx, &ip, &anchor, ml, ref)) goto _output_error; + + if (ml2) + { + ip = start2; ref = ref2; ml = ml2; + ml2 = 0; + goto _Search; + } + } + + /* Encode Last Literals */ + ip = iend; + if (Lizard_encodeLastLiterals_LIZv1(ctx, &ip, &anchor)) goto _output_error; + + /* End */ + return 1; +_output_error: + return 0; +} + diff --git a/contrib/lizard/lib/xxhash/xxhash.c b/contrib/lizard/lib/xxhash/xxhash.c new file mode 100644 index 00000000000..833b99f3b21 --- /dev/null +++ b/contrib/lizard/lib/xxhash/xxhash.c @@ -0,0 +1,888 @@ +/* +* xxHash - Fast Hash algorithm +* Copyright (C) 2012-2016, Yann Collet +* +* BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) +* +* Redistribution and use in source and binary forms, with or without +* modification, are permitted provided that the following conditions are +* met: +* +* * Redistributions of source code must retain the above copyright +* notice, this list of conditions and the following disclaimer. +* * Redistributions in binary form must reproduce the above +* copyright notice, this list of conditions and the following disclaimer +* in the documentation and/or other materials provided with the +* distribution. +* +* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +* +* You can contact the author at : +* - xxHash homepage: http://www.xxhash.com +* - xxHash source repository : https://github.com/Cyan4973/xxHash +*/ + + +/* ************************************* +* Tuning parameters +***************************************/ +/*!XXH_FORCE_MEMORY_ACCESS : + * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable. + * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal. + * The below switch allow to select different access method for improved performance. + * Method 0 (default) : use `memcpy()`. Safe and portable. + * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable). + * This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`. + * Method 2 : direct access. This method doesn't depend on compiler but violate C standard. + * It can generate buggy code on targets which do not support unaligned memory accesses. + * But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6) + * See http://stackoverflow.com/a/32095106/646947 for details. + * Prefer these methods in priority order (0 > 1 > 2) + */ +#ifndef XXH_FORCE_MEMORY_ACCESS /* can be defined externally, on command line for example */ +# if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) ) +# define XXH_FORCE_MEMORY_ACCESS 2 +# elif defined(__INTEL_COMPILER) || \ + (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) )) +# define XXH_FORCE_MEMORY_ACCESS 1 +# endif +#endif + +/*!XXH_ACCEPT_NULL_INPUT_POINTER : + * If the input pointer is a null pointer, xxHash default behavior is to trigger a memory access error, since it is a bad pointer. + * When this option is enabled, xxHash output for null input pointers will be the same as a null-length input. + * By default, this option is disabled. To enable it, uncomment below define : + */ +/* #define XXH_ACCEPT_NULL_INPUT_POINTER 1 */ + +/*!XXH_FORCE_NATIVE_FORMAT : + * By default, xxHash library provides endian-independent Hash values, based on little-endian convention. + * Results are therefore identical for little-endian and big-endian CPU. + * This comes at a performance cost for big-endian CPU, since some swapping is required to emulate little-endian format. + * Should endian-independence be of no importance for your application, you may set the #define below to 1, + * to improve speed for Big-endian CPU. + * This option has no impact on Little_Endian CPU. + */ +#ifndef XXH_FORCE_NATIVE_FORMAT /* can be defined externally */ +# define XXH_FORCE_NATIVE_FORMAT 0 +#endif + +/*!XXH_FORCE_ALIGN_CHECK : + * This is a minor performance trick, only useful with lots of very small keys. + * It means : check for aligned/unaligned input. + * The check costs one initial branch per hash; + * set it to 0 when the input is guaranteed to be aligned, + * or when alignment doesn't matter for performance. + */ +#ifndef XXH_FORCE_ALIGN_CHECK /* can be defined externally */ +# if defined(__i386) || defined(_M_IX86) || defined(__x86_64__) || defined(_M_X64) +# define XXH_FORCE_ALIGN_CHECK 0 +# else +# define XXH_FORCE_ALIGN_CHECK 1 +# endif +#endif + + +/* ************************************* +* Includes & Memory related functions +***************************************/ +/*! Modify the local functions below should you wish to use some other memory routines +* for malloc(), free() */ +#include +static void* XXH_malloc(size_t s) { return malloc(s); } +static void XXH_free (void* p) { free(p); } +/*! and for memcpy() */ +#include +static void* XXH_memcpy(void* dest, const void* src, size_t size) { return memcpy(dest,src,size); } + +#define XXH_STATIC_LINKING_ONLY +#include "xxhash.h" + + +/* ************************************* +* Compiler Specific Options +***************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# define FORCE_INLINE static __forceinline +#else +# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# ifdef __GNUC__ +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +# else +# define FORCE_INLINE static +# endif /* __STDC_VERSION__ */ +#endif + + +/* ************************************* +* Basic Types +***************************************/ +#ifndef MEM_MODULE +# if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) +# include + typedef uint8_t BYTE; + typedef uint16_t U16; + typedef uint32_t U32; +# else + typedef unsigned char BYTE; + typedef unsigned short U16; + typedef unsigned int U32; +# endif +#endif + +#if (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==2)) + +/* Force direct memory access. Only works on CPU which support unaligned memory access in hardware */ +static U32 XXH_read32(const void* memPtr) { return *(const U32*) memPtr; } + +#elif (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==1)) + +/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */ +/* currently only defined for gcc and icc */ +typedef union { U32 u32; } __attribute__((packed)) unalign; +static U32 XXH_read32(const void* ptr) { return ((const unalign*)ptr)->u32; } + +#else + +/* portable and safe solution. Generally efficient. + * see : http://stackoverflow.com/a/32095106/646947 + */ +static U32 XXH_read32(const void* memPtr) +{ + U32 val; + memcpy(&val, memPtr, sizeof(val)); + return val; +} + +#endif /* XXH_FORCE_DIRECT_MEMORY_ACCESS */ + + +/* **************************************** +* Compiler-specific Functions and Macros +******************************************/ +#define XXH_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) + +/* Note : although _rotl exists for minGW (GCC under windows), performance seems poor */ +#if defined(_MSC_VER) +# define XXH_rotl32(x,r) _rotl(x,r) +# define XXH_rotl64(x,r) _rotl64(x,r) +#else +# define XXH_rotl32(x,r) ((x << r) | (x >> (32 - r))) +# define XXH_rotl64(x,r) ((x << r) | (x >> (64 - r))) +#endif + +#if defined(_MSC_VER) /* Visual Studio */ +# define XXH_swap32 _byteswap_ulong +#elif XXH_GCC_VERSION >= 403 +# define XXH_swap32 __builtin_bswap32 +#else +static U32 XXH_swap32 (U32 x) +{ + return ((x << 24) & 0xff000000 ) | + ((x << 8) & 0x00ff0000 ) | + ((x >> 8) & 0x0000ff00 ) | + ((x >> 24) & 0x000000ff ); +} +#endif + + +/* ************************************* +* Architecture Macros +***************************************/ +typedef enum { XXH_bigEndian=0, XXH_littleEndian=1 } XXH_endianess; + +/* XXH_CPU_LITTLE_ENDIAN can be defined externally, for example on the compiler command line */ +#ifndef XXH_CPU_LITTLE_ENDIAN + static const int g_one = 1; +# define XXH_CPU_LITTLE_ENDIAN (*(const char*)(&g_one)) +#endif + + +/* *************************** +* Memory reads +*****************************/ +typedef enum { XXH_aligned, XXH_unaligned } XXH_alignment; + +FORCE_INLINE U32 XXH_readLE32_align(const void* ptr, XXH_endianess endian, XXH_alignment align) +{ + if (align==XXH_unaligned) + return endian==XXH_littleEndian ? XXH_read32(ptr) : XXH_swap32(XXH_read32(ptr)); + else + return endian==XXH_littleEndian ? *(const U32*)ptr : XXH_swap32(*(const U32*)ptr); +} + +FORCE_INLINE U32 XXH_readLE32(const void* ptr, XXH_endianess endian) +{ + return XXH_readLE32_align(ptr, endian, XXH_unaligned); +} + +static U32 XXH_readBE32(const void* ptr) +{ + return XXH_CPU_LITTLE_ENDIAN ? XXH_swap32(XXH_read32(ptr)) : XXH_read32(ptr); +} + + +/* ************************************* +* Macros +***************************************/ +#define XXH_STATIC_ASSERT(c) { enum { XXH_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ +XXH_PUBLIC_API unsigned XXH_versionNumber (void) { return XXH_VERSION_NUMBER; } + + +/* ******************************************************************* +* 32-bits hash functions +*********************************************************************/ +static const U32 PRIME32_1 = 2654435761U; +static const U32 PRIME32_2 = 2246822519U; +static const U32 PRIME32_3 = 3266489917U; +static const U32 PRIME32_4 = 668265263U; +static const U32 PRIME32_5 = 374761393U; + +static U32 XXH32_round(U32 seed, U32 input) +{ + seed += input * PRIME32_2; + seed = XXH_rotl32(seed, 13); + seed *= PRIME32_1; + return seed; +} + +FORCE_INLINE U32 XXH32_endian_align(const void* input, size_t len, U32 seed, XXH_endianess endian, XXH_alignment align) +{ + const BYTE* p = (const BYTE*)input; + const BYTE* bEnd = p + len; + U32 h32; +#define XXH_get32bits(p) XXH_readLE32_align(p, endian, align) + +#ifdef XXH_ACCEPT_NULL_INPUT_POINTER + if (p==NULL) { + len=0; + bEnd=p=(const BYTE*)(size_t)16; + } +#endif + + if (len>=16) { + const BYTE* const limit = bEnd - 16; + U32 v1 = seed + PRIME32_1 + PRIME32_2; + U32 v2 = seed + PRIME32_2; + U32 v3 = seed + 0; + U32 v4 = seed - PRIME32_1; + + do { + v1 = XXH32_round(v1, XXH_get32bits(p)); p+=4; + v2 = XXH32_round(v2, XXH_get32bits(p)); p+=4; + v3 = XXH32_round(v3, XXH_get32bits(p)); p+=4; + v4 = XXH32_round(v4, XXH_get32bits(p)); p+=4; + } while (p<=limit); + + h32 = XXH_rotl32(v1, 1) + XXH_rotl32(v2, 7) + XXH_rotl32(v3, 12) + XXH_rotl32(v4, 18); + } else { + h32 = seed + PRIME32_5; + } + + h32 += (U32) len; + + while (p+4<=bEnd) { + h32 += XXH_get32bits(p) * PRIME32_3; + h32 = XXH_rotl32(h32, 17) * PRIME32_4 ; + p+=4; + } + + while (p> 15; + h32 *= PRIME32_2; + h32 ^= h32 >> 13; + h32 *= PRIME32_3; + h32 ^= h32 >> 16; + + return h32; +} + + +XXH_PUBLIC_API unsigned int XXH32 (const void* input, size_t len, unsigned int seed) +{ +#if 0 + /* Simple version, good for code maintenance, but unfortunately slow for small inputs */ + XXH32_state_t state; + XXH32_reset(&state, seed); + XXH32_update(&state, input, len); + return XXH32_digest(&state); +#else + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if (XXH_FORCE_ALIGN_CHECK) { + if ((((size_t)input) & 3) == 0) { /* Input is 4-bytes aligned, leverage the speed benefit */ + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned); + else + return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned); + } } + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned); + else + return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned); +#endif +} + + + +/*====== Hash streaming ======*/ + +XXH_PUBLIC_API XXH32_state_t* XXH32_createState(void) +{ + return (XXH32_state_t*)XXH_malloc(sizeof(XXH32_state_t)); +} +XXH_PUBLIC_API XXH_errorcode XXH32_freeState(XXH32_state_t* statePtr) +{ + XXH_free(statePtr); + return XXH_OK; +} + +XXH_PUBLIC_API void XXH32_copyState(XXH32_state_t* dstState, const XXH32_state_t* srcState) +{ + memcpy(dstState, srcState, sizeof(*dstState)); +} + +XXH_PUBLIC_API XXH_errorcode XXH32_reset(XXH32_state_t* statePtr, unsigned int seed) +{ + XXH32_state_t state; /* using a local state to memcpy() in order to avoid strict-aliasing warnings */ + memset(&state, 0, sizeof(state)-4); /* do not write into reserved, for future removal */ + state.v1 = seed + PRIME32_1 + PRIME32_2; + state.v2 = seed + PRIME32_2; + state.v3 = seed + 0; + state.v4 = seed - PRIME32_1; + memcpy(statePtr, &state, sizeof(state)); + return XXH_OK; +} + + +FORCE_INLINE XXH_errorcode XXH32_update_endian (XXH32_state_t* state, const void* input, size_t len, XXH_endianess endian) +{ + const BYTE* p = (const BYTE*)input; + const BYTE* const bEnd = p + len; + +#ifdef XXH_ACCEPT_NULL_INPUT_POINTER + if (input==NULL) return XXH_ERROR; +#endif + + state->total_len_32 += (unsigned)len; + state->large_len |= (len>=16) | (state->total_len_32>=16); + + if (state->memsize + len < 16) { /* fill in tmp buffer */ + XXH_memcpy((BYTE*)(state->mem32) + state->memsize, input, len); + state->memsize += (unsigned)len; + return XXH_OK; + } + + if (state->memsize) { /* some data left from previous update */ + XXH_memcpy((BYTE*)(state->mem32) + state->memsize, input, 16-state->memsize); + { const U32* p32 = state->mem32; + state->v1 = XXH32_round(state->v1, XXH_readLE32(p32, endian)); p32++; + state->v2 = XXH32_round(state->v2, XXH_readLE32(p32, endian)); p32++; + state->v3 = XXH32_round(state->v3, XXH_readLE32(p32, endian)); p32++; + state->v4 = XXH32_round(state->v4, XXH_readLE32(p32, endian)); + } + p += 16-state->memsize; + state->memsize = 0; + } + + if (p <= bEnd-16) { + const BYTE* const limit = bEnd - 16; + U32 v1 = state->v1; + U32 v2 = state->v2; + U32 v3 = state->v3; + U32 v4 = state->v4; + + do { + v1 = XXH32_round(v1, XXH_readLE32(p, endian)); p+=4; + v2 = XXH32_round(v2, XXH_readLE32(p, endian)); p+=4; + v3 = XXH32_round(v3, XXH_readLE32(p, endian)); p+=4; + v4 = XXH32_round(v4, XXH_readLE32(p, endian)); p+=4; + } while (p<=limit); + + state->v1 = v1; + state->v2 = v2; + state->v3 = v3; + state->v4 = v4; + } + + if (p < bEnd) { + XXH_memcpy(state->mem32, p, (size_t)(bEnd-p)); + state->memsize = (unsigned)(bEnd-p); + } + + return XXH_OK; +} + +XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* state_in, const void* input, size_t len) +{ + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_update_endian(state_in, input, len, XXH_littleEndian); + else + return XXH32_update_endian(state_in, input, len, XXH_bigEndian); +} + + + +FORCE_INLINE U32 XXH32_digest_endian (const XXH32_state_t* state, XXH_endianess endian) +{ + const BYTE * p = (const BYTE*)state->mem32; + const BYTE* const bEnd = (const BYTE*)(state->mem32) + state->memsize; + U32 h32; + + if (state->large_len) { + h32 = XXH_rotl32(state->v1, 1) + XXH_rotl32(state->v2, 7) + XXH_rotl32(state->v3, 12) + XXH_rotl32(state->v4, 18); + } else { + h32 = state->v3 /* == seed */ + PRIME32_5; + } + + h32 += state->total_len_32; + + while (p+4<=bEnd) { + h32 += XXH_readLE32(p, endian) * PRIME32_3; + h32 = XXH_rotl32(h32, 17) * PRIME32_4; + p+=4; + } + + while (p> 15; + h32 *= PRIME32_2; + h32 ^= h32 >> 13; + h32 *= PRIME32_3; + h32 ^= h32 >> 16; + + return h32; +} + + +XXH_PUBLIC_API unsigned int XXH32_digest (const XXH32_state_t* state_in) +{ + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_digest_endian(state_in, XXH_littleEndian); + else + return XXH32_digest_endian(state_in, XXH_bigEndian); +} + + +/*====== Canonical representation ======*/ + +/*! Default XXH result types are basic unsigned 32 and 64 bits. +* The canonical representation follows human-readable write convention, aka big-endian (large digits first). +* These functions allow transformation of hash result into and from its canonical format. +* This way, hash values can be written into a file or buffer, and remain comparable across different systems and programs. +*/ + +XXH_PUBLIC_API void XXH32_canonicalFromHash(XXH32_canonical_t* dst, XXH32_hash_t hash) +{ + XXH_STATIC_ASSERT(sizeof(XXH32_canonical_t) == sizeof(XXH32_hash_t)); + if (XXH_CPU_LITTLE_ENDIAN) hash = XXH_swap32(hash); + memcpy(dst, &hash, sizeof(*dst)); +} + +XXH_PUBLIC_API XXH32_hash_t XXH32_hashFromCanonical(const XXH32_canonical_t* src) +{ + return XXH_readBE32(src); +} + + +#ifndef XXH_NO_LONG_LONG + +/* ******************************************************************* +* 64-bits hash functions +*********************************************************************/ + +/*====== Memory access ======*/ + +#ifndef MEM_MODULE +# define MEM_MODULE +# if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) +# include + typedef uint64_t U64; +# else + typedef unsigned long long U64; /* if your compiler doesn't support unsigned long long, replace by another 64-bit type here. Note that xxhash.h will also need to be updated. */ +# endif +#endif + + +#if (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==2)) + +/* Force direct memory access. Only works on CPU which support unaligned memory access in hardware */ +static U64 XXH_read64(const void* memPtr) { return *(const U64*) memPtr; } + +#elif (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==1)) + +/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */ +/* currently only defined for gcc and icc */ +typedef union { U32 u32; U64 u64; } __attribute__((packed)) unalign64; +static U64 XXH_read64(const void* ptr) { return ((const unalign64*)ptr)->u64; } + +#else + +/* portable and safe solution. Generally efficient. + * see : http://stackoverflow.com/a/32095106/646947 + */ + +static U64 XXH_read64(const void* memPtr) +{ + U64 val; + memcpy(&val, memPtr, sizeof(val)); + return val; +} + +#endif /* XXH_FORCE_DIRECT_MEMORY_ACCESS */ + +#if defined(_MSC_VER) /* Visual Studio */ +# define XXH_swap64 _byteswap_uint64 +#elif XXH_GCC_VERSION >= 403 +# define XXH_swap64 __builtin_bswap64 +#else +static U64 XXH_swap64 (U64 x) +{ + return ((x << 56) & 0xff00000000000000ULL) | + ((x << 40) & 0x00ff000000000000ULL) | + ((x << 24) & 0x0000ff0000000000ULL) | + ((x << 8) & 0x000000ff00000000ULL) | + ((x >> 8) & 0x00000000ff000000ULL) | + ((x >> 24) & 0x0000000000ff0000ULL) | + ((x >> 40) & 0x000000000000ff00ULL) | + ((x >> 56) & 0x00000000000000ffULL); +} +#endif + +FORCE_INLINE U64 XXH_readLE64_align(const void* ptr, XXH_endianess endian, XXH_alignment align) +{ + if (align==XXH_unaligned) + return endian==XXH_littleEndian ? XXH_read64(ptr) : XXH_swap64(XXH_read64(ptr)); + else + return endian==XXH_littleEndian ? *(const U64*)ptr : XXH_swap64(*(const U64*)ptr); +} + +FORCE_INLINE U64 XXH_readLE64(const void* ptr, XXH_endianess endian) +{ + return XXH_readLE64_align(ptr, endian, XXH_unaligned); +} + +static U64 XXH_readBE64(const void* ptr) +{ + return XXH_CPU_LITTLE_ENDIAN ? XXH_swap64(XXH_read64(ptr)) : XXH_read64(ptr); +} + + +/*====== xxh64 ======*/ + +static const U64 PRIME64_1 = 11400714785074694791ULL; +static const U64 PRIME64_2 = 14029467366897019727ULL; +static const U64 PRIME64_3 = 1609587929392839161ULL; +static const U64 PRIME64_4 = 9650029242287828579ULL; +static const U64 PRIME64_5 = 2870177450012600261ULL; + +static U64 XXH64_round(U64 acc, U64 input) +{ + acc += input * PRIME64_2; + acc = XXH_rotl64(acc, 31); + acc *= PRIME64_1; + return acc; +} + +static U64 XXH64_mergeRound(U64 acc, U64 val) +{ + val = XXH64_round(0, val); + acc ^= val; + acc = acc * PRIME64_1 + PRIME64_4; + return acc; +} + +FORCE_INLINE U64 XXH64_endian_align(const void* input, size_t len, U64 seed, XXH_endianess endian, XXH_alignment align) +{ + const BYTE* p = (const BYTE*)input; + const BYTE* bEnd = p + len; + U64 h64; +#define XXH_get64bits(p) XXH_readLE64_align(p, endian, align) + +#ifdef XXH_ACCEPT_NULL_INPUT_POINTER + if (p==NULL) { + len=0; + bEnd=p=(const BYTE*)(size_t)32; + } +#endif + + if (len>=32) { + const BYTE* const limit = bEnd - 32; + U64 v1 = seed + PRIME64_1 + PRIME64_2; + U64 v2 = seed + PRIME64_2; + U64 v3 = seed + 0; + U64 v4 = seed - PRIME64_1; + + do { + v1 = XXH64_round(v1, XXH_get64bits(p)); p+=8; + v2 = XXH64_round(v2, XXH_get64bits(p)); p+=8; + v3 = XXH64_round(v3, XXH_get64bits(p)); p+=8; + v4 = XXH64_round(v4, XXH_get64bits(p)); p+=8; + } while (p<=limit); + + h64 = XXH_rotl64(v1, 1) + XXH_rotl64(v2, 7) + XXH_rotl64(v3, 12) + XXH_rotl64(v4, 18); + h64 = XXH64_mergeRound(h64, v1); + h64 = XXH64_mergeRound(h64, v2); + h64 = XXH64_mergeRound(h64, v3); + h64 = XXH64_mergeRound(h64, v4); + + } else { + h64 = seed + PRIME64_5; + } + + h64 += (U64) len; + + while (p+8<=bEnd) { + U64 const k1 = XXH64_round(0, XXH_get64bits(p)); + h64 ^= k1; + h64 = XXH_rotl64(h64,27) * PRIME64_1 + PRIME64_4; + p+=8; + } + + if (p+4<=bEnd) { + h64 ^= (U64)(XXH_get32bits(p)) * PRIME64_1; + h64 = XXH_rotl64(h64, 23) * PRIME64_2 + PRIME64_3; + p+=4; + } + + while (p> 33; + h64 *= PRIME64_2; + h64 ^= h64 >> 29; + h64 *= PRIME64_3; + h64 ^= h64 >> 32; + + return h64; +} + + +XXH_PUBLIC_API unsigned long long XXH64 (const void* input, size_t len, unsigned long long seed) +{ +#if 0 + /* Simple version, good for code maintenance, but unfortunately slow for small inputs */ + XXH64_state_t state; + XXH64_reset(&state, seed); + XXH64_update(&state, input, len); + return XXH64_digest(&state); +#else + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if (XXH_FORCE_ALIGN_CHECK) { + if ((((size_t)input) & 7)==0) { /* Input is aligned, let's leverage the speed advantage */ + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH64_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned); + else + return XXH64_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned); + } } + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH64_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned); + else + return XXH64_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned); +#endif +} + +/*====== Hash Streaming ======*/ + +XXH_PUBLIC_API XXH64_state_t* XXH64_createState(void) +{ + return (XXH64_state_t*)XXH_malloc(sizeof(XXH64_state_t)); +} +XXH_PUBLIC_API XXH_errorcode XXH64_freeState(XXH64_state_t* statePtr) +{ + XXH_free(statePtr); + return XXH_OK; +} + +XXH_PUBLIC_API void XXH64_copyState(XXH64_state_t* dstState, const XXH64_state_t* srcState) +{ + memcpy(dstState, srcState, sizeof(*dstState)); +} + +XXH_PUBLIC_API XXH_errorcode XXH64_reset(XXH64_state_t* statePtr, unsigned long long seed) +{ + XXH64_state_t state; /* using a local state to memcpy() in order to avoid strict-aliasing warnings */ + memset(&state, 0, sizeof(state)-8); /* do not write into reserved, for future removal */ + state.v1 = seed + PRIME64_1 + PRIME64_2; + state.v2 = seed + PRIME64_2; + state.v3 = seed + 0; + state.v4 = seed - PRIME64_1; + memcpy(statePtr, &state, sizeof(state)); + return XXH_OK; +} + +FORCE_INLINE XXH_errorcode XXH64_update_endian (XXH64_state_t* state, const void* input, size_t len, XXH_endianess endian) +{ + const BYTE* p = (const BYTE*)input; + const BYTE* const bEnd = p + len; + +#ifdef XXH_ACCEPT_NULL_INPUT_POINTER + if (input==NULL) return XXH_ERROR; +#endif + + state->total_len += len; + + if (state->memsize + len < 32) { /* fill in tmp buffer */ + XXH_memcpy(((BYTE*)state->mem64) + state->memsize, input, len); + state->memsize += (U32)len; + return XXH_OK; + } + + if (state->memsize) { /* tmp buffer is full */ + XXH_memcpy(((BYTE*)state->mem64) + state->memsize, input, 32-state->memsize); + state->v1 = XXH64_round(state->v1, XXH_readLE64(state->mem64+0, endian)); + state->v2 = XXH64_round(state->v2, XXH_readLE64(state->mem64+1, endian)); + state->v3 = XXH64_round(state->v3, XXH_readLE64(state->mem64+2, endian)); + state->v4 = XXH64_round(state->v4, XXH_readLE64(state->mem64+3, endian)); + p += 32-state->memsize; + state->memsize = 0; + } + + if (p+32 <= bEnd) { + const BYTE* const limit = bEnd - 32; + U64 v1 = state->v1; + U64 v2 = state->v2; + U64 v3 = state->v3; + U64 v4 = state->v4; + + do { + v1 = XXH64_round(v1, XXH_readLE64(p, endian)); p+=8; + v2 = XXH64_round(v2, XXH_readLE64(p, endian)); p+=8; + v3 = XXH64_round(v3, XXH_readLE64(p, endian)); p+=8; + v4 = XXH64_round(v4, XXH_readLE64(p, endian)); p+=8; + } while (p<=limit); + + state->v1 = v1; + state->v2 = v2; + state->v3 = v3; + state->v4 = v4; + } + + if (p < bEnd) { + XXH_memcpy(state->mem64, p, (size_t)(bEnd-p)); + state->memsize = (unsigned)(bEnd-p); + } + + return XXH_OK; +} + +XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* state_in, const void* input, size_t len) +{ + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH64_update_endian(state_in, input, len, XXH_littleEndian); + else + return XXH64_update_endian(state_in, input, len, XXH_bigEndian); +} + +FORCE_INLINE U64 XXH64_digest_endian (const XXH64_state_t* state, XXH_endianess endian) +{ + const BYTE * p = (const BYTE*)state->mem64; + const BYTE* const bEnd = (const BYTE*)state->mem64 + state->memsize; + U64 h64; + + if (state->total_len >= 32) { + U64 const v1 = state->v1; + U64 const v2 = state->v2; + U64 const v3 = state->v3; + U64 const v4 = state->v4; + + h64 = XXH_rotl64(v1, 1) + XXH_rotl64(v2, 7) + XXH_rotl64(v3, 12) + XXH_rotl64(v4, 18); + h64 = XXH64_mergeRound(h64, v1); + h64 = XXH64_mergeRound(h64, v2); + h64 = XXH64_mergeRound(h64, v3); + h64 = XXH64_mergeRound(h64, v4); + } else { + h64 = state->v3 + PRIME64_5; + } + + h64 += (U64) state->total_len; + + while (p+8<=bEnd) { + U64 const k1 = XXH64_round(0, XXH_readLE64(p, endian)); + h64 ^= k1; + h64 = XXH_rotl64(h64,27) * PRIME64_1 + PRIME64_4; + p+=8; + } + + if (p+4<=bEnd) { + h64 ^= (U64)(XXH_readLE32(p, endian)) * PRIME64_1; + h64 = XXH_rotl64(h64, 23) * PRIME64_2 + PRIME64_3; + p+=4; + } + + while (p> 33; + h64 *= PRIME64_2; + h64 ^= h64 >> 29; + h64 *= PRIME64_3; + h64 ^= h64 >> 32; + + return h64; +} + +XXH_PUBLIC_API unsigned long long XXH64_digest (const XXH64_state_t* state_in) +{ + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH64_digest_endian(state_in, XXH_littleEndian); + else + return XXH64_digest_endian(state_in, XXH_bigEndian); +} + + +/*====== Canonical representation ======*/ + +XXH_PUBLIC_API void XXH64_canonicalFromHash(XXH64_canonical_t* dst, XXH64_hash_t hash) +{ + XXH_STATIC_ASSERT(sizeof(XXH64_canonical_t) == sizeof(XXH64_hash_t)); + if (XXH_CPU_LITTLE_ENDIAN) hash = XXH_swap64(hash); + memcpy(dst, &hash, sizeof(*dst)); +} + +XXH_PUBLIC_API XXH64_hash_t XXH64_hashFromCanonical(const XXH64_canonical_t* src) +{ + return XXH_readBE64(src); +} + +#endif /* XXH_NO_LONG_LONG */ diff --git a/contrib/lizard/lib/xxhash/xxhash.h b/contrib/lizard/lib/xxhash/xxhash.h new file mode 100644 index 00000000000..9d831e03b35 --- /dev/null +++ b/contrib/lizard/lib/xxhash/xxhash.h @@ -0,0 +1,293 @@ +/* + xxHash - Extremely Fast Hash algorithm + Header File + Copyright (C) 2012-2016, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - xxHash source repository : https://github.com/Cyan4973/xxHash +*/ + +/* Notice extracted from xxHash homepage : + +xxHash is an extremely fast Hash algorithm, running at RAM speed limits. +It also successfully passes all tests from the SMHasher suite. + +Comparison (single thread, Windows Seven 32 bits, using SMHasher on a Core 2 Duo @3GHz) + +Name Speed Q.Score Author +xxHash 5.4 GB/s 10 +CrapWow 3.2 GB/s 2 Andrew +MumurHash 3a 2.7 GB/s 10 Austin Appleby +SpookyHash 2.0 GB/s 10 Bob Jenkins +SBox 1.4 GB/s 9 Bret Mulvey +Lookup3 1.2 GB/s 9 Bob Jenkins +SuperFastHash 1.2 GB/s 1 Paul Hsieh +CityHash64 1.05 GB/s 10 Pike & Alakuijala +FNV 0.55 GB/s 5 Fowler, Noll, Vo +CRC32 0.43 GB/s 9 +MD5-32 0.33 GB/s 10 Ronald L. Rivest +SHA1-32 0.28 GB/s 10 + +Q.Score is a measure of quality of the hash function. +It depends on successfully passing SMHasher test set. +10 is a perfect score. + +A 64-bits version, named XXH64, is available since r35. +It offers much better speed, but for 64-bits applications only. +Name Speed on 64 bits Speed on 32 bits +XXH64 13.8 GB/s 1.9 GB/s +XXH32 6.8 GB/s 6.0 GB/s +*/ + +#ifndef XXHASH_H_5627135585666179 +#define XXHASH_H_5627135585666179 1 + +#if defined (__cplusplus) +extern "C" { +#endif + + +/* **************************** +* Definitions +******************************/ +#include /* size_t */ +typedef enum { XXH_OK=0, XXH_ERROR } XXH_errorcode; + + +/* **************************** +* API modifier +******************************/ +/** XXH_PRIVATE_API +* This is useful to include xxhash functions in `static` mode +* in order to inline them, and remove their symbol from the public list. +* Methodology : +* #define XXH_PRIVATE_API +* #include "xxhash.h" +* `xxhash.c` is automatically included. +* It's not useful to compile and link it as a separate module. +*/ +#ifdef XXH_PRIVATE_API +# ifndef XXH_STATIC_LINKING_ONLY +# define XXH_STATIC_LINKING_ONLY +# endif +# if defined(__GNUC__) +# define XXH_PUBLIC_API static __inline __attribute__((unused)) +# elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) +# define XXH_PUBLIC_API static inline +# elif defined(_MSC_VER) +# define XXH_PUBLIC_API static __inline +# else +# define XXH_PUBLIC_API static /* this version may generate warnings for unused static functions; disable the relevant warning */ +# endif +#else +# define XXH_PUBLIC_API /* do nothing */ +#endif /* XXH_PRIVATE_API */ + +/*!XXH_NAMESPACE, aka Namespace Emulation : + +If you want to include _and expose_ xxHash functions from within your own library, +but also want to avoid symbol collisions with other libraries which may also include xxHash, + +you can use XXH_NAMESPACE, to automatically prefix any public symbol from xxhash library +with the value of XXH_NAMESPACE (therefore, avoid NULL and numeric values). + +Note that no change is required within the calling program as long as it includes `xxhash.h` : +regular symbol name will be automatically translated by this header. +*/ +#ifdef XXH_NAMESPACE +# define XXH_CAT(A,B) A##B +# define XXH_NAME2(A,B) XXH_CAT(A,B) +# define XXH_versionNumber XXH_NAME2(XXH_NAMESPACE, XXH_versionNumber) +# define XXH32 XXH_NAME2(XXH_NAMESPACE, XXH32) +# define XXH32_createState XXH_NAME2(XXH_NAMESPACE, XXH32_createState) +# define XXH32_freeState XXH_NAME2(XXH_NAMESPACE, XXH32_freeState) +# define XXH32_reset XXH_NAME2(XXH_NAMESPACE, XXH32_reset) +# define XXH32_update XXH_NAME2(XXH_NAMESPACE, XXH32_update) +# define XXH32_digest XXH_NAME2(XXH_NAMESPACE, XXH32_digest) +# define XXH32_copyState XXH_NAME2(XXH_NAMESPACE, XXH32_copyState) +# define XXH32_canonicalFromHash XXH_NAME2(XXH_NAMESPACE, XXH32_canonicalFromHash) +# define XXH32_hashFromCanonical XXH_NAME2(XXH_NAMESPACE, XXH32_hashFromCanonical) +# define XXH64 XXH_NAME2(XXH_NAMESPACE, XXH64) +# define XXH64_createState XXH_NAME2(XXH_NAMESPACE, XXH64_createState) +# define XXH64_freeState XXH_NAME2(XXH_NAMESPACE, XXH64_freeState) +# define XXH64_reset XXH_NAME2(XXH_NAMESPACE, XXH64_reset) +# define XXH64_update XXH_NAME2(XXH_NAMESPACE, XXH64_update) +# define XXH64_digest XXH_NAME2(XXH_NAMESPACE, XXH64_digest) +# define XXH64_copyState XXH_NAME2(XXH_NAMESPACE, XXH64_copyState) +# define XXH64_canonicalFromHash XXH_NAME2(XXH_NAMESPACE, XXH64_canonicalFromHash) +# define XXH64_hashFromCanonical XXH_NAME2(XXH_NAMESPACE, XXH64_hashFromCanonical) +#endif + + +/* ************************************* +* Version +***************************************/ +#define XXH_VERSION_MAJOR 0 +#define XXH_VERSION_MINOR 6 +#define XXH_VERSION_RELEASE 2 +#define XXH_VERSION_NUMBER (XXH_VERSION_MAJOR *100*100 + XXH_VERSION_MINOR *100 + XXH_VERSION_RELEASE) +XXH_PUBLIC_API unsigned XXH_versionNumber (void); + + +/*-********************************************************************** +* 32-bits hash +************************************************************************/ +typedef unsigned int XXH32_hash_t; + +/*! XXH32() : + Calculate the 32-bits hash of sequence "length" bytes stored at memory address "input". + The memory between input & input+length must be valid (allocated and read-accessible). + "seed" can be used to alter the result predictably. + Speed on Core 2 Duo @ 3 GHz (single thread, SMHasher benchmark) : 5.4 GB/s */ +XXH_PUBLIC_API XXH32_hash_t XXH32 (const void* input, size_t length, unsigned int seed); + +/*====== Streaming ======*/ +typedef struct XXH32_state_s XXH32_state_t; /* incomplete type */ +XXH_PUBLIC_API XXH32_state_t* XXH32_createState(void); +XXH_PUBLIC_API XXH_errorcode XXH32_freeState(XXH32_state_t* statePtr); +XXH_PUBLIC_API void XXH32_copyState(XXH32_state_t* dst_state, const XXH32_state_t* src_state); + +XXH_PUBLIC_API XXH_errorcode XXH32_reset (XXH32_state_t* statePtr, unsigned int seed); +XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* statePtr, const void* input, size_t length); +XXH_PUBLIC_API XXH32_hash_t XXH32_digest (const XXH32_state_t* statePtr); + +/* +These functions generate the xxHash of an input provided in multiple segments. +Note that, for small input, they are slower than single-call functions, due to state management. +For small input, prefer `XXH32()` and `XXH64()` . + +XXH state must first be allocated, using XXH*_createState() . + +Start a new hash by initializing state with a seed, using XXH*_reset(). + +Then, feed the hash state by calling XXH*_update() as many times as necessary. +Obviously, input must be allocated and read accessible. +The function returns an error code, with 0 meaning OK, and any other value meaning there is an error. + +Finally, a hash value can be produced anytime, by using XXH*_digest(). +This function returns the nn-bits hash as an int or long long. + +It's still possible to continue inserting input into the hash state after a digest, +and generate some new hashes later on, by calling again XXH*_digest(). + +When done, free XXH state space if it was allocated dynamically. +*/ + +/*====== Canonical representation ======*/ + +typedef struct { unsigned char digest[4]; } XXH32_canonical_t; +XXH_PUBLIC_API void XXH32_canonicalFromHash(XXH32_canonical_t* dst, XXH32_hash_t hash); +XXH_PUBLIC_API XXH32_hash_t XXH32_hashFromCanonical(const XXH32_canonical_t* src); + +/* Default result type for XXH functions are primitive unsigned 32 and 64 bits. +* The canonical representation uses human-readable write convention, aka big-endian (large digits first). +* These functions allow transformation of hash result into and from its canonical format. +* This way, hash values can be written into a file / memory, and remain comparable on different systems and programs. +*/ + + +#ifndef XXH_NO_LONG_LONG +/*-********************************************************************** +* 64-bits hash +************************************************************************/ +typedef unsigned long long XXH64_hash_t; + +/*! XXH64() : + Calculate the 64-bits hash of sequence of length "len" stored at memory address "input". + "seed" can be used to alter the result predictably. + This function runs faster on 64-bits systems, but slower on 32-bits systems (see benchmark). +*/ +XXH_PUBLIC_API XXH64_hash_t XXH64 (const void* input, size_t length, unsigned long long seed); + +/*====== Streaming ======*/ +typedef struct XXH64_state_s XXH64_state_t; /* incomplete type */ +XXH_PUBLIC_API XXH64_state_t* XXH64_createState(void); +XXH_PUBLIC_API XXH_errorcode XXH64_freeState(XXH64_state_t* statePtr); +XXH_PUBLIC_API void XXH64_copyState(XXH64_state_t* dst_state, const XXH64_state_t* src_state); + +XXH_PUBLIC_API XXH_errorcode XXH64_reset (XXH64_state_t* statePtr, unsigned long long seed); +XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* statePtr, const void* input, size_t length); +XXH_PUBLIC_API XXH64_hash_t XXH64_digest (const XXH64_state_t* statePtr); + +/*====== Canonical representation ======*/ +typedef struct { unsigned char digest[8]; } XXH64_canonical_t; +XXH_PUBLIC_API void XXH64_canonicalFromHash(XXH64_canonical_t* dst, XXH64_hash_t hash); +XXH_PUBLIC_API XXH64_hash_t XXH64_hashFromCanonical(const XXH64_canonical_t* src); +#endif /* XXH_NO_LONG_LONG */ + + +#ifdef XXH_STATIC_LINKING_ONLY + +/* ================================================================================================ + This section contains definitions which are not guaranteed to remain stable. + They may change in future versions, becoming incompatible with a different version of the library. + They shall only be used with static linking. + Never use these definitions in association with dynamic linking ! +=================================================================================================== */ + +/* These definitions are only meant to make possible + static allocation of XXH state, on stack or in a struct for example. + Never use members directly. */ + +struct XXH32_state_s { + unsigned total_len_32; + unsigned large_len; + unsigned v1; + unsigned v2; + unsigned v3; + unsigned v4; + unsigned mem32[4]; /* buffer defined as U32 for alignment */ + unsigned memsize; + unsigned reserved; /* never read nor write, will be removed in a future version */ +}; /* typedef'd to XXH32_state_t */ + +#ifndef XXH_NO_LONG_LONG /* remove 64-bits support */ +struct XXH64_state_s { + unsigned long long total_len; + unsigned long long v1; + unsigned long long v2; + unsigned long long v3; + unsigned long long v4; + unsigned long long mem64[4]; /* buffer defined as U64 for alignment */ + unsigned memsize; + unsigned reserved[2]; /* never read nor write, will be removed in a future version */ +}; /* typedef'd to XXH64_state_t */ +#endif + +#ifdef XXH_PRIVATE_API +# include "xxhash.c" /* include xxhash function bodies as `static`, for inlining */ +#endif + +#endif /* XXH_STATIC_LINKING_ONLY */ + + +#if defined (__cplusplus) +} +#endif + +#endif /* XXHASH_H_5627135585666179 */ diff --git a/contrib/lizard/programs/.gitignore b/contrib/lizard/programs/.gitignore new file mode 100644 index 00000000000..c2bfcafa9ee --- /dev/null +++ b/contrib/lizard/programs/.gitignore @@ -0,0 +1,15 @@ +# local binary (Makefile) +lizard +lizardc32 +datagen +frametest +frametest32 +fullbench +fullbench32 +fuzzer +fuzzer32 +*.exe + +# tests files +tmp* +_* \ No newline at end of file diff --git a/contrib/lizard/programs/COPYING b/contrib/lizard/programs/COPYING new file mode 100644 index 00000000000..d159169d105 --- /dev/null +++ b/contrib/lizard/programs/COPYING @@ -0,0 +1,339 @@ + GNU GENERAL PUBLIC LICENSE + Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +License is intended to guarantee your freedom to share and change free +software--to make sure the software is free for all its users. This +General Public License applies to most of the Free Software +Foundation's software and to any other program whose authors commit to +using it. (Some other Free Software Foundation software is covered by +the GNU Lesser General Public License instead.) You can apply it to +your programs, too. + + When we speak of free software, we are referring to freedom, not +price. Our General Public Licenses are designed to make sure that you +have the freedom to distribute copies of free software (and charge for +this service if you wish), that you receive source code or can get it +if you want it, that you can change the software or use pieces of it +in new free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid +anyone to deny you these rights or to ask you to surrender the rights. +These restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether +gratis or for a fee, you must give the recipients all the rights that +you have. You must make sure that they, too, receive or can get the +source code. And you must show them these terms so they know their +rights. + + We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + + Finally, any free program is threatened constantly by software +patents. We wish to avoid the danger that redistributors of a free +program will individually obtain patent licenses, in effect making the +program proprietary. To prevent this, we have made it clear that any +patent must be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and +modification follow. + + GNU GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains +a notice placed by the copyright holder saying it may be distributed +under the terms of this General Public License. The "Program", below, +refers to any such program or work, and a "work based on the Program" +means either the Program or any derivative work under copyright law: +that is to say, a work containing the Program or a portion of it, +either verbatim or with modifications and/or translated into another +language. (Hereinafter, translation is included without limitation in +the term "modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running the Program is not restricted, and the output from the Program +is covered only if its contents constitute a work based on the +Program (independent of having been made by running the Program). +Whether that is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's +source code as you receive it, in any medium, provided that you +conspicuously and appropriately publish on each copy an appropriate +copyright notice and disclaimer of warranty; keep intact all the +notices that refer to this License and to the absence of any warranty; +and give any other recipients of the Program a copy of this License +along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion +of it, thus forming a work based on the Program, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any + part thereof, to be licensed as a whole at no charge to all third + parties under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a + notice that there is no warranty (or else, saying that you provide + a warranty) and that users may redistribute the program under + these conditions, and telling the user how to view a copy of this + License. (Exception: if the Program itself is interactive but + does not normally print such an announcement, your work based on + the Program is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Program, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections + 1 and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your + cost of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer + to distribute corresponding source code. (This alternative is + allowed only for noncommercial distribution and only if you + received the program in object code or executable form with such + an offer, in accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source +code means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to +control compilation and installation of the executable. However, as a +special exception, the source code distributed need not include +anything that is normally distributed (in either source or binary +form) with the major components (compiler, kernel, and so on) of the +operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering +access to copy from a designated place, then offering equivalent +access to copy the source code from the same place counts as +distribution of the source code, even though third parties are not +compelled to copy the source along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt +otherwise to copy, modify, sublicense or distribute the Program is +void, and will automatically terminate your rights under this License. +However, parties who have received copies, or rights, from you under +this License will not have their licenses terminated so long as such +parties remain in full compliance. + + 5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties to +this License. + + 7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Program at all. For example, if a patent +license would not permit royalty-free redistribution of the Program by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License +may add an explicit geographical distribution limitation excluding +those countries, so that distribution is permitted only in or among +countries not thus excluded. In such case, this License incorporates +the limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new versions +of the General Public License from time to time. Such new versions will +be similar in spirit to the present version, but may differ in detail to +address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and conditions +either of that version or of any later version published by the Free +Software Foundation. If the Program does not specify a version number of +this License, you may choose any version ever published by the Free Software +Foundation. + + 10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the author +to ask for permission. For software which is copyrighted by the Free +Software Foundation, write to the Free Software Foundation; we sometimes +make exceptions for this. Our decision will be guided by the two goals +of preserving the free status of all derivatives of our free software and +of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY +FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN +OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES +PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE +PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, +REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING +WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR +REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, +INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING +OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED +TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY +YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER +PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest +to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least +the "copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. + This is free software, and you are welcome to redistribute it + under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may +be called something other than `show w' and `show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the program + `Gnomovision' (which makes passes at compilers) written by James Hacker. + + , 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Lesser General +Public License instead of this License. diff --git a/contrib/lizard/programs/README.md b/contrib/lizard/programs/README.md new file mode 100644 index 00000000000..ee116c116c9 --- /dev/null +++ b/contrib/lizard/programs/README.md @@ -0,0 +1,73 @@ +Command Line Interface for LZ4 library +============================================ + +Command Line Interface (CLI) can be created using the `make` command without any additional parameters. +There are also multiple targets that create different variations of CLI: +- `lizard` : default CLI, with a command line syntax close to gzip +- `lizardc32` : Same as `lizard`, but forced to compile in 32-bits mode + + +#### Aggregation of parameters +CLI supports aggregation of parameters i.e. `-b1`, `-e18`, and `-i1` can be joined into `-b1e18i1`. + + + +#### Benchmark in Command Line Interface +CLI includes in-memory compression benchmark module for lizard. +The benchmark is conducted using a given filename. +The file is read into memory. +It makes benchmark more precise as it eliminates I/O overhead. + +The benchmark measures ratio, compressed size, compression and decompression speed. +One can select compression levels starting from `-b` and ending with `-e`. +The `-i` parameter selects a number of seconds used for each of tested levels. + + + +#### Usage of Command Line Interface +The full list of commands can be obtained with `-h` or `-H` parameter: +``` +Usage : + lizard [arg] [input] [output] + +input : a filename + with no FILE, or when FILE is - or stdin, read standard input +Arguments : + -10...-19 : compression method fastLZ4 = 16-bit bytewise codewords + higher number == more compression but slower + -20...-29 : compression method LIZv1 = 24-bit bytewise codewords + -30...-39 : compression method fastLZ4 + Huffman + -40...-49 : compression method LIZv1 + Huffman + -d : decompression (default for .liz extension) + -z : force compression + -f : overwrite output without prompting +--rm : remove source file(s) after successful de/compression + -h/-H : display help/long help and exit + +Advanced arguments : + -V : display Version number and exit + -v : verbose mode + -q : suppress warnings; specify twice to suppress errors too + -c : force write to standard output, even if it is the console + -t : test compressed file integrity + -m : multiple input files (implies automatic output filenames) + -r : operate recursively on directories (sets also -m) + -l : compress using Legacy format (Linux kernel compression) + -B# : Block size [1-7] = 128KB, 256KB, 1MB, 4MB, 16MB, 64MB, 256MB (default : 4) + -BD : Block dependency (improve compression ratio) +--no-frame-crc : disable stream checksum (default:enabled) +--content-size : compressed frame includes original size (default:not present) +--[no-]sparse : sparse mode (default:enabled on file, disabled on stdout) +Benchmark arguments : + -b# : benchmark file(s), using # compression level (default : 1) + -e# : test all compression levels from -bX to # (default : 1) + -i# : minimum evaluation time in seconds (default : 3s) + -B# : cut file into independent blocks of size # bytes [32+] + or predefined block size [1-7] (default: 4) +``` + +#### License + +All files in this directory are licensed under GPL-v2. +See [COPYING](COPYING) for details. +The text of the license is also included at the top of each source file. diff --git a/contrib/lizard/programs/bench.c b/contrib/lizard/programs/bench.c new file mode 100644 index 00000000000..3fe56d1c060 --- /dev/null +++ b/contrib/lizard/programs/bench.c @@ -0,0 +1,502 @@ +/* + bench.c - Demo program to benchmark open-source compression algorithms + Copyright (C) Yann Collet 2012-2016 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + + +/*-************************************ +* Compiler options +**************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +#endif +#if defined(__MINGW32__) && !defined(_POSIX_SOURCE) +# define _POSIX_SOURCE 1 /* disable %llu warnings with MinGW on Windows */ +#endif + + +/* ************************************* +* Includes +***************************************/ +#include "platform.h" /* Compiler options */ +#include /* malloc, free */ +#include /* memset */ +#include /* fprintf, fopen, ftello64 */ +#include /* clock_t, clock, CLOCKS_PER_SEC */ + +#include "datagen.h" /* RDG_genBuffer */ +#include "xxhash/xxhash.h" +#include "lizard_common.h" +#include "lizard_decompress.h" +#define UTIL_WITHOUT_BASIC_TYPES +#include "util.h" /* UTIL_GetFileSize, UTIL_sleep */ + +#define Lizard_isError(errcode) (errcode==0) + + +/* ************************************* +* Constants +***************************************/ +#ifndef LIZARD_GIT_COMMIT_STRING +# define LIZARD_GIT_COMMIT_STRING "" +#else +# define LIZARD_GIT_COMMIT_STRING LIZARD_EXPAND_AND_QUOTE(Lizard_GIT_COMMIT) +#endif + +#define NBSECONDS 3 +#define TIMELOOP_MICROSEC 1*1000000ULL /* 1 second */ +#define ACTIVEPERIOD_MICROSEC 70*1000000ULL /* 70 seconds */ +#define COOLPERIOD_SEC 10 +#define DECOMP_MULT 2 /* test decompression DECOMP_MULT times longer than compression */ + +#define KB *(1 <<10) +#define MB *(1 <<20) +#define GB *(1U<<30) + +static const size_t maxMemory = (sizeof(size_t)==4) ? (2 GB - 64 MB) : (size_t)(1ULL << ((sizeof(size_t)*8)-31)); + +static U32 g_compressibilityDefault = 50; + + +/* ************************************* +* console display +***************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define DISPLAYLEVEL(l, ...) if (g_displayLevel>=l) { DISPLAY(__VA_ARGS__); } +static U32 g_displayLevel = 2; /* 0 : no display; 1: errors; 2 : + result + interaction + warnings; 3 : + progression; 4 : + information */ + +#define DISPLAYUPDATE(l, ...) if (g_displayLevel>=l) { \ + if ((clock() - g_time > refreshRate) || (g_displayLevel>=4)) \ + { g_time = clock(); DISPLAY(__VA_ARGS__); \ + if (g_displayLevel>=4) fflush(stdout); } } +static const clock_t refreshRate = CLOCKS_PER_SEC * 15 / 100; +static clock_t g_time = 0; + + +/* ************************************* +* Exceptions +***************************************/ +#ifndef DEBUG +# define DEBUG 0 +#endif +#define DEBUGOUTPUT(...) if (DEBUG) DISPLAY(__VA_ARGS__); +#define EXM_THROW(error, ...) \ +{ \ + DEBUGOUTPUT("Error defined at %s, line %i : \n", __FILE__, __LINE__); \ + DISPLAYLEVEL(1, "Error %i : ", error); \ + DISPLAYLEVEL(1, __VA_ARGS__); \ + DISPLAYLEVEL(1, "\n"); \ + exit(error); \ +} + + +/* ************************************* +* Benchmark Parameters +***************************************/ +static U32 g_nbSeconds = NBSECONDS; +static size_t g_blockSize = 0; +int g_additionalParam = 0; + +void BMK_setNotificationLevel(unsigned level) { g_displayLevel=level; } + +void BMK_setAdditionalParam(int additionalParam) { g_additionalParam=additionalParam; } + +void BMK_SetNbSeconds(unsigned nbSeconds) +{ + g_nbSeconds = nbSeconds; + DISPLAYLEVEL(3, "- test >= %u seconds per compression / decompression -\n", g_nbSeconds); +} + +void BMK_SetBlockSize(size_t blockSize) +{ + g_blockSize = blockSize; +} + + +/* ******************************************************** +* Bench functions +**********************************************************/ +typedef struct { + const char* srcPtr; + size_t srcSize; + char* cPtr; + size_t cRoom; + size_t cSize; + char* resPtr; + size_t resSize; +} blockParam_t; + + + +static int BMK_benchMem(const void* srcBuffer, size_t srcSize, + const char* displayName, int cLevel, + const size_t* fileSizes, U32 nbFiles) +{ + size_t const blockSize = (g_blockSize>=32 ? g_blockSize : srcSize) + (!srcSize) /* avoid div by 0 */ ; + U32 const maxNbBlocks = (U32) ((srcSize + (blockSize-1)) / blockSize) + nbFiles; + blockParam_t* const blockTable = (blockParam_t*) malloc(maxNbBlocks * sizeof(blockParam_t)); + size_t const maxCompressedSize = Lizard_compressBound((int)srcSize) + (maxNbBlocks * 1024); /* add some room for safety */ + void* const compressedBuffer = malloc(maxCompressedSize); + void* const resultBuffer = malloc(srcSize); + U32 nbBlocks; + UTIL_time_t ticksPerSecond; + + /* checks */ + if (!compressedBuffer || !resultBuffer || !blockTable) + EXM_THROW(31, "allocation error : not enough memory"); + + /* init */ + if (strlen(displayName)>17) displayName += strlen(displayName)-17; /* can only display 17 characters */ + UTIL_initTimer(&ticksPerSecond); + + /* Init blockTable data */ + { const char* srcPtr = (const char*)srcBuffer; + char* cPtr = (char*)compressedBuffer; + char* resPtr = (char*)resultBuffer; + U32 fileNb; + for (nbBlocks=0, fileNb=0; fileNb ACTIVEPERIOD_MICROSEC) { + DISPLAYLEVEL(2, "\rcooling down ... \r"); + UTIL_sleep(COOLPERIOD_SEC); + UTIL_getTime(&coolTime); + } + + /* Compression */ + DISPLAYLEVEL(2, "%2s-%-17.17s :%10u ->\r", marks[markNb], displayName, (U32)srcSize); + if (!cCompleted) memset(compressedBuffer, 0xE5, maxCompressedSize); /* warm up and erase result buffer */ + + UTIL_sleepMilli(1); /* give processor time to other processes */ + UTIL_waitForNextTick(ticksPerSecond); + UTIL_getTime(&clockStart); + + if (!cCompleted) { /* still some time to do compression tests */ + U32 nbLoops = 0; + do { + U32 blockNb; + for (blockNb=0; blockNbmaxTime; + } } + + cSize = 0; + { U32 blockNb; for (blockNb=0; blockNb%10u (%5.3f),%6.1f MB/s\r", + marks[markNb], displayName, (U32)srcSize, (U32)cSize, ratio, + (double)srcSize / fastestC ); + + (void)fastestD; (void)crcOrig; /* unused when decompression disabled */ +#if 1 + /* Decompression */ + if (!dCompleted) memset(resultBuffer, 0xD6, srcSize); /* warm result buffer */ + + UTIL_sleepMilli(1); /* give processor time to other processes */ + UTIL_waitForNextTick(ticksPerSecond); + UTIL_getTime(&clockStart); + + if (!dCompleted) { + U32 nbLoops = 0; + do { + U32 blockNb; + for (blockNb=0; blockNb(DECOMP_MULT*maxTime); + } } + + markNb = (markNb+1) % NB_MARKS; + DISPLAYLEVEL(2, "%2s-%-17.17s :%10u ->%10u (%5.3f),%6.1f MB/s ,%6.1f MB/s\r", + marks[markNb], displayName, (U32)srcSize, (U32)cSize, ratio, + (double)srcSize / fastestC, + (double)srcSize / fastestD ); + + /* CRC Checking */ + { U64 const crcCheck = XXH64(resultBuffer, srcSize, 0); + if (crcOrig!=crcCheck) { + size_t u; + DISPLAY("!!! WARNING !!! %14s : Invalid Checksum : %x != %x \n", displayName, (unsigned)crcOrig, (unsigned)crcCheck); + for (u=0; u u) break; + bacc += blockTable[segNb].srcSize; + } + pos = (U32)(u - bacc); + bNb = pos / (128 KB); + DISPLAY("(block %u, sub %u, pos %u) \n", segNb, bNb, pos); + break; + } + if (u==srcSize-1) { /* should never happen */ + DISPLAY("no difference detected\n"); + } } + break; + } } /* CRC Checking */ +#endif + } /* for (testNb = 1; testNb <= (g_nbSeconds + !g_nbSeconds); testNb++) */ + + if (g_displayLevel == 1) { + double cSpeed = (double)srcSize / fastestC; + double dSpeed = (double)srcSize / fastestD; + if (g_additionalParam) + DISPLAY("-%-3i%11i (%5.3f) %6.2f MB/s %6.1f MB/s %s (param=%d)\n", cLevel, (int)cSize, ratio, cSpeed, dSpeed, displayName, g_additionalParam); + else + DISPLAY("-%-3i%11i (%5.3f) %6.2f MB/s %6.1f MB/s %s\n", cLevel, (int)cSize, ratio, cSpeed, dSpeed, displayName); + } + DISPLAYLEVEL(2, "%2i#\n", cLevel); + } /* Bench */ + + /* clean up */ + free(blockTable); + free(compressedBuffer); + free(resultBuffer); + return 0; +} + + +static size_t BMK_findMaxMem(U64 requiredMem) +{ + size_t step = 64 MB; + BYTE* testmem=NULL; + + requiredMem = (((requiredMem >> 26) + 1) << 26); + requiredMem += 2*step; + if (requiredMem > maxMemory) requiredMem = maxMemory; + + while (!testmem) { + if (requiredMem > step) requiredMem -= step; + else requiredMem >>= 1; + testmem = (BYTE*) malloc ((size_t)requiredMem); + } + free (testmem); + + /* keep some space available */ + if (requiredMem > step) requiredMem -= step; + else requiredMem >>= 1; + + return (size_t)requiredMem; +} + + +static void BMK_benchCLevel(void* srcBuffer, size_t benchedSize, + const char* displayName, int cLevel, int cLevelLast, + const size_t* fileSizes, unsigned nbFiles) +{ + int l; + + const char* pch = strrchr(displayName, '\\'); /* Windows */ + if (!pch) pch = strrchr(displayName, '/'); /* Linux */ + if (pch) displayName = pch+1; + + SET_REALTIME_PRIORITY; + + if (g_displayLevel == 1 && !g_additionalParam) + DISPLAY("bench %s %s: input %u bytes, %u seconds, %u KB blocks\n", LIZARD_VERSION_STRING, LIZARD_GIT_COMMIT_STRING, (U32)benchedSize, g_nbSeconds, (U32)(g_blockSize>>10)); + + if (cLevelLast < cLevel) cLevelLast = cLevel; + + for (l=cLevel; l <= cLevelLast; l++) { + BMK_benchMem(srcBuffer, benchedSize, + displayName, l, + fileSizes, nbFiles); + } +} + + +/*! BMK_loadFiles() : + Loads `buffer` with content of files listed within `fileNamesTable`. + At most, fills `buffer` entirely */ +static void BMK_loadFiles(void* buffer, size_t bufferSize, + size_t* fileSizes, + const char** fileNamesTable, unsigned nbFiles) +{ + size_t pos = 0, totalSize = 0; + unsigned n; + for (n=0; n bufferSize-pos) fileSize = bufferSize-pos, nbFiles=n; /* buffer too small - stop after this file */ + { size_t const readSize = fread(((char*)buffer)+pos, 1, (size_t)fileSize, f); + if (readSize != (size_t)fileSize) EXM_THROW(11, "could not read %s", fileNamesTable[n]); + DISPLAYLEVEL(4, "readSize=%llu\n", (unsigned long long)readSize); + pos += readSize; } + fileSizes[n] = (size_t)fileSize; + totalSize += (size_t)fileSize; + fclose(f); + } + + if (totalSize == 0) EXM_THROW(12, "no data to bench"); +} + +static void BMK_benchFileTable(const char** fileNamesTable, unsigned nbFiles, + int cLevel, int cLevelLast) +{ + void* srcBuffer; + size_t benchedSize; + size_t* fileSizes = (size_t*)malloc(nbFiles * sizeof(size_t)); + U64 const totalSizeToLoad = UTIL_getTotalFileSize(fileNamesTable, nbFiles); + char mfName[20] = {0}; + + if (!fileSizes) EXM_THROW(12, "not enough memory for fileSizes"); + + /* Memory allocation & restrictions */ + benchedSize = BMK_findMaxMem(totalSizeToLoad * 3) / 3; + if (benchedSize==0) EXM_THROW(12, "not enough memory"); + if ((U64)benchedSize > totalSizeToLoad) benchedSize = (size_t)totalSizeToLoad; + if (benchedSize > LIZARD_MAX_INPUT_SIZE) { + benchedSize = LIZARD_MAX_INPUT_SIZE; + DISPLAY("File(s) bigger than Lizard's max input size; testing %u MB only...\n", (U32)(benchedSize >> 20)); + } + else + if (benchedSize < totalSizeToLoad) { + DISPLAY("Not enough memory; testing %u MB only...\n", (U32)(benchedSize >> 20)); + } + srcBuffer = malloc(benchedSize + !benchedSize); /* avoid alloc of zero */ + if (!srcBuffer) EXM_THROW(12, "not enough memory"); + + /* Load input buffer */ + BMK_loadFiles(srcBuffer, benchedSize, fileSizes, fileNamesTable, nbFiles); + + /* Bench */ + snprintf (mfName, sizeof(mfName), " %u files", nbFiles); + { const char* displayName = (nbFiles > 1) ? mfName : fileNamesTable[0]; + BMK_benchCLevel(srcBuffer, benchedSize, + displayName, cLevel, cLevelLast, + fileSizes, nbFiles); + } + + /* clean up */ + free(srcBuffer); + free(fileSizes); +} + + +static void BMK_syntheticTest(int cLevel, int cLevelLast, double compressibility) +{ + char name[20] = {0}; + size_t benchedSize = 10000000; + void* const srcBuffer = malloc(benchedSize); + + /* Memory allocation */ + if (!srcBuffer) EXM_THROW(21, "not enough memory"); + + /* Fill input buffer */ + RDG_genBuffer(srcBuffer, benchedSize, compressibility, 0.0, 0); + + /* Bench */ + snprintf (name, sizeof(name), "Synthetic %2u%%", (unsigned)(compressibility*100)); + BMK_benchCLevel(srcBuffer, benchedSize, name, cLevel, cLevelLast, &benchedSize, 1); + + /* clean up */ + free(srcBuffer); +} + + +int BMK_benchFiles(const char** fileNamesTable, unsigned nbFiles, + int cLevel, int cLevelLast) +{ + double const compressibility = (double)g_compressibilityDefault / 100; + + if (cLevel < LIZARD_MIN_CLEVEL) cLevel = LIZARD_MIN_CLEVEL; + if (cLevel > LIZARD_MAX_CLEVEL) cLevel = LIZARD_MAX_CLEVEL; + if (cLevelLast > LIZARD_MAX_CLEVEL) cLevelLast = LIZARD_MAX_CLEVEL; + if (cLevelLast < cLevel) cLevelLast = cLevel; + if (cLevelLast > cLevel) DISPLAYLEVEL(2, "Benchmarking levels from %d to %d\n", cLevel, cLevelLast); + + if (nbFiles == 0) + BMK_syntheticTest(cLevel, cLevelLast, compressibility); + else + BMK_benchFileTable(fileNamesTable, nbFiles, cLevel, cLevelLast); + return 0; +} diff --git a/contrib/lizard/programs/bench.h b/contrib/lizard/programs/bench.h new file mode 100644 index 00000000000..feff51293f3 --- /dev/null +++ b/contrib/lizard/programs/bench.h @@ -0,0 +1,37 @@ +/* + bench.h - Demo program to benchmark open-source compression algorithm + Copyright (C) Yann Collet 2012-2016 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ +#ifndef BENCH_H_12 +#define BENCH_H_12 + +#include + +int BMK_benchFiles(const char** fileNamesTable, unsigned nbFiles, + int cLevel, int cLevelLast); + +/* Set Parameters */ +void BMK_SetNbSeconds(unsigned nbLoops); +void BMK_SetBlockSize(size_t blockSize); +void BMK_setAdditionalParam(int additionalParam); +void BMK_setNotificationLevel(unsigned level); + +#endif /* BENCH_H_125623623633 */ diff --git a/contrib/lizard/programs/datagen.c b/contrib/lizard/programs/datagen.c new file mode 100644 index 00000000000..87d6044830a --- /dev/null +++ b/contrib/lizard/programs/datagen.c @@ -0,0 +1,188 @@ +/* + datagen.c - compressible data generator test tool + Copyright (C) Yann Collet 2012-2016 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +/************************************** +* Includes +**************************************/ +#include "platform.h" /* Compiler options, SET_BINARY_MODE */ +#include "util.h" /* U32 */ +#include /* malloc */ +#include /* FILE, fwrite */ +#include /* memcpy */ + + +/************************************** +* Constants +**************************************/ +#define KB *(1 <<10) + +#define PRIME1 2654435761U +#define PRIME2 2246822519U + + +/************************************** +* Local types +**************************************/ +#define LTLOG 13 +#define LTSIZE (1<> (32 - r))) +static unsigned int RDG_rand(U32* src) +{ + U32 rand32 = *src; + rand32 *= PRIME1; + rand32 ^= PRIME2; + rand32 = RDG_rotl32(rand32, 13); + *src = rand32; + return rand32; +} + + +static void RDG_fillLiteralDistrib(litDistribTable lt, double ld) +{ + BYTE const firstChar = ld <= 0.0 ? 0 : '('; + BYTE const lastChar = ld <= 0.0 ? 255 : '}'; + BYTE character = ld <= 0.0 ? 0 : '0'; + U32 u = 0; + + while (u lastChar) character = firstChar; + } +} + + +static BYTE RDG_genChar(U32* seed, const litDistribTable lt) +{ + U32 id = RDG_rand(seed) & LTMASK; + return (lt[id]); +} + + +#define RDG_DICTSIZE (32 KB) +#define RDG_RAND15BITS ((RDG_rand(seed) >> 3) & 32767) +#define RDG_RANDLENGTH ( ((RDG_rand(seed) >> 7) & 7) ? (RDG_rand(seed) & 15) : (RDG_rand(seed) & 511) + 15) +void RDG_genBlock(void* buffer, size_t buffSize, size_t prefixSize, double matchProba, litDistribTable lt, unsigned* seedPtr) +{ + BYTE* buffPtr = (BYTE*)buffer; + const U32 matchProba32 = (U32)(32768 * matchProba); + size_t pos = prefixSize; + U32* seed = seedPtr; + + /* special case */ + while (matchProba >= 1.0) + { + size_t size0 = RDG_rand(seed) & 3; + size0 = (size_t)1 << (16 + size0 * 2); + size0 += RDG_rand(seed) & (size0-1); /* because size0 is power of 2*/ + if (buffSize < pos + size0) + { + memset(buffPtr+pos, 0, buffSize-pos); + return; + } + memset(buffPtr+pos, 0, size0); + pos += size0; + buffPtr[pos-1] = RDG_genChar(seed, lt); + } + + /* init */ + if (pos==0) buffPtr[0] = RDG_genChar(seed, lt), pos=1; + + /* Generate compressible data */ + while (pos < buffSize) + { + /* Select : Literal (char) or Match (within 32K) */ + if (RDG_RAND15BITS < matchProba32) + { + /* Copy (within 32K) */ + size_t match; + size_t d; + int length = RDG_RANDLENGTH + 4; + U32 offset = RDG_RAND15BITS + 1; + if (offset > pos) offset = (U32)pos; + match = pos - offset; + d = pos + length; + if (d > buffSize) d = buffSize; + while (pos < d) buffPtr[pos++] = buffPtr[match++]; + } + else + { + /* Literal (noise) */ + size_t d; + size_t length = RDG_RANDLENGTH; + d = pos + length; + if (d > buffSize) d = buffSize; + while (pos < d) buffPtr[pos++] = RDG_genChar(seed, lt); + } + } +} + + +void RDG_genBuffer(void* buffer, size_t size, double matchProba, double litProba, unsigned seed) +{ + litDistribTable lt; + if (litProba==0.0) litProba = matchProba / 4.5; + RDG_fillLiteralDistrib(lt, litProba); + RDG_genBlock(buffer, size, 0, matchProba, lt, &seed); +} + + +#define RDG_BLOCKSIZE (128 KB) +void RDG_genOut(unsigned long long size, double matchProba, double litProba, unsigned seed) +{ + BYTE buff[RDG_DICTSIZE + RDG_BLOCKSIZE]; + U64 total = 0; + size_t genBlockSize = RDG_BLOCKSIZE; + litDistribTable lt; + + /* init */ + if (litProba==0.0) litProba = matchProba / 4.5; + RDG_fillLiteralDistrib(lt, litProba); + SET_BINARY_MODE(stdout); + + /* Generate dict */ + RDG_genBlock(buff, RDG_DICTSIZE, 0, matchProba, lt, &seed); + + /* Generate compressible data */ + while (total < size) + { + RDG_genBlock(buff, RDG_DICTSIZE+RDG_BLOCKSIZE, RDG_DICTSIZE, matchProba, lt, &seed); + if (size-total < RDG_BLOCKSIZE) genBlockSize = (size_t)(size-total); + total += genBlockSize; + fwrite(buff, 1, genBlockSize, stdout); + /* update dict */ + memcpy(buff, buff + RDG_BLOCKSIZE, RDG_DICTSIZE); + } +} diff --git a/contrib/lizard/programs/datagen.h b/contrib/lizard/programs/datagen.h new file mode 100644 index 00000000000..2cc41d2a3ff --- /dev/null +++ b/contrib/lizard/programs/datagen.h @@ -0,0 +1,39 @@ +/* + datagen.h - compressible data generator header + Copyright (C) Yann Collet 2012-2015 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + + +#include /* size_t */ + +void RDG_genOut(unsigned long long size, double matchProba, double litProba, unsigned seed); +void RDG_genBuffer(void* buffer, size_t size, double matchProba, double litProba, unsigned seed); +/* RDG_genOut + Generate 'size' bytes of compressible data into stdout. + Compressibility can be controlled using 'matchProba'. + 'LitProba' is optional, and affect variability of bytes. If litProba==0.0, default value is used. + Generated data can be selected using 'seed'. + If (matchProba, litProba and seed) are equal, the function always generate the same content. + + RDG_genBuffer + Same as RDG_genOut, but generate data into provided buffer +*/ diff --git a/contrib/lizard/programs/lizard.1 b/contrib/lizard/programs/lizard.1 new file mode 100644 index 00000000000..b5d84213442 --- /dev/null +++ b/contrib/lizard/programs/lizard.1 @@ -0,0 +1,229 @@ +\." +\." lizard.1: This is a manual page for 'lizard' program. This file is part of the +\." lizard project. +\." Author: Yann Collet +\." +. +\." No hyphenation +.hy 0 +.nr HY 0 +. +.TH lizard "1" "2015-03-21" "lizard" "User Commands" +.SH NAME +\fBlizard, unlizard, lizardcat\fR \- Compress or decompress .liz files + +.SH SYNOPSIS +.TP 5 +\fBlizard\fR [\fBOPTIONS\fR] [-|INPUT-FILE] +.PP +.B unlizard +is equivalent to +.BR "lizard \-d" +.br +.B lizardcat +is equivalent to +.BR "lizard \-dc" +.br +.PP +When writing scripts that need to decompress files, +it is recommended to always use the name +.B lizard +with appropriate arguments +.RB ( "lizard \-d" +or +.BR "lizard \-dc" ) +instead of the names +.B unlizard +and +.BR lizardcat . + + +.SH DESCRIPTION +.PP +\fBlizard\fR is an extremely fast lossless compression algorithm, +based on \fBbyte-aligned LZ77\fR family of compression scheme. +\fBlizard\fR offers compression speeds of 400 MB/s per core, linearly scalable with multi-core CPUs. +It features an extremely fast decoder, with speed in multiple GB/s per core, +typically reaching RAM speed limit on multi-core systems. +The native file format is the +.B .liz +format. + +.B lizard +supports a command line syntax similar but not identical to +.BR gzip (1). +Differences are : +\fBlizard\fR preserve original files ; +\fBlizard file1 file2\fR means : compress file1 \fIinto\fR file2 ; +\fBlizard file\fR shows real-time statistics during compression . + +Default behaviors can be modified by opt-in commands, described below. +\fBlizard --quiet --multiple\fR more closely mimics \fBgzip\fR behavior. + +.SS "Concatenation of .liz files" +It is possible to concatenate +.B .liz +files as is. +.B lizard +will decompress such files as if they were a single +.B .liz +file. For example: + lizard file1 > foo.liz + lizard file2 >> foo.liz + then + lizardcat foo.liz + is equivalent to : + cat file1 file2 + +.PP + +.SH OPTIONS +. +.SS "Short commands concatenation" +In some cases, some options can be expressed using short command +.B "-x" +or long command +.B "--long-word" . +Short commands can be concatenated together. For example, +.B "-d -c" +is equivalent to +.B "-dc" . +Long commands cannot be concatenated. +They must be clearly separated by a space. +.SS "Multiple commands" +When multiple contradictory commands are issued on a same command line, +only the latest one will be applied. +. +.SS "Operation mode" +.TP +.BR \-z ", " \-\-compress +Compress. +This is the default operation mode +when no operation mode option is specified , +no other operation mode is implied from the command name +(for example, +.B unlizard +implies +.B \-\-decompress ), +nor from the input file name +(for example, a file extension +.B .liz +implies +.B \-\-decompress +by default). +.B -z +can also be used to force compression of an already compressed +.B .liz +file. +.TP +.BR \-d ", " \-\-decompress ", " \-\-uncompress +Decompress. +.B --decompress +is also the default operation when the input filename has an +.B .liz +extensionq +.TP +.BR \-t ", " \-\-test +Test the integrity of compressed +.B .liz +files. +The decompressed data is discarded. +No files are created nor removed. +. +.SS "Operation modifiers" +.TP +.B \-1 + fast compression (default) +.TP +.B \-9 + high compression + +.TP +.BR \-f ", " --[no-]force + This option has several effects: +.RS +.IP \(bu 3 +If the target file already exists, +overwrite it without prompting. +.IP \(bu 3 +When used with +.B \-\-decompress +and +.B lizard +cannot recognize the type of the source file, +copy the source file as is to standard output. +This allows +.B lizardcat +.B \-\-force +to be used like +.BR cat (1) +for files that have not been compressed with +.BR lizard . +.RE + +.TP +.BR \-c ", " \--stdout ", " \--to-stdout + force write to standard output, even if it is the console + +.TP +.BR \-m ", " \--multiple + Multiple file names. + By default, the second filename is used as the destination filename for the compressed file. + With +.B -m +, you can specify any number of input filenames. Each of them will be compressed +independently, and the resulting name of each compressed file will be +.B filename.liz +. + +.TP +.B \-B# + block size [4-7](default : 7) + B1 = 128KB, B2=256KB, B3=1MB, B4=4MB, B5=16MB, B6=64MB, B7=256MB +.TP +.B \-BD + block dependency (improves compression ratio on small blocks) +.TP +.B \--[no-]frame-crc + select frame checksum (default:enabled) +.TP +.B \--[no-]content-size + header includes original size (default:not present) + Note : this option can only be activated when the original size can be determined, +hence for a file. It won't work with unknown source size, such as stdin or pipe. +.TP +.B \--[no-]sparse + sparse mode support (default:enabled on file, disabled on stdout) +.TP +.B \-l + use Legacy format (useful for Linux Kernel compression) +. +.SS "Other options" +.TP +.BR \-v ", " --verbose + verbose mode +.TP +.BR \-q ", " --quiet + suppress warnings and real-time statistics; specify twice to suppress errors too +.TP +.B \-h/\-H + display help/long help and exit +.TP +.BR \-V ", " \--version + display Version number and exit +.TP +.BR \-k ", " \--keep + Don't delete source file. +This is default behavior anyway, so this option is just for compatibility with gzip/xz. +.TP +.B \-b + benchmark file(s) +.TP +.B \-i# + iteration loops [1-9](default : 3), benchmark mode only + +.SH BUGS +Report bugs at: https://github.com/inikep/lizard/issues + +.SH AUTHOR +Yann Collet diff --git a/contrib/lizard/programs/lizardcli.c b/contrib/lizard/programs/lizardcli.c new file mode 100644 index 00000000000..2a63fe8e001 --- /dev/null +++ b/contrib/lizard/programs/lizardcli.c @@ -0,0 +1,581 @@ +/* + Lizardcli - Lizard Command Line Interface + Copyright (C) Yann Collet 2011-2016 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ +/* + Note : this is stand-alone program. + It is not part of Lizard compression library, it is a user program of the Lizard library. + The license of Lizard library is BSD. + The license of xxHash library is BSD. + The license of this compression CLI program is GPLv2. +*/ + + +/**************************** +* Includes +*****************************/ +#include "platform.h" /* Compiler options, IS_CONSOLE */ +#include "util.h" /* UTIL_HAS_CREATEFILELIST, UTIL_createFileList */ +#include /* fprintf, getchar */ +#include /* exit, calloc, free */ +#include /* strcmp, strlen */ +#include "bench.h" /* BMK_benchFile, BMK_SetNbIterations, BMK_SetBlocksize, BMK_SetPause */ +#include "lizardio.h" /* LizardIO_compressFilename, LizardIO_decompressFilename, LizardIO_compressMultipleFilenames */ +#include "lizard_compress.h" /* LizardHC_DEFAULT_CLEVEL, LIZARD_VERSION_STRING */ + + + +/***************************** +* Constants +******************************/ +#define COMPRESSOR_NAME "Lizard command line interface" +#define AUTHOR "Y.Collet & P.Skibinski" +#define WELCOME_MESSAGE "%s %i-bit %s by %s (%s)\n", COMPRESSOR_NAME, (int)(sizeof(void*)*8), LIZARD_VERSION_STRING, AUTHOR, __DATE__ +#define LIZARD_EXTENSION ".liz" +#define LIZARDCAT "lizardcat" +#define UNLIZARD "unlizard" + +#define KB *(1U<<10) +#define MB *(1U<<20) +#define GB *(1U<<30) + +#define LIZARD_BLOCKSIZEID_DEFAULT 4 + + +/*-************************************ +* Macros +***************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define DISPLAYLEVEL(l, ...) if (displayLevel>=l) { DISPLAY(__VA_ARGS__); } +static unsigned displayLevel = 2; /* 0 : no display ; 1: errors only ; 2 : downgradable normal ; 3 : non-downgradable normal; 4 : + information */ + + +/*-************************************ +* Exceptions +***************************************/ +#define DEBUG 0 +#define DEBUGOUTPUT(...) if (DEBUG) DISPLAY(__VA_ARGS__); +#define EXM_THROW(error, ...) \ +{ \ + DEBUGOUTPUT("Error defined at %s, line %i : \n", __FILE__, __LINE__); \ + DISPLAYLEVEL(1, "Error %i : ", error); \ + DISPLAYLEVEL(1, __VA_ARGS__); \ + DISPLAYLEVEL(1, "\n"); \ + exit(error); \ +} + + +/*-************************************ +* Version modifiers +***************************************/ +#define EXTENDED_ARGUMENTS +#define EXTENDED_HELP +#define EXTENDED_FORMAT +#define DEFAULT_COMPRESSOR LizardIO_compressFilename +#define DEFAULT_DECOMPRESSOR LizardIO_decompressFilename +int LizardIO_compressFilename_Legacy(const char* input_filename, const char* output_filename, int compressionlevel); /* hidden function */ + + +/*-*************************** +* Functions +*****************************/ +static int usage(const char* exeName) +{ + DISPLAY( "Usage :\n"); + DISPLAY( " %s [arg] [input] [output]\n", exeName); + DISPLAY( "\n"); + DISPLAY( "input : a filename\n"); + DISPLAY( " with no FILE, or when FILE is - or %s, read standard input\n", stdinmark); + DISPLAY( "Arguments :\n"); + DISPLAY( " -10...-19 : compression method fastLZ4 = 16-bit bytewise codewords\n"); + DISPLAY( " higher number == more compression but slower\n"); + DISPLAY( " -20...-29 : compression method LIZv1 = 24-bit bytewise codewords\n"); +#ifndef LIZARD_NO_HUFFMAN + DISPLAY( " -30...-39 : compression method fastLZ4 + Huffman\n"); + DISPLAY( " -40...-49 : compression method LIZv1 + Huffman\n"); +#endif + DISPLAY( " -d : decompression (default for %s extension)\n", LIZARD_EXTENSION); + DISPLAY( " -z : force compression\n"); + DISPLAY( " -f : overwrite output without prompting \n"); + DISPLAY( "--rm : remove source file(s) after successful de/compression \n"); + DISPLAY( " -h/-H : display help/long help and exit\n"); + return 0; +} + +static int usage_advanced(const char* exeName) +{ + DISPLAY(WELCOME_MESSAGE); + usage(exeName); + DISPLAY( "\n"); + DISPLAY( "Advanced arguments :\n"); + DISPLAY( " -V : display Version number and exit\n"); + DISPLAY( " -v : verbose mode\n"); + DISPLAY( " -q : suppress warnings; specify twice to suppress errors too\n"); + DISPLAY( " -c : force write to standard output, even if it is the console\n"); + DISPLAY( " -t : test compressed file integrity\n"); + DISPLAY( " -m : multiple input files (implies automatic output filenames)\n"); +#ifdef UTIL_HAS_CREATEFILELIST + DISPLAY( " -r : operate recursively on directories (sets also -m)\n"); +#endif + DISPLAY( " -B# : Block size [1-7] = 128KB, 256KB, 1MB, 4MB, 16MB, 64MB, 256MB (default : 4)\n"); + DISPLAY( " -BD : Block dependency (improve compression ratio)\n"); + /* DISPLAY( " -BX : enable block checksum (default:disabled)\n"); *//* Option currently inactive */ + DISPLAY( "--no-frame-crc : disable stream checksum (default:enabled)\n"); + DISPLAY( "--content-size : compressed frame includes original size (default:not present)\n"); + DISPLAY( "--[no-]sparse : sparse mode (default:enabled on file, disabled on stdout)\n"); + DISPLAY( "Benchmark arguments :\n"); + DISPLAY( " -b# : benchmark file(s), using # compression level (default : 1) \n"); + DISPLAY( " -e# : test all compression levels from -bX to # (default : 1)\n"); + DISPLAY( " -i# : minimum evaluation time in seconds (default : 3s)\n"); + DISPLAY( " -B# : cut file into independent blocks of size # bytes [32+]\n"); + DISPLAY( " or predefined block size [1-7] (default: 4)\n"); + EXTENDED_HELP; + return 0; +} + +static int usage_longhelp(const char* exeName) +{ + usage_advanced(exeName); + DISPLAY( "\n"); + DISPLAY( "****************************\n"); + DISPLAY( "***** Advanced comment *****\n"); + DISPLAY( "****************************\n"); + DISPLAY( "\n"); + DISPLAY( "Which values can [output] have ? \n"); + DISPLAY( "---------------------------------\n"); + DISPLAY( "[output] : a filename \n"); + DISPLAY( " '%s', or '-' for standard output (pipe mode)\n", stdoutmark); + DISPLAY( " '%s' to discard output (test mode) \n", NULL_OUTPUT); + DISPLAY( "[output] can be left empty. In this case, it receives the following value :\n"); + DISPLAY( " - if stdout is not the console, then [output] = stdout \n"); + DISPLAY( " - if stdout is console : \n"); + DISPLAY( " + for compression, output to filename%s \n", LIZARD_EXTENSION); + DISPLAY( " + for decompression, output to filename without '%s'\n", LIZARD_EXTENSION); + DISPLAY( " > if input filename has no '%s' extension : error \n", LIZARD_EXTENSION); + DISPLAY( "\n"); + DISPLAY( "stdin, stdout and the console : \n"); + DISPLAY( "--------------------------------\n"); + DISPLAY( "To protect the console from binary flooding (bad argument mistake)\n"); + DISPLAY( "%s will refuse to read from console, or write to console \n", exeName); + DISPLAY( "except if '-c' command is specified, to force output to console \n"); + DISPLAY( "\n"); + DISPLAY( "Simple example :\n"); + DISPLAY( "----------------\n"); + DISPLAY( "1 : compress 'filename' fast, using default output name 'filename.liz'\n"); + DISPLAY( " %s filename\n", exeName); + DISPLAY( "\n"); + DISPLAY( "Short arguments can be aggregated. For example :\n"); + DISPLAY( "----------------------------------\n"); + DISPLAY( "2 : compress 'filename' in high compression mode, overwrite output if exists\n"); + DISPLAY( " %s -9 -f filename \n", exeName); + DISPLAY( " is equivalent to :\n"); + DISPLAY( " %s -9f filename \n", exeName); + DISPLAY( "\n"); + DISPLAY( "%s can be used in 'pure pipe mode'. For example :\n", exeName); + DISPLAY( "-------------------------------------\n"); + DISPLAY( "3 : compress data stream from 'generator', send result to 'consumer'\n"); + DISPLAY( " generator | %s | consumer \n", exeName); + return 0; +} + +static int badusage(const char* exeName) +{ + DISPLAYLEVEL(1, "Incorrect parameters\n"); + if (displayLevel >= 1) usage(exeName); + exit(1); +} + + +static void waitEnter(void) +{ + DISPLAY("Press enter to continue...\n"); + (void)getchar(); +} + + +static const char* lastNameFromPath(const char* path) +{ + const char* name = strrchr(path, '/'); + if (name==NULL) name = strrchr(path, '\\'); /* windows */ + if (name==NULL) return path; + return name+1; +} + + +/*! readU32FromChar() : + @return : unsigned integer value reach from input in `char` format + Will also modify `*stringPtr`, advancing it to position where it stopped reading. + Note : this function can overflow if result > MAX_UINT */ +static unsigned readU32FromChar(const char** stringPtr) +{ + unsigned result = 0; + while ((**stringPtr >='0') && (**stringPtr <='9')) + result *= 10, result += **stringPtr - '0', (*stringPtr)++ ; + return result; +} + +typedef enum { om_auto, om_compress, om_decompress, om_test, om_bench } operationMode_e; + +int main(int argc, const char** argv) +{ + int i, + cLevel=1, + cLevelLast=1, + forceStdout=0, + main_pause=0, + multiple_inputs=0, + operationResult=0; + operationMode_e mode = om_auto; + const char* input_filename = NULL; + const char* output_filename= NULL; + char* dynNameSpace = NULL; + const char** inFileNames = (const char**) calloc(argc, sizeof(char*)); + unsigned ifnIdx=0; + const char nullOutput[] = NULL_OUTPUT; + const char extension[] = LIZARD_EXTENSION; + size_t blockSize = LizardIO_setBlockSizeID(LIZARD_BLOCKSIZEID_DEFAULT); + const char* const exeName = lastNameFromPath(argv[0]); +#ifdef UTIL_HAS_CREATEFILELIST + const char** extendedFileList = NULL; + char* fileNamesBuf = NULL; + unsigned fileNamesNb, recursive=0; +#endif + + /* Init */ + if (inFileNames==NULL) { + DISPLAY("Allocation error : not enough memory \n"); + return 1; + } + inFileNames[0] = stdinmark; + LizardIO_setOverwrite(0); + + /* lizardcat predefined behavior */ + if (!strcmp(exeName, LIZARDCAT)) { + mode = om_decompress; + LizardIO_setOverwrite(1); + forceStdout=1; + output_filename=stdoutmark; + displayLevel=1; + multiple_inputs=1; + } + if (!strcmp(exeName, UNLIZARD)) { mode = om_decompress; } + + /* command switches */ + for(i=1; i='0') && (*argument<='9')) { + cLevel = readU32FromChar(&argument); + argument--; + continue; + } + + + switch(argument[0]) + { + /* Display help */ + case 'V': DISPLAY(WELCOME_MESSAGE); goto _cleanup; /* Version */ + case 'h': usage_advanced(exeName); goto _cleanup; + case 'H': usage_longhelp(exeName); goto _cleanup; + + case 'e': + argument++; + cLevelLast = readU32FromChar(&argument); + argument--; + break; + + /* Compression (default) */ + case 'z': mode = om_compress; break; + + /* Decoding */ + case 'd': mode = om_decompress; break; + + /* Force stdout, even if stdout==console */ + case 'c': forceStdout=1; output_filename=stdoutmark; break; + + /* Test integrity */ + case 't': mode = om_test; break; + + /* Overwrite */ + case 'f': LizardIO_setOverwrite(1); break; + + /* Verbose mode */ + case 'v': displayLevel++; break; + + /* Quiet mode */ + case 'q': if (displayLevel) displayLevel--; break; + + /* keep source file (default anyway, so useless) (for xz/lzma compatibility) */ + case 'k': LizardIO_setRemoveSrcFile(0); break; + + /* Modify Block Properties */ + case 'B': + while (argument[1]!=0) { + int exitBlockProperties=0; + switch(argument[1]) + { + case 'D': LizardIO_setBlockMode(LizardIO_blockLinked); argument++; break; + case 'X': LizardIO_setBlockChecksumMode(1); argument ++; break; /* disabled by default */ + default : + if (argument[1] < '0' || argument[1] > '9') { + exitBlockProperties=1; + break; + } else { + unsigned B; + argument++; + B = readU32FromChar(&argument); + argument--; + if (B < 1) badusage(exeName); + if (B <= 7) { + blockSize = LizardIO_setBlockSizeID(B); + BMK_SetBlockSize(blockSize); + DISPLAYLEVEL(2, "using blocks of size %u KB \n", (U32)(blockSize>>10)); + } else { + if (B < 32) badusage(exeName); + BMK_SetBlockSize(B); + if (B >= 1024) { + DISPLAYLEVEL(2, "bench: using blocks of size %u KB \n", (U32)(B>>10)); + } else { + DISPLAYLEVEL(2, "bench: using blocks of size %u bytes \n", (U32)(B)); + } + } + break; + } + } + if (exitBlockProperties) break; + } + break; + + /* Benchmark */ + case 'b': mode = om_bench; multiple_inputs=1; + break; + +#ifdef UTIL_HAS_CREATEFILELIST + /* recursive */ + case 'r': recursive=1; + /* fallthrough */ +#endif + /* Treat non-option args as input files. See https://code.google.com/p/lizard/issues/detail?id=151 */ + /* fallthrough */ + case 'm': multiple_inputs=1; + break; + + /* Modify Nb Seconds (benchmark only) */ + case 'i': + { unsigned iters; + argument++; + iters = readU32FromChar(&argument); + argument--; + BMK_setNotificationLevel(displayLevel); + BMK_SetNbSeconds(iters); /* notification if displayLevel >= 3 */ + } + break; + + /* Pause at the end (hidden option) */ + case 'p': main_pause=1; break; + + /* Specific commands for customized versions */ + EXTENDED_ARGUMENTS; + + /* Unrecognised command */ + default : badusage(exeName); + } + } + continue; + } + + /* Store in *inFileNames[] if -m is used. */ + if (multiple_inputs) { inFileNames[ifnIdx++]=argument; continue; } + + /* Store first non-option arg in input_filename to preserve original cli logic. */ + if (!input_filename) { input_filename=argument; continue; } + + /* Second non-option arg in output_filename to preserve original cli logic. */ + if (!output_filename) { + output_filename=argument; + if (!strcmp (output_filename, nullOutput)) output_filename = nulmark; + continue; + } + + /* 3rd non-option arg should not exist */ + DISPLAYLEVEL(1, "Warning : %s won't be used ! Do you want multiple input files (-m) ? \n", argument); + } + + DISPLAYLEVEL(3, WELCOME_MESSAGE); + if ((mode == om_compress) || (mode == om_bench)) DISPLAYLEVEL(4, "Blocks size : %i KB\n", (U32)(blockSize>>10)); + + if (multiple_inputs) { + input_filename = inFileNames[0]; +#ifdef UTIL_HAS_CREATEFILELIST + if (recursive) { /* at this stage, filenameTable is a list of paths, which can contain both files and directories */ + extendedFileList = UTIL_createFileList(inFileNames, ifnIdx, &fileNamesBuf, &fileNamesNb); + if (extendedFileList) { + unsigned u; + for (u=0; u try to select one automatically (when possible) */ + while ((!output_filename) && (multiple_inputs==0)) { + if (!IS_CONSOLE(stdout)) { output_filename=stdoutmark; break; } /* Default to stdout whenever possible (i.e. not a console) */ + if (mode == om_auto) { /* auto-determine compression or decompression, based on file extension */ + size_t const inSize = strlen(input_filename); + size_t const extSize = strlen(LIZARD_EXTENSION); + size_t const extStart= (inSize > extSize) ? inSize-extSize : 0; + if (!strcmp(input_filename+extStart, LIZARD_EXTENSION)) mode = om_decompress; + else mode = om_compress; + } + if (mode == om_compress) { /* compression to file */ + size_t const l = strlen(input_filename); + dynNameSpace = (char*)calloc(1,l+5); + if (dynNameSpace==NULL) { perror(exeName); exit(1); } + strcpy(dynNameSpace, input_filename); + strcat(dynNameSpace, LIZARD_EXTENSION); + output_filename = dynNameSpace; + DISPLAYLEVEL(2, "Compressed filename will be : %s \n", output_filename); + break; + } + if (mode == om_decompress) {/* decompression to file (automatic name will work only if input filename has correct format extension) */ + size_t outl; + size_t const inl = strlen(input_filename); + dynNameSpace = (char*)calloc(1,inl+1); + if (dynNameSpace==NULL) { perror(exeName); exit(1); } + strcpy(dynNameSpace, input_filename); + outl = inl; + if (inl>4) + while ((outl >= inl-4) && (input_filename[outl] == extension[outl-inl+4])) dynNameSpace[outl--]=0; + if (outl != inl-5) { DISPLAYLEVEL(1, "Cannot determine an output filename\n"); badusage(exeName); } + output_filename = dynNameSpace; + DISPLAYLEVEL(2, "Decoding file %s \n", output_filename); + } + break; + } + + /* Check if output is defined as console; trigger an error in this case */ + if (!output_filename) output_filename = "*\\dummy^!//"; + if (!strcmp(output_filename,stdoutmark) && IS_CONSOLE(stdout) && !forceStdout) { + DISPLAYLEVEL(1, "refusing to write to console without -c\n"); + exit(1); + } + + /* Downgrade notification level in stdout and multiple file mode */ + if (!strcmp(output_filename,stdoutmark) && (displayLevel==2)) displayLevel=1; + if ((multiple_inputs) && (displayLevel==2)) displayLevel=1; + + /* IO Stream/File */ + LizardIO_setNotificationLevel(displayLevel); + if (ifnIdx == 0) multiple_inputs = 0; + if (mode == om_decompress) { + if (multiple_inputs) + operationResult = LizardIO_decompressMultipleFilenames(inFileNames, ifnIdx, !strcmp(output_filename,stdoutmark) ? stdoutmark : LIZARD_EXTENSION); + else + operationResult = LizardIO_decompressFilename(input_filename, output_filename); + } else { /* compression is default action */ + { + if (multiple_inputs) + operationResult = LizardIO_compressMultipleFilenames(inFileNames, ifnIdx, LIZARD_EXTENSION, cLevel); + else + operationResult = LizardIO_compressFilename(input_filename, output_filename, cLevel); + } + } + +_cleanup: + if (main_pause) waitEnter(); + if (dynNameSpace) free(dynNameSpace); +#ifdef UTIL_HAS_CREATEFILELIST + if (extendedFileList) + UTIL_freeFileList(extendedFileList, fileNamesBuf); + else +#endif + free((void*)inFileNames); + return operationResult; +} diff --git a/contrib/lizard/programs/lizardio.c b/contrib/lizard/programs/lizardio.c new file mode 100644 index 00000000000..7638a55178a --- /dev/null +++ b/contrib/lizard/programs/lizardio.c @@ -0,0 +1,896 @@ +/* + Lizardio.c - Lizard File/Stream Interface + Copyright (C) Yann Collet 2011-2015 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ +/* + Note : this is stand-alone program. + It is not part of Lizard compression library, it is a user code of the Lizard library. + - The license of Lizard library is BSD. + - The license of xxHash library is BSD. + - The license of this source file is GPLv2. +*/ + +/*-************************************ +* Compiler options +**************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +#endif +#if defined(__MINGW32__) && !defined(_POSIX_SOURCE) +# define _POSIX_SOURCE 1 /* disable %llu warnings with MinGW on Windows */ +#endif + + +/***************************** +* Includes +*****************************/ +#include "platform.h" /* Large File Support, SET_BINARY_MODE, SET_SPARSE_FILE_MODE, PLATFORM_POSIX_VERSION, __64BIT__ */ +#include "util.h" /* UTIL_getFileStat, UTIL_setFileStat */ +#include /* fprintf, fopen, fread, stdin, stdout, fflush, getchar */ +#include /* malloc, free */ +#include /* strcmp, strlen */ +#include /* clock */ +#include /* stat64 */ +#include /* stat64 */ +#include "lizardio.h" +#include "lizard_frame.h" + + + +/***************************** +* Constants +*****************************/ +#define KB *(1 <<10) +#define MB *(1 <<20) +#define GB *(1U<<30) + +#define _1BIT 0x01 +#define _2BITS 0x03 +#define _3BITS 0x07 +#define _4BITS 0x0F +#define _8BITS 0xFF + +#define MAGICNUMBER_SIZE 4 +#define LIZARDIO_MAGICNUMBER 0x184D2206U +#define LIZARDIO_SKIPPABLE0 0x184D2A50U +#define LIZARDIO_SKIPPABLEMASK 0xFFFFFFF0U + +#define CACHELINE 64 +#define MIN_STREAM_BUFSIZE (192 KB) +#define LIZARDIO_BLOCKSIZEID_DEFAULT 7 + +#define sizeT sizeof(size_t) +#define maskT (sizeT - 1) + + +/************************************** +* Macros +**************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define DISPLAYLEVEL(l, ...) if (g_displayLevel>=l) { DISPLAY(__VA_ARGS__); } +static int g_displayLevel = 0; /* 0 : no display ; 1: errors ; 2 : + result + interaction + warnings ; 3 : + progression; 4 : + information */ + +#define DISPLAYUPDATE(l, ...) if (g_displayLevel>=l) { \ + if (((clock_t)(g_time - clock()) > refreshRate) || (g_displayLevel>=4)) \ + { g_time = clock(); DISPLAY(__VA_ARGS__); \ + if (g_displayLevel>=4) fflush(stderr); } } +static const clock_t refreshRate = CLOCKS_PER_SEC / 6; +static clock_t g_time = 0; + + +/************************************** +* Local Parameters +**************************************/ +static int g_overwrite = 1; +static int g_testMode = 0; +static int g_blockSizeId = LIZARDIO_BLOCKSIZEID_DEFAULT; +static int g_blockChecksum = 0; +static int g_streamChecksum = 1; +static int g_blockIndependence = 1; +static int g_sparseFileSupport = 1; +static int g_contentSizeFlag = 0; + + +/************************************** +* Exceptions +***************************************/ +#ifndef DEBUG +# define DEBUG 0 +#endif +#define DEBUGOUTPUT(...) if (DEBUG) DISPLAY(__VA_ARGS__); +#define EXM_THROW(error, ...) \ +{ \ + DEBUGOUTPUT("Error defined at %s, line %i : \n", __FILE__, __LINE__); \ + DISPLAYLEVEL(1, "Error %i : ", error); \ + DISPLAYLEVEL(1, __VA_ARGS__); \ + DISPLAYLEVEL(1, " \n"); \ + exit(error); \ +} + + +/************************************** +* Version modifiers +**************************************/ +#define EXTENDED_ARGUMENTS +#define EXTENDED_HELP +#define EXTENDED_FORMAT +#define DEFAULT_DECOMPRESSOR LizardIO_decompressLizardF + + +/* ************************************************** */ +/* ****************** Parameters ******************** */ +/* ************************************************** */ + +/* Default setting : overwrite = 1; return : overwrite mode (0/1) */ +int LizardIO_setOverwrite(int yes) +{ + g_overwrite = (yes!=0); + return g_overwrite; +} + +/* Default setting : testMode = 0; return : testMode (0/1) */ +int LizardIO_setTestMode(int yes) +{ + g_testMode = (yes!=0); + return g_testMode; +} + +/* blockSizeID : valid values : 1-7 */ +size_t LizardIO_setBlockSizeID(unsigned bsid) +{ + static const int blockSizeTable[] = { 128 KB, 256 KB, 1 MB, 4 MB, 16 MB, 64 MB, 256 MB }; + static const unsigned minBlockSizeID = 1; + static const unsigned maxBlockSizeID = 7; + if ((bsid < minBlockSizeID) || (bsid > maxBlockSizeID)) return 0; + + g_blockSizeId = bsid; + return blockSizeTable[g_blockSizeId-minBlockSizeID]; +} + + +static size_t LizardIO_GetBlockSize_FromBlockId(unsigned blockSizeID) +{ + static const size_t blockSizes[7] = { 128 KB, 256 KB, 1 MB, 4 MB, 16 MB, 64 MB, 256 MB }; + + if (blockSizeID == 0) blockSizeID = LIZARDIO_BLOCKSIZEID_DEFAULT; + blockSizeID -= 1; + if (blockSizeID >= 7) blockSizeID = LIZARDIO_BLOCKSIZEID_DEFAULT - 1; + + return blockSizes[blockSizeID]; +} + + + +int LizardIO_setBlockMode(LizardIO_blockMode_t blockMode) +{ + g_blockIndependence = (blockMode == LizardIO_blockIndependent); + return g_blockIndependence; +} + +/* Default setting : no checksum */ +int LizardIO_setBlockChecksumMode(int xxhash) +{ + g_blockChecksum = (xxhash != 0); + return g_blockChecksum; +} + +/* Default setting : checksum enabled */ +int LizardIO_setStreamChecksumMode(int xxhash) +{ + g_streamChecksum = (xxhash != 0); + return g_streamChecksum; +} + +/* Default setting : 0 (no notification) */ +int LizardIO_setNotificationLevel(int level) +{ + g_displayLevel = level; + return g_displayLevel; +} + +/* Default setting : 0 (disabled) */ +int LizardIO_setSparseFile(int enable) +{ + g_sparseFileSupport = (enable!=0); + return g_sparseFileSupport; +} + +/* Default setting : 0 (disabled) */ +int LizardIO_setContentSize(int enable) +{ + g_contentSizeFlag = (enable!=0); + return g_contentSizeFlag; +} + +static U32 g_removeSrcFile = 0; +void LizardIO_setRemoveSrcFile(unsigned flag) { g_removeSrcFile = (flag>0); } + + + +/* ************************************************************************ ** +** ********************** Lizard File / Pipe compression ********************* ** +** ************************************************************************ */ + +static int LizardIO_isSkippableMagicNumber(unsigned int magic) { return (magic & LIZARDIO_SKIPPABLEMASK) == LIZARDIO_SKIPPABLE0; } + + +/** LizardIO_openSrcFile() : + * condition : `dstFileName` must be non-NULL. + * @result : FILE* to `dstFileName`, or NULL if it fails */ +static FILE* LizardIO_openSrcFile(const char* srcFileName) +{ + FILE* f; + + if (!strcmp (srcFileName, stdinmark)) { + DISPLAYLEVEL(4,"Using stdin for input\n"); + f = stdin; + SET_BINARY_MODE(stdin); + } else { + f = fopen(srcFileName, "rb"); + if ( f==NULL ) DISPLAYLEVEL(1, "%s: %s \n", srcFileName, strerror(errno)); + } + + return f; +} + +/** FIO_openDstFile() : + * condition : `dstFileName` must be non-NULL. + * @result : FILE* to `dstFileName`, or NULL if it fails */ +static FILE* LizardIO_openDstFile(const char* dstFileName) +{ + FILE* f; + + if (!strcmp (dstFileName, stdoutmark)) { + DISPLAYLEVEL(4,"Using stdout for output\n"); + f = stdout; + SET_BINARY_MODE(stdout); + if (g_sparseFileSupport==1) { + g_sparseFileSupport = 0; + DISPLAYLEVEL(4, "Sparse File Support is automatically disabled on stdout ; try --sparse \n"); + } + } else { + if (!g_overwrite && strcmp (dstFileName, nulmark)) { /* Check if destination file already exists */ + f = fopen( dstFileName, "rb" ); + if (f != NULL) { /* dest exists, prompt for overwrite authorization */ + fclose(f); + if (g_displayLevel <= 1) { /* No interaction possible */ + DISPLAY("%s already exists; not overwritten \n", dstFileName); + return NULL; + } + DISPLAY("%s already exists; do you wish to overwrite (y/N) ? ", dstFileName); + { int ch = getchar(); + if ((ch!='Y') && (ch!='y')) { + DISPLAY(" not overwritten \n"); + return NULL; + } + while ((ch!=EOF) && (ch!='\n')) ch = getchar(); /* flush rest of input line */ + } } } + f = fopen( dstFileName, "wb" ); + if (f==NULL) DISPLAYLEVEL(1, "%s: %s\n", dstFileName, strerror(errno)); + } + + /* sparse file */ + if (f && g_sparseFileSupport) { SET_SPARSE_FILE_MODE(f); } + + return f; +} + + +/* unoptimized version; solves endianess & alignment issues */ +static void LizardIO_writeLE32 (void* p, unsigned value32) +{ + unsigned char* dstPtr = (unsigned char*)p; + dstPtr[0] = (unsigned char)value32; + dstPtr[1] = (unsigned char)(value32 >> 8); + dstPtr[2] = (unsigned char)(value32 >> 16); + dstPtr[3] = (unsigned char)(value32 >> 24); +} + + + +/********************************************* +* Compression using Frame format +*********************************************/ + +typedef struct { + void* srcBuffer; + size_t srcBufferSize; + void* dstBuffer; + size_t dstBufferSize; + LizardF_compressionContext_t ctx; +} cRess_t; + +static cRess_t LizardIO_createCResources(void) +{ + const size_t blockSize = (size_t)LizardIO_GetBlockSize_FromBlockId (g_blockSizeId); + cRess_t ress; + + LizardF_errorCode_t const errorCode = LizardF_createCompressionContext(&(ress.ctx), LIZARDF_VERSION); + if (LizardF_isError(errorCode)) EXM_THROW(30, "Allocation error : can't create LizardF context : %s", LizardF_getErrorName(errorCode)); + + /* Allocate Memory */ + ress.srcBuffer = malloc(blockSize); + ress.srcBufferSize = blockSize; + ress.dstBufferSize = LizardF_compressFrameBound(blockSize, NULL); /* cover worst case */ + ress.dstBuffer = malloc(ress.dstBufferSize); + if (!ress.srcBuffer || !ress.dstBuffer) EXM_THROW(31, "Allocation error : not enough memory"); + + return ress; +} + +static void LizardIO_freeCResources(cRess_t ress) +{ + free(ress.srcBuffer); + free(ress.dstBuffer); + { LizardF_errorCode_t const errorCode = LizardF_freeCompressionContext(ress.ctx); + if (LizardF_isError(errorCode)) EXM_THROW(38, "Error : can't free LizardF context resource : %s", LizardF_getErrorName(errorCode)); } +} + +/* + * LizardIO_compressFilename_extRess() + * result : 0 : compression completed correctly + * 1 : missing or pb opening srcFileName + */ +static int LizardIO_compressFilename_extRess(cRess_t ress, const char* srcFileName, const char* dstFileName, int compressionLevel) +{ + unsigned long long filesize = 0; + unsigned long long compressedfilesize = 0; + FILE* srcFile; + FILE* dstFile; + void* const srcBuffer = ress.srcBuffer; + void* const dstBuffer = ress.dstBuffer; + const size_t dstBufferSize = ress.dstBufferSize; + const size_t blockSize = (size_t)LizardIO_GetBlockSize_FromBlockId (g_blockSizeId); + size_t readSize; + LizardF_compressionContext_t ctx = ress.ctx; /* just a pointer */ + LizardF_preferences_t prefs; + + /* Init */ + srcFile = LizardIO_openSrcFile(srcFileName); + if (srcFile == NULL) return 1; + dstFile = LizardIO_openDstFile(dstFileName); + if (dstFile == NULL) { fclose(srcFile); return 1; } + memset(&prefs, 0, sizeof(prefs)); + + + /* Set compression parameters */ + prefs.autoFlush = 1; + prefs.compressionLevel = compressionLevel; + prefs.frameInfo.blockMode = (LizardF_blockMode_t)g_blockIndependence; + prefs.frameInfo.blockSizeID = (LizardF_blockSizeID_t)g_blockSizeId; + prefs.frameInfo.contentChecksumFlag = (LizardF_contentChecksum_t)g_streamChecksum; + if (g_contentSizeFlag) { + U64 const fileSize = UTIL_getFileSize(srcFileName); + prefs.frameInfo.contentSize = fileSize; /* == 0 if input == stdin */ + if (fileSize==0) + DISPLAYLEVEL(3, "Warning : cannot determine input content size \n"); + } + + /* read first block */ + readSize = fread(srcBuffer, (size_t)1, blockSize, srcFile); + if (ferror(srcFile)) EXM_THROW(30, "Error reading %s ", srcFileName); + filesize += readSize; + + /* single-block file */ + if (readSize < blockSize) { + /* Compress in single pass */ + size_t const cSize = LizardF_compressFrame(dstBuffer, dstBufferSize, srcBuffer, readSize, &prefs); + if (LizardF_isError(cSize)) EXM_THROW(31, "Compression failed : %s", LizardF_getErrorName(cSize)); + compressedfilesize = cSize; + DISPLAYUPDATE(2, "\rRead : %u MB ==> %.2f%% ", + (unsigned)(filesize>>20), (double)compressedfilesize/(filesize+!filesize)*100); /* avoid division by zero */ + + /* Write Block */ + { size_t const sizeCheck = fwrite(dstBuffer, 1, cSize, dstFile); + if (sizeCheck!=cSize) EXM_THROW(32, "Write error : cannot write compressed block"); + } } + + else + + /* multiple-blocks file */ + { + /* Write Archive Header */ + size_t headerSize = LizardF_compressBegin(ctx, dstBuffer, dstBufferSize, &prefs); + if (LizardF_isError(headerSize)) EXM_THROW(33, "File header generation failed : %s", LizardF_getErrorName(headerSize)); + { size_t const sizeCheck = fwrite(dstBuffer, 1, headerSize, dstFile); + if (sizeCheck!=headerSize) EXM_THROW(34, "Write error : cannot write header"); } + compressedfilesize += headerSize; + + /* Main Loop */ + while (readSize>0) { + size_t outSize; + + /* Compress Block */ + outSize = LizardF_compressUpdate(ctx, dstBuffer, dstBufferSize, srcBuffer, readSize, NULL); + if (LizardF_isError(outSize)) EXM_THROW(35, "Compression failed : %s", LizardF_getErrorName(outSize)); + compressedfilesize += outSize; + DISPLAYUPDATE(2, "\rRead : %u MB ==> %.2f%% ", (unsigned)(filesize>>20), (double)compressedfilesize/filesize*100); + + /* Write Block */ + { size_t const sizeCheck = fwrite(dstBuffer, 1, outSize, dstFile); + if (sizeCheck!=outSize) EXM_THROW(36, "Write error : cannot write compressed block"); } + + /* Read next block */ + readSize = fread(srcBuffer, (size_t)1, (size_t)blockSize, srcFile); + filesize += readSize; + } + if (ferror(srcFile)) EXM_THROW(37, "Error reading %s ", srcFileName); + + /* End of Stream mark */ + headerSize = LizardF_compressEnd(ctx, dstBuffer, dstBufferSize, NULL); + if (LizardF_isError(headerSize)) EXM_THROW(38, "End of file generation failed : %s", LizardF_getErrorName(headerSize)); + + { size_t const sizeCheck = fwrite(dstBuffer, 1, headerSize, dstFile); + if (sizeCheck!=headerSize) EXM_THROW(39, "Write error : cannot write end of stream"); } + compressedfilesize += headerSize; + } + + /* Release files */ + fclose (srcFile); + fclose (dstFile); + + /* Copy owner, file permissions and modification time */ + { stat_t statbuf; + if (strcmp (srcFileName, stdinmark) && strcmp (dstFileName, stdoutmark) && UTIL_getFileStat(srcFileName, &statbuf)) + UTIL_setFileStat(dstFileName, &statbuf); + } + + if (g_removeSrcFile) { if (remove(srcFileName)) EXM_THROW(40, "Remove error : %s: %s", srcFileName, strerror(errno)); } /* remove source file : --rm */ + + /* Final Status */ + DISPLAYLEVEL(2, "\r%79s\r", ""); + DISPLAYLEVEL(2, "Compressed %llu bytes into %llu bytes ==> %.2f%%\n", + filesize, compressedfilesize, (double)compressedfilesize/(filesize + !filesize)*100); /* avoid division by zero */ + + return 0; +} + + +int LizardIO_compressFilename(const char* srcFileName, const char* dstFileName, int compressionLevel) +{ + clock_t const start = clock(); + cRess_t const ress = LizardIO_createCResources(); + + int const issueWithSrcFile = LizardIO_compressFilename_extRess(ress, srcFileName, dstFileName, compressionLevel); + + /* Free resources */ + LizardIO_freeCResources(ress); + + /* Final Status */ + { clock_t const end = clock(); + double const seconds = (double)(end - start) / CLOCKS_PER_SEC; + DISPLAYLEVEL(4, "Completed in %.2f sec \n", seconds); + } + + return issueWithSrcFile; +} + + +#define FNSPACE 30 +int LizardIO_compressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix, int compressionLevel) +{ + int i; + int missed_files = 0; + char* dstFileName = (char*)malloc(FNSPACE); + size_t ofnSize = FNSPACE; + const size_t suffixSize = strlen(suffix); + cRess_t const ress = LizardIO_createCResources(); + + if (dstFileName == NULL) return ifntSize; /* not enough memory */ + + /* loop on each file */ + for (i=0; i 1 GB) { + int const seekResult = UTIL_fseek(file, 1 GB, SEEK_CUR); + if (seekResult != 0) EXM_THROW(71, "1 GB skip error (sparse file support)"); + storedSkips -= 1 GB; + } + + while (ptrT < bufferTEnd) { + size_t seg0SizeT = segmentSizeT; + size_t nb0T; + + /* count leading zeros */ + if (seg0SizeT > bufferSizeT) seg0SizeT = bufferSizeT; + bufferSizeT -= seg0SizeT; + for (nb0T=0; (nb0T < seg0SizeT) && (ptrT[nb0T] == 0); nb0T++) ; + storedSkips += (unsigned)(nb0T * sizeT); + + if (nb0T != seg0SizeT) { /* not all 0s */ + int const seekResult = UTIL_fseek(file, storedSkips, SEEK_CUR); + if (seekResult) EXM_THROW(72, "Sparse skip error ; try --no-sparse"); + storedSkips = 0; + seg0SizeT -= nb0T; + ptrT += nb0T; + { size_t const sizeCheck = fwrite(ptrT, sizeT, seg0SizeT, file); + if (sizeCheck != seg0SizeT) EXM_THROW(73, "Write error : cannot write decoded block"); + } } + ptrT += seg0SizeT; + } + + if (bufferSize & maskT) { /* size not multiple of sizeT : implies end of block */ + const char* const restStart = (const char*)bufferTEnd; + const char* restPtr = restStart; + size_t const restSize = bufferSize & maskT; + const char* const restEnd = restStart + restSize; + for (; (restPtr < restEnd) && (*restPtr == 0); restPtr++) ; + storedSkips += (unsigned) (restPtr - restStart); + if (restPtr != restEnd) { + int const seekResult = UTIL_fseek(file, storedSkips, SEEK_CUR); + if (seekResult) EXM_THROW(74, "Sparse skip error ; try --no-sparse"); + storedSkips = 0; + { size_t const sizeCheck = fwrite(restPtr, 1, restEnd - restPtr, file); + if (sizeCheck != (size_t)(restEnd - restPtr)) EXM_THROW(75, "Write error : cannot write decoded end of block"); + } } + } + + return storedSkips; +} + +static void LizardIO_fwriteSparseEnd(FILE* file, unsigned storedSkips) +{ + if (storedSkips>0) { /* implies g_sparseFileSupport>0 */ + int const seekResult = UTIL_fseek(file, storedSkips-1, SEEK_CUR); + if (seekResult != 0) EXM_THROW(69, "Final skip error (sparse file)\n"); + { const char lastZeroByte[1] = { 0 }; + size_t const sizeCheck = fwrite(lastZeroByte, 1, 1, file); + if (sizeCheck != 1) EXM_THROW(69, "Write error : cannot write last zero\n"); + } } +} + + + +typedef struct { + void* srcBuffer; + size_t srcBufferSize; + void* dstBuffer; + size_t dstBufferSize; + FILE* dstFile; + LizardF_decompressionContext_t dCtx; +} dRess_t; + +static const size_t LizardIO_dBufferSize = 64 KB; +static unsigned g_magicRead = 0; +static dRess_t LizardIO_createDResources(void) +{ + dRess_t ress; + + /* init */ + LizardF_errorCode_t const errorCode = LizardF_createDecompressionContext(&ress.dCtx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) EXM_THROW(60, "Can't create LizardF context : %s", LizardF_getErrorName(errorCode)); + + /* Allocate Memory */ + ress.srcBufferSize = LizardIO_dBufferSize; + ress.srcBuffer = malloc(ress.srcBufferSize); + ress.dstBufferSize = LizardIO_dBufferSize; + ress.dstBuffer = malloc(ress.dstBufferSize); + if (!ress.srcBuffer || !ress.dstBuffer) EXM_THROW(61, "Allocation error : not enough memory"); + + ress.dstFile = NULL; + return ress; +} + +static void LizardIO_freeDResources(dRess_t ress) +{ + LizardF_errorCode_t errorCode = LizardF_freeDecompressionContext(ress.dCtx); + if (LizardF_isError(errorCode)) EXM_THROW(69, "Error : can't free LizardF context resource : %s", LizardF_getErrorName(errorCode)); + free(ress.srcBuffer); + free(ress.dstBuffer); +} + + +static unsigned long long LizardIO_decompressLizardF(dRess_t ress, FILE* srcFile, FILE* dstFile) +{ + unsigned long long filesize = 0; + LizardF_errorCode_t nextToLoad; + unsigned storedSkips = 0; + + /* Init feed with magic number (already consumed from FILE* sFile) */ + { size_t inSize = MAGICNUMBER_SIZE; + size_t outSize= 0; + LizardIO_writeLE32(ress.srcBuffer, LIZARDIO_MAGICNUMBER); + nextToLoad = LizardF_decompress(ress.dCtx, ress.dstBuffer, &outSize, ress.srcBuffer, &inSize, NULL); + if (LizardF_isError(nextToLoad)) EXM_THROW(62, "Header error : %s", LizardF_getErrorName(nextToLoad)); + } + + /* Main Loop */ + for (;nextToLoad;) { + size_t readSize; + size_t pos = 0; + size_t decodedBytes = ress.dstBufferSize; + + /* Read input */ + if (nextToLoad > ress.srcBufferSize) nextToLoad = ress.srcBufferSize; + readSize = fread(ress.srcBuffer, 1, nextToLoad, srcFile); + if (!readSize) break; /* reached end of file or stream */ + + while ((pos < readSize) || (decodedBytes == ress.dstBufferSize)) { /* still to read, or still to flush */ + /* Decode Input (at least partially) */ + size_t remaining = readSize - pos; + decodedBytes = ress.dstBufferSize; + nextToLoad = LizardF_decompress(ress.dCtx, ress.dstBuffer, &decodedBytes, (char*)(ress.srcBuffer)+pos, &remaining, NULL); + if (LizardF_isError(nextToLoad)) EXM_THROW(66, "Decompression error : %s", LizardF_getErrorName(nextToLoad)); + pos += remaining; + + /* Write Block */ + if (decodedBytes) { + if (!g_testMode) + storedSkips = LizardIO_fwriteSparse(dstFile, ress.dstBuffer, decodedBytes, storedSkips); + filesize += decodedBytes; + DISPLAYUPDATE(2, "\rDecompressed : %u MB ", (unsigned)(filesize>>20)); + } + + if (!nextToLoad) break; + } + } + /* can be out because readSize == 0, which could be an fread() error */ + if (ferror(srcFile)) EXM_THROW(67, "Read error"); + + if (!g_testMode) LizardIO_fwriteSparseEnd(dstFile, storedSkips); + if (nextToLoad!=0) EXM_THROW(68, "Unfinished stream"); + + return filesize; +} + + +#define PTSIZE (64 KB) +#define PTSIZET (PTSIZE / sizeof(size_t)) +static unsigned long long LizardIO_passThrough(FILE* finput, FILE* foutput, unsigned char MNstore[MAGICNUMBER_SIZE]) +{ + size_t buffer[PTSIZET]; + size_t readBytes = 1; + unsigned long long total = MAGICNUMBER_SIZE; + unsigned storedSkips = 0; + + size_t const sizeCheck = fwrite(MNstore, 1, MAGICNUMBER_SIZE, foutput); + if (sizeCheck != MAGICNUMBER_SIZE) EXM_THROW(50, "Pass-through write error"); + + while (readBytes) { + readBytes = fread(buffer, 1, PTSIZE, finput); + total += readBytes; + storedSkips = LizardIO_fwriteSparse(foutput, buffer, readBytes, storedSkips); + } + if (ferror(finput)) EXM_THROW(51, "Read Error") + + LizardIO_fwriteSparseEnd(foutput, storedSkips); + return total; +} + + +/** Safely handle cases when (unsigned)offset > LONG_MAX */ +static int fseek_u32(FILE *fp, unsigned offset, int where) +{ + const unsigned stepMax = 1U << 30; + int errorNb = 0; + + if (where != SEEK_CUR) return -1; /* Only allows SEEK_CUR */ + while (offset > 0) { + unsigned s = offset; + if (s > stepMax) s = stepMax; + errorNb = UTIL_fseek(fp, (long) s, SEEK_CUR); + if (errorNb != 0) break; + offset -= s; + } + return errorNb; +} + +#define ENDOFSTREAM ((unsigned long long)-1) +static unsigned long long selectDecoder(dRess_t ress, FILE* finput, FILE* foutput) +{ + unsigned char MNstore[MAGICNUMBER_SIZE]; + unsigned magicNumber; + static unsigned nbCalls = 0; + + /* init */ + nbCalls++; + + /* Check Archive Header */ + if (g_magicRead) { /* magic number already read from finput (see legacy frame)*/ + magicNumber = g_magicRead; + g_magicRead = 0; + } else { + size_t const nbReadBytes = fread(MNstore, 1, MAGICNUMBER_SIZE, finput); + if (nbReadBytes==0) { nbCalls = 0; return ENDOFSTREAM; } /* EOF */ + if (nbReadBytes != MAGICNUMBER_SIZE) EXM_THROW(40, "Unrecognized header : Magic Number unreadable"); + magicNumber = LizardIO_readLE32(MNstore); /* Little Endian format */ + } + if (LizardIO_isSkippableMagicNumber(magicNumber)) magicNumber = LIZARDIO_SKIPPABLE0; /* fold skippable magic numbers */ + + switch(magicNumber) + { + case LIZARDIO_MAGICNUMBER: + return LizardIO_decompressLizardF(ress, finput, foutput); + case LIZARDIO_SKIPPABLE0: + DISPLAYLEVEL(4, "Skipping detected skippable area \n"); + { size_t const nbReadBytes = fread(MNstore, 1, 4, finput); + if (nbReadBytes != 4) EXM_THROW(42, "Stream error : skippable size unreadable"); } + { unsigned const size = LizardIO_readLE32(MNstore); /* Little Endian format */ + int const errorNb = fseek_u32(finput, size, SEEK_CUR); + if (errorNb != 0) EXM_THROW(43, "Stream error : cannot skip skippable area"); } + return 0; + EXTENDED_FORMAT; /* macro extension for custom formats */ + default: + if (nbCalls == 1) { /* just started */ + if (!g_testMode && g_overwrite) { + nbCalls = 0; + return LizardIO_passThrough(finput, foutput, MNstore); + } + EXM_THROW(44,"Unrecognized header : file cannot be decoded"); /* Wrong magic number at the beginning of 1st stream */ + } + DISPLAYLEVEL(2, "Stream followed by undecodable data\n"); + return ENDOFSTREAM; + } +} + + +static int LizardIO_decompressSrcFile(dRess_t ress, const char* input_filename, const char* output_filename) +{ + FILE* const foutput = ress.dstFile; + unsigned long long filesize = 0, decodedSize=0; + FILE* finput; + + /* Init */ + finput = LizardIO_openSrcFile(input_filename); + if (finput==NULL) return 1; + + /* Loop over multiple streams */ + do { + decodedSize = selectDecoder(ress, finput, foutput); + if (decodedSize != ENDOFSTREAM) + filesize += decodedSize; + } while (decodedSize != ENDOFSTREAM); + + /* Close */ + fclose(finput); + + if (g_removeSrcFile) { if (remove(input_filename)) EXM_THROW(45, "Remove error : %s: %s", input_filename, strerror(errno)); } /* remove source file : --rm */ + + /* Final Status */ + DISPLAYLEVEL(2, "\r%79s\r", ""); + DISPLAYLEVEL(2, "%-20.20s : decoded %llu bytes \n", input_filename, filesize); + (void)output_filename; + + return 0; +} + + +static int LizardIO_decompressDstFile(dRess_t ress, const char* input_filename, const char* output_filename) +{ + FILE* foutput; + + /* Init */ + foutput = LizardIO_openDstFile(output_filename); + if (foutput==NULL) return 1; /* failure */ + + ress.dstFile = foutput; + LizardIO_decompressSrcFile(ress, input_filename, output_filename); + + fclose(foutput); + + /* Copy owner, file permissions and modification time */ + { stat_t statbuf; + if (strcmp (input_filename, stdinmark) && strcmp (output_filename, stdoutmark) && UTIL_getFileStat(input_filename, &statbuf)) + UTIL_setFileStat(output_filename, &statbuf); + } + + return 0; +} + + +int LizardIO_decompressFilename(const char* input_filename, const char* output_filename) +{ + dRess_t const ress = LizardIO_createDResources(); + clock_t const start = clock(); + + int const missingFiles = LizardIO_decompressDstFile(ress, input_filename, output_filename); + + { clock_t const end = clock(); + double const seconds = (double)(end - start) / CLOCKS_PER_SEC; + DISPLAYLEVEL(4, "Done in %.2f sec \n", seconds); + } + + LizardIO_freeDResources(ress); + return missingFiles; +} + + +int LizardIO_decompressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix) +{ + int i; + int skippedFiles = 0; + int missingFiles = 0; + char* outFileName = (char*)malloc(FNSPACE); + size_t ofnSize = FNSPACE; + size_t const suffixSize = strlen(suffix); + dRess_t ress = LizardIO_createDResources(); + + if (outFileName==NULL) return ifntSize; /* not enough memory */ + ress.dstFile = LizardIO_openDstFile(stdoutmark); + + for (i=0; i /* size_t */ + + +/* ************************************************** */ +/* Special input/output values */ +/* ************************************************** */ +#define NULL_OUTPUT "null" +static const char stdinmark[] = "stdin"; +static const char stdoutmark[] = "stdout"; +#ifdef _WIN32 +static const char nulmark[] = "nul"; +#else +static const char nulmark[] = "/dev/null"; +#endif + + +/* ************************************************** */ +/* ****************** Functions ********************* */ +/* ************************************************** */ + +int LizardIO_compressFilename (const char* input_filename, const char* output_filename, int compressionlevel); +int LizardIO_decompressFilename(const char* input_filename, const char* output_filename); + +int LizardIO_compressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix, int compressionlevel); +int LizardIO_decompressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix); + + +/* ************************************************** */ +/* ****************** Parameters ******************** */ +/* ************************************************** */ + +/* Default setting : overwrite = 1; + return : overwrite mode (0/1) */ +int LizardIO_setOverwrite(int yes); + +/* Default setting : testMode = 0; + return : testMode (0/1) */ +int LizardIO_setTestMode(int yes); + +/* blockSizeID : valid values : 4-5-6-7 + return : 0 if error, blockSize if OK */ +size_t LizardIO_setBlockSizeID(unsigned blockSizeID); + +/* Default setting : independent blocks */ +typedef enum { LizardIO_blockLinked=0, LizardIO_blockIndependent} LizardIO_blockMode_t; +int LizardIO_setBlockMode(LizardIO_blockMode_t blockMode); + +/* Default setting : no block checksum */ +int LizardIO_setBlockChecksumMode(int xxhash); + +/* Default setting : stream checksum enabled */ +int LizardIO_setStreamChecksumMode(int xxhash); + +/* Default setting : 0 (no notification) */ +int LizardIO_setNotificationLevel(int level); + +/* Default setting : 0 (disabled) */ +int LizardIO_setSparseFile(int enable); + +/* Default setting : 0 (disabled) */ +int LizardIO_setContentSize(int enable); + +void LizardIO_setRemoveSrcFile(unsigned flag); + + +#endif /* LIZARDIO_H_237902873 */ diff --git a/contrib/lizard/programs/platform.h b/contrib/lizard/programs/platform.h new file mode 100644 index 00000000000..51ce1ac2d14 --- /dev/null +++ b/contrib/lizard/programs/platform.h @@ -0,0 +1,145 @@ +/* + platform.h - compiler and OS detection + Copyright (C) 2016-present, Przemyslaw Skibinski, Yann Collet + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. +*/ + +#ifndef PLATFORM_H_MODULE +#define PLATFORM_H_MODULE + +#if defined (__cplusplus) +extern "C" { +#endif + + + +/* ************************************** +* Compiler Options +****************************************/ +#if defined(_MSC_VER) +# define _CRT_SECURE_NO_WARNINGS /* Disable Visual Studio warning messages for fopen, strncpy, strerror */ +# define _CRT_SECURE_NO_DEPRECATE /* VS2005 - must be declared before and */ +# if (_MSC_VER <= 1800) /* (1800 = Visual Studio 2013) */ +# define snprintf sprintf_s /* snprintf unsupported by Visual <= 2013 */ +# endif +#endif + + +/* ************************************** +* Detect 64-bit OS +* http://nadeausoftware.com/articles/2012/02/c_c_tip_how_detect_processor_type_using_compiler_predefined_macros +****************************************/ +#if defined __ia64 || defined _M_IA64 /* Intel Itanium */ \ + || defined __powerpc64__ || defined __ppc64__ || defined __PPC64__ /* POWER 64-bit */ \ + || (defined __sparc && (defined __sparcv9 || defined __sparc_v9__ || defined __arch64__)) || defined __sparc64__ /* SPARC 64-bit */ \ + || defined __x86_64__s || defined _M_X64 /* x86 64-bit */ \ + || defined __arm64__ || defined __aarch64__ || defined __ARM64_ARCH_8__ /* ARM 64-bit */ \ + || (defined __mips && (__mips == 64 || __mips == 4 || __mips == 3)) /* MIPS 64-bit */ \ + || defined _LP64 || defined __LP64__ /* NetBSD, OpenBSD */ || defined __64BIT__ /* AIX */ || defined _ADDR64 /* Cray */ \ + || (defined __SIZEOF_POINTER__ && __SIZEOF_POINTER__ == 8) /* gcc */ +# if !defined(__64BIT__) +# define __64BIT__ 1 +# endif +#endif + + +/* ********************************************************* +* Turn on Large Files support (>4GB) for 32-bit Linux/Unix +***********************************************************/ +#if !defined(__64BIT__) || defined(__MINGW32__) /* No point defining Large file for 64 bit but MinGW-w64 requires it */ +# if !defined(_FILE_OFFSET_BITS) +# define _FILE_OFFSET_BITS 64 /* turn off_t into a 64-bit type for ftello, fseeko */ +# endif +# if !defined(_LARGEFILE_SOURCE) /* obsolete macro, replaced with _FILE_OFFSET_BITS */ +# define _LARGEFILE_SOURCE 1 /* Large File Support extension (LFS) - fseeko, ftello */ +# endif +# if defined(_AIX) || defined(__hpux) +# define _LARGE_FILES /* Large file support on 32-bits AIX and HP-UX */ +# endif +#endif + + +/* ************************************************************ +* Detect POSIX version +* PLATFORM_POSIX_VERSION = -1 for non-Unix e.g. Windows +* PLATFORM_POSIX_VERSION = 0 for Unix-like non-POSIX +* PLATFORM_POSIX_VERSION >= 1 is equal to found _POSIX_VERSION +***************************************************************/ +#if !defined(_WIN32) && (defined(__unix__) || defined(__unix) || (defined(__APPLE__) && defined(__MACH__)) /* UNIX-like OS */ \ + || defined(__midipix__) || defined(__VMS)) +# if (defined(__APPLE__) && defined(__MACH__)) || defined(__SVR4) || defined(_AIX) || defined(__hpux) /* POSIX.1–2001 (SUSv3) conformant */ \ + || defined(__DragonFly__) || defined(__FreeBSD__) || defined(__NetBSD__) || defined(__OpenBSD__) /* BSD distros */ +# define PLATFORM_POSIX_VERSION 200112L +# else +# if defined(__linux__) || defined(__linux) +# ifndef _POSIX_C_SOURCE +# define _POSIX_C_SOURCE 200112L /* use feature test macro */ +# endif +# endif +# include /* declares _POSIX_VERSION */ +# if defined(_POSIX_VERSION) /* POSIX compliant */ +# define PLATFORM_POSIX_VERSION _POSIX_VERSION +# else +# define PLATFORM_POSIX_VERSION 0 +# endif +# endif +#endif +#if !defined(PLATFORM_POSIX_VERSION) +# define PLATFORM_POSIX_VERSION -1 +#endif + + +/*-********************************************* +* Detect if isatty() and fileno() are available +************************************************/ +#if (defined(__linux__) && (PLATFORM_POSIX_VERSION >= 1)) || (PLATFORM_POSIX_VERSION >= 200112L) || defined(__DJGPP__) +# include /* isatty */ +# define IS_CONSOLE(stdStream) isatty(fileno(stdStream)) +#elif defined(MSDOS) || defined(OS2) || defined(WIN32) || defined(_WIN32) || defined(__CYGWIN__) +# include /* _isatty */ +# define IS_CONSOLE(stdStream) _isatty(_fileno(stdStream)) +#else +# define IS_CONSOLE(stdStream) 0 +#endif + + +/****************************** +* OS-specific Includes +******************************/ +#if defined(MSDOS) || defined(OS2) || defined(WIN32) || defined(_WIN32) +# include /* _O_BINARY */ +# include /* _setmode, _fileno, _get_osfhandle */ +# if !defined(__DJGPP__) +# include /* DeviceIoControl, HANDLE, FSCTL_SET_SPARSE */ +# include /* FSCTL_SET_SPARSE */ +# define SET_BINARY_MODE(file) { int unused=_setmode(_fileno(file), _O_BINARY); (void)unused; } +# define SET_SPARSE_FILE_MODE(file) { DWORD dw; DeviceIoControl((HANDLE) _get_osfhandle(_fileno(file)), FSCTL_SET_SPARSE, 0, 0, 0, 0, &dw, 0); } +# else +# define SET_BINARY_MODE(file) setmode(fileno(file), O_BINARY) +# define SET_SPARSE_FILE_MODE(file) +# endif +#else +# define SET_BINARY_MODE(file) +# define SET_SPARSE_FILE_MODE(file) +#endif + + + +#if defined (__cplusplus) +} +#endif + +#endif /* PLATFORM_H_MODULE */ diff --git a/contrib/lizard/programs/util.h b/contrib/lizard/programs/util.h new file mode 100644 index 00000000000..4c2313eef00 --- /dev/null +++ b/contrib/lizard/programs/util.h @@ -0,0 +1,497 @@ +/* + util.h - utility functions + Copyright (C) 2016-present, Przemyslaw Skibinski, Yann Collet + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. +*/ + +#ifndef UTIL_H_MODULE +#define UTIL_H_MODULE + +#if defined (__cplusplus) +extern "C" { +#endif + + + +/*-**************************************** +* Dependencies +******************************************/ +#include "platform.h" /* PLATFORM_POSIX_VERSION */ +#include /* malloc */ +#include /* size_t, ptrdiff_t */ +#include /* fprintf */ +#include /* strlen, strncpy */ +#include /* stat, utime */ +#include /* stat */ +#if defined(_MSC_VER) +# include /* utime */ +# include /* _chmod */ +#else +# include /* chown, stat */ +# include /* utime */ +#endif +#include /* time */ +#include + + + +#ifndef UTIL_WITHOUT_BASIC_TYPES +/*-************************************************************** +* Basic Types +*****************************************************************/ +#if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) +# include + typedef uint8_t BYTE; + typedef uint16_t U16; + typedef int16_t S16; + typedef uint32_t U32; + typedef int32_t S32; + typedef uint64_t U64; + typedef int64_t S64; +#else + typedef unsigned char BYTE; + typedef unsigned short U16; + typedef signed short S16; + typedef unsigned int U32; + typedef signed int S32; + typedef unsigned long long U64; + typedef signed long long S64; +#endif +#endif + + +/* ************************************************************ +* Avoid fseek()'s 2GiB barrier with MSVC, MacOS, *BSD, MinGW +***************************************************************/ +#if defined(_MSC_VER) && (_MSC_VER >= 1400) +# define UTIL_fseek _fseeki64 +#elif !defined(__64BIT__) && (PLATFORM_POSIX_VERSION >= 200112L) /* No point defining Large file for 64 bit */ +# define UTIL_fseek fseeko +#elif defined(__MINGW32__) && defined(__MSVCRT__) && !defined(__STRICT_ANSI__) && !defined(__NO_MINGW_LFS) +# define UTIL_fseek fseeko64 +#else +# define UTIL_fseek fseek +#endif + + +/*-**************************************** +* Sleep functions: Windows - Posix - others +******************************************/ +#if defined(_WIN32) +# include +# define SET_REALTIME_PRIORITY SetPriorityClass(GetCurrentProcess(), REALTIME_PRIORITY_CLASS) +# define UTIL_sleep(s) Sleep(1000*s) +# define UTIL_sleepMilli(milli) Sleep(milli) +#elif PLATFORM_POSIX_VERSION >= 0 /* Unix-like operating system */ +# include +# include /* setpriority */ +# include /* clock_t, nanosleep, clock, CLOCKS_PER_SEC */ +# if defined(PRIO_PROCESS) +# define SET_REALTIME_PRIORITY setpriority(PRIO_PROCESS, 0, -20) +# else +# define SET_REALTIME_PRIORITY /* disabled */ +# endif +# define UTIL_sleep(s) sleep(s) +# if (defined(__linux__) && (PLATFORM_POSIX_VERSION >= 199309L)) || (PLATFORM_POSIX_VERSION >= 200112L) /* nanosleep requires POSIX.1-2001 */ +# define UTIL_sleepMilli(milli) { struct timespec t; t.tv_sec=0; t.tv_nsec=milli*1000000ULL; nanosleep(&t, NULL); } +# else +# define UTIL_sleepMilli(milli) /* disabled */ +# endif +#else +# define SET_REALTIME_PRIORITY /* disabled */ +# define UTIL_sleep(s) /* disabled */ +# define UTIL_sleepMilli(milli) /* disabled */ +#endif + + +/* ************************************* +* Constants +***************************************/ +#define LIST_SIZE_INCREASE (8*1024) + + +/*-**************************************** +* Compiler specifics +******************************************/ +#if defined(__INTEL_COMPILER) +# pragma warning(disable : 177) /* disable: message #177: function was declared but never referenced, useful with UTIL_STATIC */ +#endif +#if defined(__GNUC__) +# define UTIL_STATIC static __attribute__((unused)) +#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) +# define UTIL_STATIC static inline +#elif defined(_MSC_VER) +# define UTIL_STATIC static __inline +#else +# define UTIL_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */ +#endif + + +/*-**************************************** +* Time functions +******************************************/ +#if (PLATFORM_POSIX_VERSION >= 1) +#include +#include /* times */ + typedef U64 UTIL_time_t; + UTIL_STATIC void UTIL_initTimer(UTIL_time_t* ticksPerSecond) { *ticksPerSecond=sysconf(_SC_CLK_TCK); } + UTIL_STATIC void UTIL_getTime(UTIL_time_t* x) { struct tms junk; clock_t newTicks = (clock_t) times(&junk); (void)junk; *x = (UTIL_time_t)newTicks; } + UTIL_STATIC U64 UTIL_getSpanTimeMicro(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000ULL * (clockEnd - clockStart) / ticksPerSecond; } + UTIL_STATIC U64 UTIL_getSpanTimeNano(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000000ULL * (clockEnd - clockStart) / ticksPerSecond; } +#elif defined(_WIN32) /* Windows */ + typedef LARGE_INTEGER UTIL_time_t; + UTIL_STATIC void UTIL_initTimer(UTIL_time_t* ticksPerSecond) { if (!QueryPerformanceFrequency(ticksPerSecond)) fprintf(stderr, "ERROR: QueryPerformance not present\n"); } + UTIL_STATIC void UTIL_getTime(UTIL_time_t* x) { QueryPerformanceCounter(x); } + UTIL_STATIC U64 UTIL_getSpanTimeMicro(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000ULL*(clockEnd.QuadPart - clockStart.QuadPart)/ticksPerSecond.QuadPart; } + UTIL_STATIC U64 UTIL_getSpanTimeNano(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000000ULL*(clockEnd.QuadPart - clockStart.QuadPart)/ticksPerSecond.QuadPart; } +#else /* relies on standard C (note : clock_t measurements can be wrong when using multi-threading) */ + typedef clock_t UTIL_time_t; + UTIL_STATIC void UTIL_initTimer(UTIL_time_t* ticksPerSecond) { *ticksPerSecond=0; } + UTIL_STATIC void UTIL_getTime(UTIL_time_t* x) { *x = clock(); } + UTIL_STATIC U64 UTIL_getSpanTimeMicro(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { (void)ticksPerSecond; return 1000000ULL * (clockEnd - clockStart) / CLOCKS_PER_SEC; } + UTIL_STATIC U64 UTIL_getSpanTimeNano(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { (void)ticksPerSecond; return 1000000000ULL * (clockEnd - clockStart) / CLOCKS_PER_SEC; } +#endif + + +/* returns time span in microseconds */ +UTIL_STATIC U64 UTIL_clockSpanMicro( UTIL_time_t clockStart, UTIL_time_t ticksPerSecond ) +{ + UTIL_time_t clockEnd; + UTIL_getTime(&clockEnd); + return UTIL_getSpanTimeMicro(ticksPerSecond, clockStart, clockEnd); +} + + +UTIL_STATIC void UTIL_waitForNextTick(UTIL_time_t ticksPerSecond) +{ + UTIL_time_t clockStart, clockEnd; + UTIL_getTime(&clockStart); + do { + UTIL_getTime(&clockEnd); + } while (UTIL_getSpanTimeNano(ticksPerSecond, clockStart, clockEnd) == 0); +} + + + +/*-**************************************** +* File functions +******************************************/ +#if defined(_MSC_VER) + #define chmod _chmod + typedef struct __stat64 stat_t; +#else + typedef struct stat stat_t; +#endif + + +UTIL_STATIC int UTIL_setFileStat(const char *filename, stat_t *statbuf) +{ + int res = 0; + struct utimbuf timebuf; + + timebuf.actime = time(NULL); + timebuf.modtime = statbuf->st_mtime; + res += utime(filename, &timebuf); /* set access and modification times */ + +#if !defined(_WIN32) + res += chown(filename, statbuf->st_uid, statbuf->st_gid); /* Copy ownership */ +#endif + + res += chmod(filename, statbuf->st_mode & 07777); /* Copy file permissions */ + + errno = 0; + return -res; /* number of errors is returned */ +} + + +UTIL_STATIC int UTIL_getFileStat(const char* infilename, stat_t *statbuf) +{ + int r; +#if defined(_MSC_VER) + r = _stat64(infilename, statbuf); + if (r || !(statbuf->st_mode & S_IFREG)) return 0; /* No good... */ +#else + r = stat(infilename, statbuf); + if (r || !S_ISREG(statbuf->st_mode)) return 0; /* No good... */ +#endif + return 1; +} + + +UTIL_STATIC int UTIL_isRegFile(const char* infilename) +{ + stat_t statbuf; + return UTIL_getFileStat(infilename, &statbuf); /* Only need to know whether it is a regular file */ +} + + +UTIL_STATIC U32 UTIL_isDirectory(const char* infilename) +{ + int r; + stat_t statbuf; +#if defined(_MSC_VER) + r = _stat64(infilename, &statbuf); + if (!r && (statbuf.st_mode & _S_IFDIR)) return 1; +#else + r = stat(infilename, &statbuf); + if (!r && S_ISDIR(statbuf.st_mode)) return 1; +#endif + return 0; +} + + +UTIL_STATIC U64 UTIL_getFileSize(const char* infilename) +{ + int r; +#if defined(_MSC_VER) + struct __stat64 statbuf; + r = _stat64(infilename, &statbuf); + if (r || !(statbuf.st_mode & S_IFREG)) return 0; /* No good... */ +#elif defined(__MINGW32__) && defined (__MSVCRT__) + struct _stati64 statbuf; + r = _stati64(infilename, &statbuf); + if (r || !(statbuf.st_mode & S_IFREG)) return 0; /* No good... */ +#else + struct stat statbuf; + r = stat(infilename, &statbuf); + if (r || !S_ISREG(statbuf.st_mode)) return 0; /* No good... */ +#endif + return (U64)statbuf.st_size; +} + + +UTIL_STATIC U64 UTIL_getTotalFileSize(const char** fileNamesTable, unsigned nbFiles) +{ + U64 total = 0; + unsigned n; + for (n=0; n= *bufEnd) { + ptrdiff_t newListSize = (*bufEnd - *bufStart) + LIST_SIZE_INCREASE; + *bufStart = (char*)UTIL_realloc(*bufStart, newListSize); + *bufEnd = *bufStart + newListSize; + if (*bufStart == NULL) { free(path); FindClose(hFile); return 0; } + } + if (*bufStart + *pos + pathLength < *bufEnd) { + strncpy(*bufStart + *pos, path, *bufEnd - (*bufStart + *pos)); + *pos += pathLength + 1; + nbFiles++; + } + } + free(path); + } while (FindNextFileA(hFile, &cFile)); + + FindClose(hFile); + return nbFiles; +} + +#elif defined(__linux__) || (PLATFORM_POSIX_VERSION >= 200112L) /* opendir, readdir require POSIX.1-2001 */ +# define UTIL_HAS_CREATEFILELIST +# include /* opendir, readdir */ +# include /* strerror, memcpy */ + +UTIL_STATIC int UTIL_prepareFileList(const char *dirName, char** bufStart, size_t* pos, char** bufEnd) +{ + DIR *dir; + struct dirent *entry; + char* path; + int dirLength, fnameLength, pathLength, nbFiles = 0; + + if (!(dir = opendir(dirName))) { + fprintf(stderr, "Cannot open directory '%s': %s\n", dirName, strerror(errno)); + return 0; + } + + dirLength = (int)strlen(dirName); + errno = 0; + while ((entry = readdir(dir)) != NULL) { + if (strcmp (entry->d_name, "..") == 0 || + strcmp (entry->d_name, ".") == 0) continue; + fnameLength = (int)strlen(entry->d_name); + path = (char*) malloc(dirLength + fnameLength + 2); + if (!path) { closedir(dir); return 0; } + memcpy(path, dirName, dirLength); + path[dirLength] = '/'; + memcpy(path+dirLength+1, entry->d_name, fnameLength); + pathLength = dirLength+1+fnameLength; + path[pathLength] = 0; + + if (UTIL_isDirectory(path)) { + nbFiles += UTIL_prepareFileList(path, bufStart, pos, bufEnd); /* Recursively call "UTIL_prepareFileList" with the new path. */ + if (*bufStart == NULL) { free(path); closedir(dir); return 0; } + } else { + if (*bufStart + *pos + pathLength >= *bufEnd) { + ptrdiff_t newListSize = (*bufEnd - *bufStart) + LIST_SIZE_INCREASE; + *bufStart = (char*)UTIL_realloc(*bufStart, newListSize); + *bufEnd = *bufStart + newListSize; + if (*bufStart == NULL) { free(path); closedir(dir); return 0; } + } + if (*bufStart + *pos + pathLength < *bufEnd) { + strncpy(*bufStart + *pos, path, *bufEnd - (*bufStart + *pos)); + *pos += pathLength + 1; + nbFiles++; + } + } + free(path); + errno = 0; /* clear errno after UTIL_isDirectory, UTIL_prepareFileList */ + } + + if (errno != 0) { + fprintf(stderr, "readdir(%s) error: %s\n", dirName, strerror(errno)); + free(*bufStart); + *bufStart = NULL; + } + closedir(dir); + return nbFiles; +} + +#else + +UTIL_STATIC int UTIL_prepareFileList(const char *dirName, char** bufStart, size_t* pos, char** bufEnd) +{ + (void)bufStart; (void)bufEnd; (void)pos; + fprintf(stderr, "Directory %s ignored (compiled without _WIN32 or _POSIX_C_SOURCE)\n", dirName); + return 0; +} + +#endif /* #ifdef _WIN32 */ + +/* + * UTIL_createFileList - takes a list of files and directories (params: inputNames, inputNamesNb), scans directories, + * and returns a new list of files (params: return value, allocatedBuffer, allocatedNamesNb). + * After finishing usage of the list the structures should be freed with UTIL_freeFileList(params: return value, allocatedBuffer) + * In case of error UTIL_createFileList returns NULL and UTIL_freeFileList should not be called. + */ +UTIL_STATIC const char** UTIL_createFileList(const char **inputNames, unsigned inputNamesNb, char** allocatedBuffer, unsigned* allocatedNamesNb) +{ + size_t pos; + unsigned i, nbFiles; + char* buf = (char*)malloc(LIST_SIZE_INCREASE); + char* bufend = buf + LIST_SIZE_INCREASE; + const char** fileTable; + + if (!buf) return NULL; + + for (i=0, pos=0, nbFiles=0; i= bufend) { + ptrdiff_t newListSize = (bufend - buf) + LIST_SIZE_INCREASE; + buf = (char*)UTIL_realloc(buf, newListSize); + bufend = buf + newListSize; + if (!buf) return NULL; + } + if (buf + pos + len < bufend) { + strncpy(buf + pos, inputNames[i], bufend - (buf + pos)); + pos += len + 1; + nbFiles++; + } + } else { + nbFiles += UTIL_prepareFileList(inputNames[i], &buf, &pos, &bufend); + if (buf == NULL) return NULL; + } } + + if (nbFiles == 0) { free(buf); return NULL; } + + fileTable = (const char**)malloc((nbFiles+1) * sizeof(const char*)); + if (!fileTable) { free(buf); return NULL; } + + for (i=0, pos=0; i bufend) { free(buf); free((void*)fileTable); return NULL; } + + *allocatedBuffer = buf; + *allocatedNamesNb = nbFiles; + + return fileTable; +} + + +UTIL_STATIC void UTIL_freeFileList(const char** filenameTable, char* allocatedBuffer) +{ + if (allocatedBuffer) free(allocatedBuffer); + if (filenameTable) free((void*)filenameTable); +} + + +#if defined (__cplusplus) +} +#endif + +#endif /* UTIL_H_MODULE */ diff --git a/contrib/lizard/tests/.gitignore b/contrib/lizard/tests/.gitignore new file mode 100644 index 00000000000..06891acf794 --- /dev/null +++ b/contrib/lizard/tests/.gitignore @@ -0,0 +1,13 @@ + +# test build artefacts +datagen +frametest +frametest32 +fullbench +fullbench32 +fuzzer +fuzzer32 +fasttest + +# test artefacts +tmp* diff --git a/contrib/lizard/tests/COPYING b/contrib/lizard/tests/COPYING new file mode 100644 index 00000000000..d159169d105 --- /dev/null +++ b/contrib/lizard/tests/COPYING @@ -0,0 +1,339 @@ + GNU GENERAL PUBLIC LICENSE + Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +License is intended to guarantee your freedom to share and change free +software--to make sure the software is free for all its users. This +General Public License applies to most of the Free Software +Foundation's software and to any other program whose authors commit to +using it. (Some other Free Software Foundation software is covered by +the GNU Lesser General Public License instead.) You can apply it to +your programs, too. + + When we speak of free software, we are referring to freedom, not +price. Our General Public Licenses are designed to make sure that you +have the freedom to distribute copies of free software (and charge for +this service if you wish), that you receive source code or can get it +if you want it, that you can change the software or use pieces of it +in new free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid +anyone to deny you these rights or to ask you to surrender the rights. +These restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether +gratis or for a fee, you must give the recipients all the rights that +you have. You must make sure that they, too, receive or can get the +source code. And you must show them these terms so they know their +rights. + + We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + + Finally, any free program is threatened constantly by software +patents. We wish to avoid the danger that redistributors of a free +program will individually obtain patent licenses, in effect making the +program proprietary. To prevent this, we have made it clear that any +patent must be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and +modification follow. + + GNU GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains +a notice placed by the copyright holder saying it may be distributed +under the terms of this General Public License. The "Program", below, +refers to any such program or work, and a "work based on the Program" +means either the Program or any derivative work under copyright law: +that is to say, a work containing the Program or a portion of it, +either verbatim or with modifications and/or translated into another +language. (Hereinafter, translation is included without limitation in +the term "modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running the Program is not restricted, and the output from the Program +is covered only if its contents constitute a work based on the +Program (independent of having been made by running the Program). +Whether that is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's +source code as you receive it, in any medium, provided that you +conspicuously and appropriately publish on each copy an appropriate +copyright notice and disclaimer of warranty; keep intact all the +notices that refer to this License and to the absence of any warranty; +and give any other recipients of the Program a copy of this License +along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion +of it, thus forming a work based on the Program, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any + part thereof, to be licensed as a whole at no charge to all third + parties under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a + notice that there is no warranty (or else, saying that you provide + a warranty) and that users may redistribute the program under + these conditions, and telling the user how to view a copy of this + License. (Exception: if the Program itself is interactive but + does not normally print such an announcement, your work based on + the Program is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Program, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections + 1 and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your + cost of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer + to distribute corresponding source code. (This alternative is + allowed only for noncommercial distribution and only if you + received the program in object code or executable form with such + an offer, in accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source +code means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to +control compilation and installation of the executable. However, as a +special exception, the source code distributed need not include +anything that is normally distributed (in either source or binary +form) with the major components (compiler, kernel, and so on) of the +operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering +access to copy from a designated place, then offering equivalent +access to copy the source code from the same place counts as +distribution of the source code, even though third parties are not +compelled to copy the source along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt +otherwise to copy, modify, sublicense or distribute the Program is +void, and will automatically terminate your rights under this License. +However, parties who have received copies, or rights, from you under +this License will not have their licenses terminated so long as such +parties remain in full compliance. + + 5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties to +this License. + + 7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Program at all. For example, if a patent +license would not permit royalty-free redistribution of the Program by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License +may add an explicit geographical distribution limitation excluding +those countries, so that distribution is permitted only in or among +countries not thus excluded. In such case, this License incorporates +the limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new versions +of the General Public License from time to time. Such new versions will +be similar in spirit to the present version, but may differ in detail to +address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and conditions +either of that version or of any later version published by the Free +Software Foundation. If the Program does not specify a version number of +this License, you may choose any version ever published by the Free Software +Foundation. + + 10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the author +to ask for permission. For software which is copyrighted by the Free +Software Foundation, write to the Free Software Foundation; we sometimes +make exceptions for this. Our decision will be guided by the two goals +of preserving the free status of all derivatives of our free software and +of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY +FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN +OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES +PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE +PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, +REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING +WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR +REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, +INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING +OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED +TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY +YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER +PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest +to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least +the "copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. + This is free software, and you are welcome to redistribute it + under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may +be called something other than `show w' and `show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the program + `Gnomovision' (which makes passes at compilers) written by James Hacker. + + , 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Lesser General +Public License instead of this License. diff --git a/contrib/lizard/tests/README.md b/contrib/lizard/tests/README.md new file mode 100644 index 00000000000..4dc2edc4396 --- /dev/null +++ b/contrib/lizard/tests/README.md @@ -0,0 +1,15 @@ +Programs and scripts for automated testing of Lizard +======================================================= + +This directory contains the following programs and scripts: +- `datagen` : Synthetic and parametrable data generator, for tests +- `frametest` : Test tool that checks lizard_frame integrity on target platform +- `fullbench` : Precisely measure speed for each lizard inner functions +- `fuzzer` : Test tool, to check lizard integrity on target platform + + +#### License + +All files in this directory are licensed under GPL-v2. +See [COPYING](COPYING) for details. +The text of the license is also included at the top of each source file. diff --git a/contrib/lizard/tests/datagencli.c b/contrib/lizard/tests/datagencli.c new file mode 100644 index 00000000000..47c79d31522 --- /dev/null +++ b/contrib/lizard/tests/datagencli.c @@ -0,0 +1,192 @@ +/* + datagencli.c + compressible data command line generator + Copyright (C) Yann Collet 2012-2015 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +/************************************** +* Includes +**************************************/ +#include /* fprintf, stderr */ +#include "datagen.h" /* RDG_generate */ + + +/************************************** +* Basic Types +**************************************/ +#if defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */ +# include + typedef uint8_t BYTE; + typedef uint16_t U16; + typedef uint32_t U32; + typedef int32_t S32; + typedef uint64_t U64; +#else + typedef unsigned char BYTE; + typedef unsigned short U16; + typedef unsigned int U32; + typedef signed int S32; + typedef unsigned long long U64; +#endif + + +/************************************** +* Constants +**************************************/ +#ifndef ZSTD_VERSION +# define ZSTD_VERSION "r1" +#endif + +#define KB *(1 <<10) +#define MB *(1 <<20) +#define GB *(1U<<30) + +#define SIZE_DEFAULT (64 KB) +#define SEED_DEFAULT 0 +#define COMPRESSIBILITY_DEFAULT 50 + + +/************************************** +* Macros +**************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define DISPLAYLEVEL(l, ...) if (displayLevel>=l) { DISPLAY(__VA_ARGS__); } +static unsigned displayLevel = 2; + + +/********************************************************* +* Command line +*********************************************************/ +static int usage(char* programName) +{ + DISPLAY( "Compressible data generator\n"); + DISPLAY( "Usage :\n"); + DISPLAY( " %s [size] [args]\n", programName); + DISPLAY( "\n"); + DISPLAY( "Arguments :\n"); + DISPLAY( " -g# : generate # data (default:%i)\n", SIZE_DEFAULT); + DISPLAY( " -s# : Select seed (default:%i)\n", SEED_DEFAULT); + DISPLAY( " -P# : Select compressibility in %% (default:%i%%)\n", COMPRESSIBILITY_DEFAULT); + DISPLAY( " -h : display help and exit\n"); + DISPLAY( "Special values :\n"); + DISPLAY( " -P0 : generate incompressible noise\n"); + DISPLAY( " -P100 : generate sparse files\n"); + return 0; +} + + +int main(int argc, char** argv) +{ + int argNb; + double proba = (double)COMPRESSIBILITY_DEFAULT / 100; + double litProba = 0.0; + U64 size = SIZE_DEFAULT; + U32 seed = SEED_DEFAULT; + char* programName; + + /* Check command line */ + programName = argv[0]; + for(argNb=1; argNb='0') && (*argument<='9')) + { + size *= 10; + size += *argument - '0'; + argument++; + } + if (*argument=='K') { size <<= 10; argument++; } + if (*argument=='M') { size <<= 20; argument++; } + if (*argument=='G') { size <<= 30; argument++; } + if (*argument=='B') { argument++; } + break; + case 's': + argument++; + seed=0; + while ((*argument>='0') && (*argument<='9')) + { + seed *= 10; + seed += *argument - '0'; + argument++; + } + break; + case 'P': + argument++; + proba=0.0; + while ((*argument>='0') && (*argument<='9')) + { + proba *= 10; + proba += *argument - '0'; + argument++; + } + if (proba>100.) proba=100.; + proba /= 100.; + break; + case 'L': /* hidden argument : Literal distribution probability */ + argument++; + litProba=0.; + while ((*argument>='0') && (*argument<='9')) + { + litProba *= 10; + litProba += *argument - '0'; + argument++; + } + if (litProba>100.) litProba=100.; + litProba /= 100.; + break; + case 'v': + displayLevel = 4; + argument++; + break; + default: + return usage(programName); + } + } + + } + } + + DISPLAYLEVEL(4, "Data Generator %s \n", ZSTD_VERSION); + DISPLAYLEVEL(3, "Seed = %u \n", seed); + if (proba!=COMPRESSIBILITY_DEFAULT) DISPLAYLEVEL(3, "Compressibility : %i%%\n", (U32)(proba*100)); + + RDG_genOut(size, proba, litProba, seed); + DISPLAYLEVEL(1, "\n"); + + return 0; +} diff --git a/contrib/lizard/tests/frametest.c b/contrib/lizard/tests/frametest.c new file mode 100644 index 00000000000..8fe64869fa4 --- /dev/null +++ b/contrib/lizard/tests/frametest.c @@ -0,0 +1,866 @@ +/* + frameTest - test tool for lizard_frame + Copyright (C) Yann Collet 2014-2016 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +/*-************************************ +* Compiler specific +**************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4146) /* disable: C4146: minus unsigned expression */ +#endif + + +/*-************************************ +* Includes +**************************************/ +#include /* malloc, free */ +#include /* fprintf */ +#include /* strcmp */ +#include /* clock_t, clock(), CLOCKS_PER_SEC */ +#include "lizard_frame_static.h" +#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ +#define XXH_STATIC_LINKING_ONLY +#include "xxhash/xxhash.h" /* XXH64 */ + + +/*-************************************ +* Basic Types +**************************************/ +#if defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */ +# include +typedef uint8_t BYTE; +typedef uint16_t U16; +typedef uint32_t U32; +typedef int32_t S32; +typedef uint64_t U64; +#else +typedef unsigned char BYTE; +typedef unsigned short U16; +typedef unsigned int U32; +typedef signed int S32; +typedef unsigned long long U64; +#endif + + +/* unoptimized version; solves endianess & alignment issues */ +static void FUZ_writeLE32 (void* dstVoidPtr, U32 value32) +{ + BYTE* dstPtr = (BYTE*)dstVoidPtr; + dstPtr[0] = (BYTE)value32; + dstPtr[1] = (BYTE)(value32 >> 8); + dstPtr[2] = (BYTE)(value32 >> 16); + dstPtr[3] = (BYTE)(value32 >> 24); +} + + +/*-************************************ +* Constants +**************************************/ +#define LIZARDF_MAGIC_SKIPPABLE_START 0x184D2A50U + +#define KB *(1U<<10) +#define MB *(1U<<20) +#define GB *(1U<<30) + +static const U32 nbTestsDefault = 256 KB; +#define COMPRESSIBLE_NOISE_LENGTH (2 MB) +#define FUZ_COMPRESSIBILITY_DEFAULT 50 +static const U32 prime1 = 2654435761U; +static const U32 prime2 = 2246822519U; + + + +/*-************************************ +* Macros +**************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define DISPLAYLEVEL(l, ...) if (displayLevel>=l) { DISPLAY(__VA_ARGS__); } +#define DISPLAYUPDATE(l, ...) if (displayLevel>=l) { \ + if ((FUZ_GetClockSpan(g_clockTime) > refreshRate) || (displayLevel>=4)) \ + { g_clockTime = clock(); DISPLAY(__VA_ARGS__); \ + if (displayLevel>=4) fflush(stdout); } } +static const clock_t refreshRate = CLOCKS_PER_SEC / 6; +static clock_t g_clockTime = 0; + + +/*-*************************************** +* Local Parameters +*****************************************/ +static U32 no_prompt = 0; +static char* programName; +static U32 displayLevel = 2; +static U32 pause = 0; + + +/*-******************************************************* +* Fuzzer functions +*********************************************************/ +static clock_t FUZ_GetClockSpan(clock_t clockStart) +{ + return clock() - clockStart; /* works even if overflow; max span ~ 30 mn */ +} + + +#define FUZ_rotl32(x,r) ((x << r) | (x >> (32 - r))) +unsigned int FUZ_rand(unsigned int* src) +{ + U32 rand32 = *src; + rand32 *= prime1; + rand32 += prime2; + rand32 = FUZ_rotl32(rand32, 13); + *src = rand32; + return rand32 >> 5; +} + + +#define FUZ_RAND15BITS (FUZ_rand(seed) & 0x7FFF) +#define FUZ_RANDLENGTH ( (FUZ_rand(seed) & 3) ? (FUZ_rand(seed) % 15) : (FUZ_rand(seed) % 510) + 15) +static void FUZ_fillCompressibleNoiseBuffer(void* buffer, unsigned bufferSize, double proba, U32* seed) +{ + BYTE* BBuffer = (BYTE*)buffer; + unsigned pos = 0; + U32 P32 = (U32)(32768 * proba); + + /* First Byte */ + BBuffer[pos++] = (BYTE)(FUZ_rand(seed)); + + while (pos < bufferSize) { + /* Select : Literal (noise) or copy (within 64K) */ + if (FUZ_RAND15BITS < P32) { + /* Copy (within 64K) */ + unsigned match, end; + unsigned length = FUZ_RANDLENGTH + 4; + unsigned offset = FUZ_RAND15BITS + 1; + if (offset > pos) offset = pos; + if (pos + length > bufferSize) length = bufferSize - pos; + match = pos - offset; + end = pos + length; + while (pos < end) BBuffer[pos++] = BBuffer[match++]; + } else { + /* Literal (noise) */ + unsigned end; + unsigned length = FUZ_RANDLENGTH; + if (pos + length > bufferSize) length = bufferSize - pos; + end = pos + length; + while (pos < end) BBuffer[pos++] = (BYTE)(FUZ_rand(seed) >> 5); + } + } +} + + +static unsigned FUZ_highbit(U32 v32) +{ + unsigned nbBits = 0; + if (v32==0) return 0; + while (v32) v32 >>= 1, nbBits ++; + return nbBits; +} + + +int basicTests(U32 seed, double compressibility) +{ + int testResult = 0; + void* CNBuffer; + void* compressedBuffer; + void* decodedBuffer; + U32 randState = seed; + size_t cSize, testSize; + LizardF_preferences_t prefs; + LizardF_decompressionContext_t dCtx = NULL; + LizardF_compressionContext_t cctx = NULL; + U64 crcOrig; + + /* Create compressible test buffer */ + memset(&prefs, 0, sizeof(prefs)); + CNBuffer = malloc(COMPRESSIBLE_NOISE_LENGTH); + compressedBuffer = malloc(LizardF_compressFrameBound(COMPRESSIBLE_NOISE_LENGTH, NULL)); + decodedBuffer = malloc(COMPRESSIBLE_NOISE_LENGTH); + FUZ_fillCompressibleNoiseBuffer(CNBuffer, COMPRESSIBLE_NOISE_LENGTH, compressibility, &randState); + crcOrig = XXH64(CNBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); + + /* Special case : null-content frame */ + testSize = 0; + DISPLAYLEVEL(3, "LizardF_compressFrame, compress null content : \n"); + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, NULL), CNBuffer, testSize, NULL); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed null content into a %i bytes frame \n", (int)cSize); + + DISPLAYLEVEL(3, "LizardF_createDecompressionContext \n"); + { LizardF_errorCode_t const errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) goto _output_error; } + + DISPLAYLEVEL(3, "LizardF_getFrameInfo on null-content frame (#157) \n"); + { size_t avail_in = cSize; + LizardF_frameInfo_t frame_info; + LizardF_errorCode_t const errorCode = LizardF_getFrameInfo(dCtx, &frame_info, compressedBuffer, &avail_in); + if (LizardF_isError(errorCode)) goto _output_error; + } + + DISPLAYLEVEL(3, "LizardF_freeDecompressionContext \n"); + { LizardF_errorCode_t const errorCode = LizardF_freeDecompressionContext(dCtx); + if (LizardF_isError(errorCode)) goto _output_error; } + + /* Trivial tests : one-step frame */ + testSize = COMPRESSIBLE_NOISE_LENGTH; + DISPLAYLEVEL(3, "LizardF_compressFrame, using default preferences : \n"); + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, NULL), CNBuffer, testSize, NULL); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "Decompression test : \n"); + { size_t decodedBufferSize = COMPRESSIBLE_NOISE_LENGTH; + size_t compressedBufferSize = cSize; + BYTE* op = (BYTE*)decodedBuffer; + BYTE* const oend = (BYTE*)decodedBuffer + COMPRESSIBLE_NOISE_LENGTH; + BYTE* ip = (BYTE*)compressedBuffer; + BYTE* const iend = (BYTE*)compressedBuffer + cSize; + U64 crcDest; + + LizardF_errorCode_t errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) goto _output_error; + + DISPLAYLEVEL(3, "Single Block : \n"); + errorCode = LizardF_decompress(dCtx, decodedBuffer, &decodedBufferSize, compressedBuffer, &compressedBufferSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); + if (crcDest != crcOrig) goto _output_error; + DISPLAYLEVEL(3, "Regenerated %i bytes \n", (int)decodedBufferSize); + + DISPLAYLEVEL(3, "Reusing decompression context \n"); + { size_t iSize = compressedBufferSize - 4; + const BYTE* cBuff = (const BYTE*) compressedBuffer; + size_t decResult; + DISPLAYLEVEL(3, "Missing last 4 bytes : "); + decResult = LizardF_decompress(dCtx, decodedBuffer, &decodedBufferSize, cBuff, &iSize, NULL); + if (LizardF_isError(decResult)) goto _output_error; + if (!decResult) goto _output_error; /* not finished */ + DISPLAYLEVEL(3, "indeed, request %u bytes \n", (unsigned)decResult); + cBuff += iSize; + iSize = decResult; + decResult = LizardF_decompress(dCtx, decodedBuffer, &decodedBufferSize, cBuff, &iSize, NULL); + if (decResult != 0) goto _output_error; /* should finish now */ + crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); + if (crcDest != crcOrig) goto _output_error; + } + + { size_t oSize = 0; + size_t iSize = 0; + LizardF_frameInfo_t fi; + + DISPLAYLEVEL(3, "Start by feeding 0 bytes, to get next input size : "); + errorCode = LizardF_decompress(dCtx, NULL, &oSize, ip, &iSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + DISPLAYLEVEL(3, " %u \n", (unsigned)errorCode); + + DISPLAYLEVEL(3, "get FrameInfo on null input : "); + errorCode = LizardF_getFrameInfo(dCtx, &fi, ip, &iSize); + if (errorCode != (size_t)-LizardF_ERROR_frameHeader_incomplete) goto _output_error; + DISPLAYLEVEL(3, " correctly failed : %s \n", LizardF_getErrorName(errorCode)); + + DISPLAYLEVEL(3, "get FrameInfo on not enough input : "); + iSize = 6; + errorCode = LizardF_getFrameInfo(dCtx, &fi, ip, &iSize); + if (errorCode != (size_t)-LizardF_ERROR_frameHeader_incomplete) goto _output_error; + DISPLAYLEVEL(3, " correctly failed : %s \n", LizardF_getErrorName(errorCode)); + ip += iSize; + + DISPLAYLEVEL(3, "get FrameInfo on enough input : "); + iSize = 15 - iSize; + errorCode = LizardF_getFrameInfo(dCtx, &fi, ip, &iSize); + if (LizardF_isError(errorCode)) goto _output_error; + DISPLAYLEVEL(3, " correctly decoded \n"); + ip += iSize; + } + + DISPLAYLEVEL(3, "Byte after byte : \n"); + while (ip < iend) { + size_t oSize = oend-op; + size_t iSize = 1; + errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += oSize; + ip += iSize; + } + crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); + if (crcDest != crcOrig) goto _output_error; + DISPLAYLEVEL(3, "Regenerated %u/%u bytes \n", (unsigned)(op-(BYTE*)decodedBuffer), COMPRESSIBLE_NOISE_LENGTH); + + errorCode = LizardF_freeDecompressionContext(dCtx); + if (LizardF_isError(errorCode)) goto _output_error; + } + + DISPLAYLEVEL(3, "Using 128 KB block : \n"); + prefs.frameInfo.blockSizeID = LizardF_max128KB; + prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "without checksum : \n"); + prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "Using 256 KB block : \n"); + prefs.frameInfo.blockSizeID = LizardF_max256KB; + prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "Decompression test : \n"); + { size_t decodedBufferSize = COMPRESSIBLE_NOISE_LENGTH; + unsigned maxBits = FUZ_highbit((U32)decodedBufferSize); + BYTE* op = (BYTE*)decodedBuffer; + BYTE* const oend = (BYTE*)decodedBuffer + COMPRESSIBLE_NOISE_LENGTH; + BYTE* ip = (BYTE*)compressedBuffer; + BYTE* const iend = (BYTE*)compressedBuffer + cSize; + U64 crcDest; + + LizardF_errorCode_t errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) goto _output_error; + + DISPLAYLEVEL(3, "random segment sizes : \n"); + while (ip < iend) { + unsigned nbBits = FUZ_rand(&randState) % maxBits; + size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; + //DISPLAY("%7i : + %6i\n", (int)(ip-(BYTE*)compressedBuffer), (int)iSize); + errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += oSize; + ip += iSize; + } + crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); + if (crcDest != crcOrig) goto _output_error; + DISPLAYLEVEL(3, "Regenerated %i bytes \n", (int)decodedBufferSize); + + errorCode = LizardF_freeDecompressionContext(dCtx); + if (LizardF_isError(errorCode)) goto _output_error; + } + + DISPLAYLEVEL(3, "without checksum : \n"); + prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "Using 1 MB block : \n"); + prefs.frameInfo.blockSizeID = LizardF_max1MB; + prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "without checksum : \n"); + prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "Using 4 MB block : \n"); + prefs.frameInfo.blockSizeID = LizardF_max4MB; + prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + DISPLAYLEVEL(3, "without checksum : \n"); + prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; + cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); + if (LizardF_isError(cSize)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); + + { size_t errorCode; + BYTE* const ostart = (BYTE*)compressedBuffer; + BYTE* op = ostart; + errorCode = LizardF_createCompressionContext(&cctx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) goto _output_error; + + DISPLAYLEVEL(3, "compress without frameSize : \n"); + memset(&(prefs.frameInfo), 0, sizeof(prefs.frameInfo)); + errorCode = LizardF_compressBegin(cctx, compressedBuffer, testSize, &prefs); + if (LizardF_isError(errorCode)) goto _output_error; + op += errorCode; + errorCode = LizardF_compressUpdate(cctx, op, LizardF_compressBound(testSize, &prefs), CNBuffer, testSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += errorCode; + errorCode = LizardF_compressEnd(cctx, compressedBuffer, testSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)(op-ostart)); + + DISPLAYLEVEL(3, "compress with frameSize : \n"); + prefs.frameInfo.contentSize = testSize; + op = ostart; + errorCode = LizardF_compressBegin(cctx, compressedBuffer, testSize, &prefs); + if (LizardF_isError(errorCode)) goto _output_error; + op += errorCode; + errorCode = LizardF_compressUpdate(cctx, op, LizardF_compressBound(testSize, &prefs), CNBuffer, testSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += errorCode; + errorCode = LizardF_compressEnd(cctx, compressedBuffer, testSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)(op-ostart)); + + DISPLAYLEVEL(3, "compress with wrong frameSize : \n"); + prefs.frameInfo.contentSize = testSize+1; + op = ostart; + errorCode = LizardF_compressBegin(cctx, compressedBuffer, testSize, &prefs); + if (LizardF_isError(errorCode)) goto _output_error; + op += errorCode; + errorCode = LizardF_compressUpdate(cctx, op, LizardF_compressBound(testSize, &prefs), CNBuffer, testSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += errorCode; + errorCode = LizardF_compressEnd(cctx, op, testSize, NULL); + if (LizardF_isError(errorCode)) { DISPLAYLEVEL(3, "Error correctly detected : %s \n", LizardF_getErrorName(errorCode)); } + else + goto _output_error; + + errorCode = LizardF_freeCompressionContext(cctx); + if (LizardF_isError(errorCode)) goto _output_error; + cctx = NULL; + } + + DISPLAYLEVEL(3, "Skippable frame test : \n"); + { size_t decodedBufferSize = COMPRESSIBLE_NOISE_LENGTH; + unsigned maxBits = FUZ_highbit((U32)decodedBufferSize); + BYTE* op = (BYTE*)decodedBuffer; + BYTE* const oend = (BYTE*)decodedBuffer + COMPRESSIBLE_NOISE_LENGTH; + BYTE* ip = (BYTE*)compressedBuffer; + BYTE* iend = (BYTE*)compressedBuffer + cSize + 8; + + LizardF_errorCode_t errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) goto _output_error; + + /* generate skippable frame */ + FUZ_writeLE32(ip, LIZARDF_MAGIC_SKIPPABLE_START); + FUZ_writeLE32(ip+4, (U32)cSize); + + DISPLAYLEVEL(3, "random segment sizes : \n"); + while (ip < iend) { + unsigned nbBits = FUZ_rand(&randState) % maxBits; + size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; + errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += oSize; + ip += iSize; + } + DISPLAYLEVEL(3, "Skipped %i bytes \n", (int)decodedBufferSize); + + /* generate zero-size skippable frame */ + DISPLAYLEVEL(3, "zero-size skippable frame\n"); + ip = (BYTE*)compressedBuffer; + op = (BYTE*)decodedBuffer; + FUZ_writeLE32(ip, LIZARDF_MAGIC_SKIPPABLE_START+1); + FUZ_writeLE32(ip+4, 0); + iend = ip+8; + + while (ip < iend) { + unsigned nbBits = FUZ_rand(&randState) % maxBits; + size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; + errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += oSize; + ip += iSize; + } + DISPLAYLEVEL(3, "Skipped %i bytes \n", (int)(ip - (BYTE*)compressedBuffer - 8)); + + DISPLAYLEVEL(3, "Skippable frame header complete in first call \n"); + ip = (BYTE*)compressedBuffer; + op = (BYTE*)decodedBuffer; + FUZ_writeLE32(ip, LIZARDF_MAGIC_SKIPPABLE_START+2); + FUZ_writeLE32(ip+4, 10); + iend = ip+18; + while (ip < iend) { + size_t iSize = 10; + size_t oSize = 10; + if (iSize > (size_t)(iend-ip)) iSize = iend-ip; + errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); + if (LizardF_isError(errorCode)) goto _output_error; + op += oSize; + ip += iSize; + } + DISPLAYLEVEL(3, "Skipped %i bytes \n", (int)(ip - (BYTE*)compressedBuffer - 8)); + } + + DISPLAY("Basic tests completed \n"); +_end: + free(CNBuffer); + free(compressedBuffer); + free(decodedBuffer); + LizardF_freeDecompressionContext(dCtx); dCtx = NULL; + LizardF_freeCompressionContext(cctx); cctx = NULL; + return testResult; + +_output_error: + testResult = 1; + DISPLAY("Error detected ! \n"); + goto _end; +} + + +static void locateBuffDiff(const void* buff1, const void* buff2, size_t size, unsigned nonContiguous) +{ + int p=0; + const BYTE* b1=(const BYTE*)buff1; + const BYTE* b2=(const BYTE*)buff2; + if (nonContiguous) { + DISPLAY("Non-contiguous output test (%i bytes)\n", (int)size); + return; + } + while (b1[p]==b2[p]) p++; + DISPLAY("Error at pos %i/%i : %02X != %02X \n", p, (int)size, b1[p], b2[p]); +} + + +static const U32 srcDataLength = 9 MB; /* needs to be > 2x4MB to test large blocks */ + +int fuzzerTests(U32 seed, unsigned nbTests, unsigned startTest, double compressibility, U32 duration_s) +{ + unsigned testResult = 0; + unsigned testNb = 0; + void* srcBuffer = NULL; + void* compressedBuffer = NULL; + void* decodedBuffer = NULL; + U32 coreRand = seed; + LizardF_decompressionContext_t dCtx = NULL; + LizardF_compressionContext_t cCtx = NULL; + size_t result; + clock_t const startClock = clock(); + clock_t const clockDuration = duration_s * CLOCKS_PER_SEC; + XXH64_state_t xxh64; +# define CHECK(cond, ...) if (cond) { DISPLAY("Error => "); DISPLAY(__VA_ARGS__); \ + DISPLAY(" (seed %u, test nb %u) \n", seed, testNb); goto _output_error; } + + /* Create buffers */ + result = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); + CHECK(LizardF_isError(result), "Allocation failed (error %i)", (int)result); + result = LizardF_createCompressionContext(&cCtx, LIZARDF_VERSION); + CHECK(LizardF_isError(result), "Allocation failed (error %i)", (int)result); + srcBuffer = malloc(srcDataLength); + CHECK(srcBuffer==NULL, "srcBuffer Allocation failed"); + compressedBuffer = malloc(LizardF_compressFrameBound(srcDataLength, NULL)); + CHECK(compressedBuffer==NULL, "compressedBuffer Allocation failed"); + decodedBuffer = calloc(1, srcDataLength); /* calloc avoids decodedBuffer being considered "garbage" by scan-build */ + CHECK(decodedBuffer==NULL, "decodedBuffer Allocation failed"); + FUZ_fillCompressibleNoiseBuffer(srcBuffer, srcDataLength, compressibility, &coreRand); + + /* jump to requested testNb */ + for (testNb =0; (testNb < startTest); testNb++) (void)FUZ_rand(&coreRand); // sync randomizer + + /* main fuzzer test loop */ + for ( ; (testNb < nbTests) || (clockDuration > FUZ_GetClockSpan(startClock)) ; testNb++) { + U32 randState = coreRand ^ prime1; + unsigned BSId = 4 + (FUZ_rand(&randState) & 3); + unsigned BMId = FUZ_rand(&randState) & 1; + unsigned CCflag = FUZ_rand(&randState) & 1; + unsigned autoflush = (FUZ_rand(&randState) & 7) == 2; + LizardF_preferences_t prefs; + LizardF_compressOptions_t cOptions; + LizardF_decompressOptions_t dOptions; + unsigned nbBits = (FUZ_rand(&randState) % (FUZ_highbit(srcDataLength-1) - 1)) + 1; + size_t srcSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; + cOptions.stableSrc = ((FUZ_rand(&randState) & 3) == 1); + + result = LizardF_compressUpdate(cCtx, op, oSize, ip, iSize, &cOptions); + CHECK(LizardF_isError(result), "Compression failed (error %i) iSize=%d oSize=%d", (int)result, (int)iSize, (int)oSize); + op += result; + ip += iSize; + + if (forceFlush) { + result = LizardF_flush(cCtx, op, oend-op, &cOptions); + CHECK(LizardF_isError(result), "Compression flush failed (error %i)", (int)result); + op += result; + } + } + result = LizardF_compressEnd(cCtx, op, oend-op, &cOptions); + CHECK(LizardF_isError(result), "Compression completion failed (error %i)", (int)result); + op += result; + cSize = op-(BYTE*)compressedBuffer; + } + + { const BYTE* ip = (const BYTE*)compressedBuffer; + const BYTE* const iend = ip + cSize; + BYTE* op = (BYTE*)decodedBuffer; + BYTE* const oend = op + srcDataLength; + size_t totalOut = 0; + unsigned maxBits = FUZ_highbit((U32)cSize); + unsigned nonContiguousDst = (FUZ_rand(&randState) & 3) == 1; + nonContiguousDst += FUZ_rand(&randState) & nonContiguousDst; /* 0=>0; 1=>1,2 */ + XXH64_reset(&xxh64, 1); + if (maxBits < 3) maxBits = 3; + while (ip < iend) { + unsigned nbBitsI = (FUZ_rand(&randState) % (maxBits-1)) + 1; + unsigned nbBitsO = (FUZ_rand(&randState) % (maxBits)) + 1; + size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; + if (oSize > (size_t)(oend-op)) oSize = oend-op; + dOptions.stableDst = FUZ_rand(&randState) & 1; + if (nonContiguousDst==2) dOptions.stableDst = 0; + result = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, &dOptions); + if (result == (size_t)-LizardF_ERROR_contentChecksum_invalid) + locateBuffDiff((BYTE*)srcBuffer+srcStart, decodedBuffer, srcSize, nonContiguousDst); + CHECK(LizardF_isError(result), "Decompression failed (error %i:%s)", (int)result, LizardF_getErrorName((LizardF_errorCode_t)result)); + XXH64_update(&xxh64, op, (U32)oSize); + totalOut += oSize; + op += oSize; + ip += iSize; + op += nonContiguousDst; + if (nonContiguousDst==2) op = (BYTE*)decodedBuffer; /* overwritten destination */ + } + CHECK(result != 0, "Frame decompression failed (error %i)", (int)result); + if (totalOut) { /* otherwise, it's a skippable frame */ + crcDecoded = XXH64_digest(&xxh64); + if (crcDecoded != crcOrig) locateBuffDiff((BYTE*)srcBuffer+srcStart, decodedBuffer, srcSize, nonContiguousDst); + CHECK(crcDecoded != crcOrig, "Decompression corruption"); + } + } + } + + DISPLAYLEVEL(2, "\rAll tests completed \n"); + +_end: + LizardF_freeDecompressionContext(dCtx); + LizardF_freeCompressionContext(cCtx); + free(srcBuffer); + free(compressedBuffer); + free(decodedBuffer); + + if (pause) { + DISPLAY("press enter to finish \n"); + (void)getchar(); + } + return testResult; + +_output_error: + testResult = 1; + goto _end; +} + + +int FUZ_usage(void) +{ + DISPLAY( "Usage :\n"); + DISPLAY( " %s [args]\n", programName); + DISPLAY( "\n"); + DISPLAY( "Arguments :\n"); + DISPLAY( " -i# : Nb of tests (default:%u) \n", nbTestsDefault); + DISPLAY( " -T# : Duration of tests, in seconds (default: use Nb of tests) \n"); + DISPLAY( " -s# : Select seed (default:prompt user)\n"); + DISPLAY( " -t# : Select starting test number (default:0)\n"); + DISPLAY( " -p# : Select compressibility in %% (default:%i%%)\n", FUZ_COMPRESSIBILITY_DEFAULT); + DISPLAY( " -v : verbose\n"); + DISPLAY( " -h : display help and exit\n"); + return 0; +} + + +int main(int argc, char** argv) +{ + U32 seed=0; + int seedset=0; + int argNb; + int nbTests = nbTestsDefault; + int testNb = 0; + int proba = FUZ_COMPRESSIBILITY_DEFAULT; + int result=0; + U32 duration=0; + + /* Check command line */ + programName = argv[0]; + for(argNb=1; argNb='0') && (*argument<='9')) { + nbTests *= 10; + nbTests += *argument - '0'; + argument++; + } + break; + + case 'T': + argument++; + nbTests = 0; duration = 0; + for (;;) { + switch(*argument) + { + case 'm': duration *= 60; argument++; continue; + case 's': + case 'n': argument++; continue; + case '0': + case '1': + case '2': + case '3': + case '4': + case '5': + case '6': + case '7': + case '8': + case '9': duration *= 10; duration += *argument++ - '0'; continue; + } + break; + } + break; + + case 's': + argument++; + seed=0; + seedset=1; + while ((*argument>='0') && (*argument<='9')) { + seed *= 10; + seed += *argument - '0'; + argument++; + } + break; + case 't': + argument++; + testNb=0; + while ((*argument>='0') && (*argument<='9')) { + testNb *= 10; + testNb += *argument - '0'; + argument++; + } + break; + case 'P': /* compressibility % */ + argument++; + proba=0; + while ((*argument>='0') && (*argument<='9')) { + proba *= 10; + proba += *argument - '0'; + argument++; + } + if (proba<0) proba=0; + if (proba>100) proba=100; + break; + default: + ; + return FUZ_usage(); + } + } + } + } + + /* Get Seed */ + printf("Starting lizard_frame tester (%i-bits, %s)\n", (int)(sizeof(size_t)*8), LIZARD_VERSION_STRING); + + if (!seedset) { + time_t const t = time(NULL); + U32 const h = XXH32(&t, sizeof(t), 1); + seed = h % 10000; + } + printf("Seed = %u\n", seed); + if (proba!=FUZ_COMPRESSIBILITY_DEFAULT) printf("Compressibility : %i%%\n", proba); + + if (nbTests<=0) nbTests=1; + + if (testNb==0) result = basicTests(seed, ((double)proba) / 100); + if (result) return 1; + return fuzzerTests(seed, nbTests, testNb, ((double)proba) / 100, duration); +} diff --git a/contrib/lizard/tests/fullbench.c b/contrib/lizard/tests/fullbench.c new file mode 100644 index 00000000000..d9070d13345 --- /dev/null +++ b/contrib/lizard/tests/fullbench.c @@ -0,0 +1,732 @@ +/* + bench.c - Demo program to benchmark open-source compression algorithm + Copyright (C) Yann Collet 2012-2015 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repository : https://github.com/inikep/lizard +*/ + +/************************************** +* Compiler Options +**************************************/ +/* Disable some Visual warning messages */ +#define _CRT_SECURE_NO_WARNINGS +#define _CRT_SECURE_NO_DEPRECATE /* VS2005 */ + +/* Unix Large Files support (>4GB) */ +#if (defined(__sun__) && (!defined(__LP64__))) // Sun Solaris 32-bits requires specific definitions +# define _LARGEFILE_SOURCE +# define _FILE_OFFSET_BITS 64 +#elif ! defined(__LP64__) // No point defining Large file for 64 bit +# define _LARGEFILE64_SOURCE +#endif + +// S_ISREG & gettimeofday() are not supported by MSVC +#if defined(_MSC_VER) || defined(_WIN32) +# define BMK_LEGACY_TIMER 1 +#endif + + +/************************************** +* Includes +**************************************/ +#include /* malloc, free */ +#include /* fprintf, fopen, ftello64 */ +#include /* stat64 */ +#include /* stat64 */ +#include /* strcmp */ +#include /* clock_t, clock(), CLOCKS_PER_SEC */ + +#include "lizard_compress.h" +#include "lizard_decompress.h" +#include "lizard_common.h" /* Lizard_compress_MinLevel, Lizard_createStream_MinLevel */ +#include "lizard_frame.h" + +#include "xxhash/xxhash.h" + + +/************************************** +* Compiler Options +**************************************/ +/* S_ISREG & gettimeofday() are not supported by MSVC */ +#if !defined(S_ISREG) +# define S_ISREG(x) (((x) & S_IFMT) == S_IFREG) +#endif + + + +/************************************** +* Constants +**************************************/ +#define PROGRAM_DESCRIPTION "Lizard speed analyzer" +#define AUTHOR "Yann Collet" +#define WELCOME_MESSAGE "*** %s v%s %i-bits, by %s ***\n", PROGRAM_DESCRIPTION, LIZARD_VERSION_STRING, (int)(sizeof(void*)*8), AUTHOR + +#define NBLOOPS 6 +#define TIMELOOP (CLOCKS_PER_SEC * 25 / 10) + +#define KNUTH 2654435761U +#define MAX_MEM (1920 MB) +#define DEFAULT_CHUNKSIZE (4 MB) + +#define ALL_COMPRESSORS 0 +#define ALL_DECOMPRESSORS 0 + + +/************************************** +* Local structures +**************************************/ +struct chunkParameters +{ + U32 id; + char* origBuffer; + char* compressedBuffer; + int origSize; + int compressedSize; +}; + + +/************************************** +* Macros +**************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define PROGRESS(...) g_noPrompt ? 0 : DISPLAY(__VA_ARGS__) + + +/************************************** +* Benchmark Parameters +**************************************/ +static int g_chunkSize = DEFAULT_CHUNKSIZE; +static int g_nbIterations = NBLOOPS; +static int g_pause = 0; +static int g_compressionTest = 1; +static int g_compressionAlgo = ALL_COMPRESSORS; +static int g_decompressionTest = 1; +static int g_decompressionAlgo = ALL_DECOMPRESSORS; +static int g_noPrompt = 0; + +static void BMK_setBlocksize(int bsize) +{ + g_chunkSize = bsize; + DISPLAY("-Using Block Size of %i KB-\n", g_chunkSize>>10); +} + +static void BMK_setNbIterations(int nbLoops) +{ + g_nbIterations = nbLoops; + DISPLAY("- %i iterations -\n", g_nbIterations); +} + +static void BMK_setPause(void) +{ + g_pause = 1; +} + + +/********************************************************* +* Private functions +*********************************************************/ +static clock_t BMK_GetClockSpan( clock_t clockStart ) +{ + return clock() - clockStart; /* works even if overflow; max span ~30 mn */ +} + + +static size_t BMK_findMaxMem(U64 requiredMem) +{ + size_t step = 64 MB; + BYTE* testmem=NULL; + + requiredMem = (((requiredMem >> 26) + 1) << 26); + requiredMem += 2*step; + if (requiredMem > MAX_MEM) requiredMem = MAX_MEM; + + while (!testmem) { + if (requiredMem > step) requiredMem -= step; + else requiredMem >>= 1; + testmem = (BYTE*) malloc ((size_t)requiredMem); + } + free (testmem); + + /* keep some space available */ + if (requiredMem > step) requiredMem -= step; + else requiredMem >>= 1; + + return (size_t)requiredMem; +} + + +static U64 BMK_GetFileSize(const char* infilename) +{ + int r; +#if defined(_MSC_VER) + struct _stat64 statbuf; + r = _stat64(infilename, &statbuf); +#else + struct stat statbuf; + r = stat(infilename, &statbuf); +#endif + if (r || !S_ISREG(statbuf.st_mode)) return 0; /* No good... */ + return (U64)statbuf.st_size; +} + + +/********************************************************* +* Benchmark function +*********************************************************/ +Lizard_stream_t *Lizard_stream; +static void local_Lizard_createStream(void) +{ + Lizard_stream = Lizard_resetStream_MinLevel(Lizard_stream); +} + +static int local_Lizard_saveDict(const char* in, char* out, int inSize) +{ + (void)in; + return Lizard_saveDict(Lizard_stream, out, inSize); +} + +static int local_Lizard_compress_default_large(const char* in, char* out, int inSize) +{ + return Lizard_compress_MinLevel(in, out, inSize, Lizard_compressBound(inSize)); +} + +static int local_Lizard_compress_default_small(const char* in, char* out, int inSize) +{ + return Lizard_compress_MinLevel(in, out, inSize, Lizard_compressBound(inSize)-1); +} + +static int local_Lizard_compress_withState(const char* in, char* out, int inSize) +{ + return Lizard_compress_extState_MinLevel(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)); +} + +static int local_Lizard_compress_limitedOutput_withState(const char* in, char* out, int inSize) +{ + return Lizard_compress_extState_MinLevel(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)-1); +} + +static int local_Lizard_compress_continue(const char* in, char* out, int inSize) +{ + return Lizard_compress_continue(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)); +} + +static int local_Lizard_compress_limitedOutput_continue(const char* in, char* out, int inSize) +{ + return Lizard_compress_continue(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)-1); +} + + +/* HC compression functions */ +Lizard_stream_t* Lizard_streamPtr; +static void local_Lizard_resetStream(void) +{ + Lizard_streamPtr = Lizard_resetStream(Lizard_streamPtr, 0); +} + +static int local_Lizard_saveDictHC(const char* in, char* out, int inSize) +{ + (void)in; + return Lizard_saveDict(Lizard_streamPtr, out, inSize); +} + +static int local_Lizard_compress_extState(const char* in, char* out, int inSize) +{ + return Lizard_compress_extState(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize), 0); +} + +static int local_Lizard_compress_extState_limitedOutput(const char* in, char* out, int inSize) +{ + return Lizard_compress_extState(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize)-1, 0); +} + +static int local_Lizard_compress(const char* in, char* out, int inSize) +{ + return Lizard_compress(in, out, inSize, Lizard_compressBound(inSize), 0); +} + +static int local_Lizard_compress_limitedOutput(const char* in, char* out, int inSize) +{ + return Lizard_compress(in, out, inSize, Lizard_compressBound(inSize)-1, 0); +} + +static int local_Lizard_compressHC_continue(const char* in, char* out, int inSize) +{ + return Lizard_compress_continue(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize)); +} + +static int local_Lizard_compress_continue_limitedOutput(const char* in, char* out, int inSize) +{ + return Lizard_compress_continue(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize)-1); +} + + +/* decompression functions */ +static int local_Lizard_decompress_safe_usingDict(const char* in, char* out, int inSize, int outSize) +{ + (void)inSize; + Lizard_decompress_safe_usingDict(in, out, inSize, outSize, out - 65536, 65536); + return outSize; +} + +extern int Lizard_decompress_safe_forceExtDict(const char* in, char* out, int inSize, int outSize, const char* dict, int dictSize); + +static int local_Lizard_decompress_safe_forceExtDict(const char* in, char* out, int inSize, int outSize) +{ + (void)inSize; + Lizard_decompress_safe_forceExtDict(in, out, inSize, outSize, out - 65536, 65536); + return outSize; +} + +static int local_Lizard_decompress_safe_partial(const char* in, char* out, int inSize, int outSize) +{ + return Lizard_decompress_safe_partial(in, out, inSize, outSize - 5, outSize); +} + + +/* frame functions */ +static int local_LizardF_compressFrame(const char* in, char* out, int inSize) +{ + return (int)LizardF_compressFrame(out, 2*inSize + 16, in, inSize, NULL); +} + +static LizardF_decompressionContext_t g_dCtx; + +static int local_LizardF_decompress(const char* in, char* out, int inSize, int outSize) +{ + size_t srcSize = inSize; + size_t dstSize = outSize; + size_t result; + result = LizardF_decompress(g_dCtx, out, &dstSize, in, &srcSize, NULL); + if (result!=0) { DISPLAY("Error decompressing frame : unfinished frame (%d)\n", (int)result); exit(8); } + if (srcSize != (size_t)inSize) { DISPLAY("Error decompressing frame : read size incorrect\n"); exit(9); } + return (int)dstSize; +} + + +#define NB_COMPRESSION_ALGORITHMS 100 +#define NB_DECOMPRESSION_ALGORITHMS 100 +int fullSpeedBench(const char** fileNamesTable, int nbFiles) +{ + int fileIdx=0; + + /* Init */ + { size_t const errorCode = LizardF_createDecompressionContext(&g_dCtx, LIZARDF_VERSION); + if (LizardF_isError(errorCode)) { DISPLAY("dctx allocation issue \n"); return 10; } } + + Lizard_streamPtr = Lizard_createStream(0); + if (!Lizard_streamPtr) { DISPLAY("Lizard_streamPtr allocation issue \n"); return 10; } + + Lizard_stream = Lizard_createStream_MinLevel(); + if (!Lizard_stream) { DISPLAY("Lizard_stream allocation issue \n"); return 10; } + + /* Loop for each fileName */ + while (fileIdx inFileSize) benchedSize = (size_t)inFileSize; + if (benchedSize < inFileSize) + DISPLAY("Not enough memory for '%s' full size; testing %i MB only...\n", inFileName, (int)(benchedSize>>20)); + + /* Allocation */ + chunkP = (struct chunkParameters*) malloc(((benchedSize / (size_t)g_chunkSize)+1) * sizeof(struct chunkParameters)); + orig_buff = (char*) malloc(benchedSize); + nbChunks = (int) ((benchedSize + (g_chunkSize-1)) / g_chunkSize); + maxCompressedChunkSize = Lizard_compressBound(g_chunkSize); + compressedBuffSize = nbChunks * maxCompressedChunkSize; + compressed_buff = (char*)malloc((size_t)compressedBuffSize); + if(!chunkP || !orig_buff || !compressed_buff) { + DISPLAY("\nError: not enough memory!\n"); + fclose(inFile); + free(orig_buff); + free(compressed_buff); + free(chunkP); + return(12); + } + + /* Fill in src buffer */ + DISPLAY("Loading %s... \r", inFileName); + readSize = fread(orig_buff, 1, benchedSize, inFile); + fclose(inFile); + + if (readSize != benchedSize) { + DISPLAY("\nError: problem reading file '%s' !! \n", inFileName); + free(orig_buff); + free(compressed_buff); + free(chunkP); + return 13; + } + + /* Calculating input Checksum */ + crcOriginal = XXH32(orig_buff, benchedSize,0); + + + /* Bench */ + { int loopNb, nb_loops, chunkNb, cAlgNb, dAlgNb; + size_t cSize=0; + double ratio=0.; + + DISPLAY("\r%79s\r", ""); + DISPLAY(" %s : \n", inFileName); + + /* Bench Compression Algorithms */ + for (cAlgNb=0; (cAlgNb <= NB_COMPRESSION_ALGORITHMS) && (g_compressionTest); cAlgNb++) { + const char* compressorName; + int (*compressionFunction)(const char*, char*, int); + void (*initFunction)(void) = NULL; + double bestTime = 100000000.; + + /* filter compressionAlgo only */ + if ((g_compressionAlgo != ALL_COMPRESSORS) && (g_compressionAlgo != cAlgNb)) continue; + + /* Init data chunks */ + { int i; + size_t remaining = benchedSize; + char* in = orig_buff; + char* out = compressed_buff; + nbChunks = (int) (((int)benchedSize + (g_chunkSize-1))/ g_chunkSize); + for (i=0; i g_chunkSize) { chunkP[i].origSize = g_chunkSize; remaining -= g_chunkSize; } else { chunkP[i].origSize = (int)remaining; remaining = 0; } + chunkP[i].compressedBuffer = out; out += maxCompressedChunkSize; + chunkP[i].compressedSize = 0; + } + } + + switch(cAlgNb) + { + case 0 : DISPLAY("Compression functions : \n"); continue; + case 1 : compressionFunction = local_Lizard_compress_default_large; compressorName = "Lizard_compress_MinLevel"; break; + case 2 : compressionFunction = local_Lizard_compress_default_small; compressorName = "Lizard_compress_MinLevel(small dst)"; break; + + case 10: compressionFunction = local_Lizard_compress; compressorName = "Lizard_compress"; break; + case 11: compressionFunction = local_Lizard_compress_limitedOutput; compressorName = "Lizard_compress limitedOutput"; break; + case 12 : compressionFunction = local_Lizard_compress_extState; compressorName = "Lizard_compress_extState"; break; + case 13: compressionFunction = local_Lizard_compress_extState_limitedOutput; compressorName = "Lizard_compress_extState limitedOutput"; break; + case 14: compressionFunction = local_Lizard_compressHC_continue; initFunction = local_Lizard_resetStream; compressorName = "Lizard_compress_continue"; break; + case 15: compressionFunction = local_Lizard_compress_continue_limitedOutput; initFunction = local_Lizard_resetStream; compressorName = "Lizard_compress_continue limitedOutput"; break; + case 30: compressionFunction = local_LizardF_compressFrame; compressorName = "LizardF_compressFrame"; + chunkP[0].origSize = (int)benchedSize; nbChunks=1; + break; + case 40: compressionFunction = local_Lizard_saveDict; compressorName = "Lizard_saveDict"; + Lizard_loadDict(Lizard_stream, chunkP[0].origBuffer, chunkP[0].origSize); + break; + case 41: compressionFunction = local_Lizard_saveDictHC; compressorName = "Lizard_saveDict"; + Lizard_loadDict(Lizard_streamPtr, chunkP[0].origBuffer, chunkP[0].origSize); + break; + case 16: compressionFunction = local_Lizard_compress_withState; compressorName = "Lizard_compress_extState_MinLevel(1)"; break; + case 17: compressionFunction = local_Lizard_compress_limitedOutput_withState; compressorName = "Lizard_compress_extState_MinLevel(1) limitedOutput"; break; + case 18: compressionFunction = local_Lizard_compress_continue; initFunction = local_Lizard_createStream; compressorName = "Lizard_compress_continue(1)"; break; + case 19: compressionFunction = local_Lizard_compress_limitedOutput_continue; initFunction = local_Lizard_createStream; compressorName = "Lizard_compress_continue(1) limitedOutput"; break; + case 60: DISPLAY("Obsolete compression functions : \n"); continue; + default : + continue; /* unknown ID : just skip */ + } + + for (loopNb = 1; loopNb <= g_nbIterations; loopNb++) { + double averageTime; + clock_t clockTime; + + PROGRESS("%1i- %-28.28s :%9i ->\r", loopNb, compressorName, (int)benchedSize); + { size_t i; for (i=0; i%9i (%5.2f%%),%7.1f MB/s\r", loopNb, compressorName, (int)benchedSize, (int)cSize, ratio, (double)benchedSize / bestTime / 1000000); + } + + if (ratio<100.) + DISPLAY("%2i-%-28.28s :%9i ->%9i (%5.2f%%),%7.1f MB/s\n", cAlgNb, compressorName, (int)benchedSize, (int)cSize, ratio, (double)benchedSize / bestTime / 1000000); + else + DISPLAY("%2i-%-28.28s :%9i ->%9i (%5.1f%%),%7.1f MB/s\n", cAlgNb, compressorName, (int)benchedSize, (int)cSize, ratio, (double)benchedSize / bestTime / 100000); + } + + /* Prepare layout for decompression */ + /* Init data chunks */ + { int i; + size_t remaining = benchedSize; + char* in = orig_buff; + char* out = compressed_buff; + + nbChunks = (int) (((int)benchedSize + (g_chunkSize-1))/ g_chunkSize); + for (i=0; i g_chunkSize) { chunkP[i].origSize = g_chunkSize; remaining -= g_chunkSize; } else { chunkP[i].origSize = (int)remaining; remaining = 0; } + chunkP[i].compressedBuffer = out; out += maxCompressedChunkSize; + chunkP[i].compressedSize = 0; + } + } + for (chunkNb=0; chunkNb\r", loopNb, dName, (int)benchedSize); + + nb_loops = 0; + clockTime = clock(); + while(clock() == clockTime); + clockTime = clock(); + while(BMK_GetClockSpan(clockTime) < TIMELOOP) { + for (chunkNb=0; chunkNb %7.1f MB/s\r", loopNb, dName, (int)benchedSize, (double)benchedSize / bestTime / 1000000); + + /* CRC Checking */ + crcDecoded = XXH32(orig_buff, (int)benchedSize, 0); + if (crcOriginal!=crcDecoded) { DISPLAY("\n!!! WARNING !!! %14s : Invalid Checksum : %x != %x\n", inFileName, (unsigned)crcOriginal, (unsigned)crcDecoded); exit(1); } + } + + DISPLAY("%2i-%-29.29s :%10i -> %7.1f MB/s\n", dAlgNb, dName, (int)benchedSize, (double)benchedSize / bestTime / 1000000); + } + } + free(orig_buff); + free(compressed_buff); + free(chunkP); + } + + Lizard_freeStream(Lizard_stream); + Lizard_freeStream(Lizard_streamPtr); + LizardF_freeDecompressionContext(g_dCtx); + if (g_pause) { printf("press enter...\n"); (void)getchar(); } + + return 0; +} + + +static int usage(const char* exename) +{ + DISPLAY( "Usage :\n"); + DISPLAY( " %s [arg] file1 file2 ... fileX\n", exename); + DISPLAY( "Arguments :\n"); + DISPLAY( " -c : compression tests only\n"); + DISPLAY( " -d : decompression tests only\n"); + DISPLAY( " -H/-h : Help (this text + advanced options)\n"); + return 0; +} + +static int usage_advanced(void) +{ + DISPLAY( "\nAdvanced options :\n"); + DISPLAY( " -c# : test only compression function # [1-%i]\n", NB_COMPRESSION_ALGORITHMS); + DISPLAY( " -d# : test only decompression function # [1-%i]\n", NB_DECOMPRESSION_ALGORITHMS); + DISPLAY( " -i# : iteration loops [1-9](default : %i)\n", NBLOOPS); + DISPLAY( " -B# : Block size [4-7](default : 7)\n"); + return 0; +} + +static int badusage(const char* exename) +{ + DISPLAY("Wrong parameters\n"); + usage(exename); + return 0; +} + +int main(int argc, const char** argv) +{ + int i, + filenamesStart=2; + const char* exename = argv[0]; + const char* input_filename=0; + + // Welcome message + DISPLAY(WELCOME_MESSAGE); + + if (argc<2) { badusage(exename); return 1; } + + for(i=1; i= '0') && (argument[1]<= '9')) { + g_compressionAlgo *= 10; + g_compressionAlgo += argument[1] - '0'; + argument++; + } + break; + + // Select decompression algorithm only + case 'd': + g_compressionTest = 0; + while ((argument[1]>= '0') && (argument[1]<= '9')) { + g_decompressionAlgo *= 10; + g_decompressionAlgo += argument[1] - '0'; + argument++; + } + break; + + // Display help on usage + case 'h' : + case 'H': usage(exename); usage_advanced(); return 0; + + // Modify Block Properties + case 'B': + while (argument[1]!=0) + switch(argument[1]) + { + case '4': + case '5': + case '6': + case '7': + { int B = argument[1] - '0'; + int S = 1 << (8 + 2*B); + BMK_setBlocksize(S); + argument++; + break; + } + case 'D': argument++; break; + default : goto _exit_blockProperties; + } +_exit_blockProperties: + break; + + // Modify Nb Iterations + case 'i': + if ((argument[1] >='0') && (argument[1] <='9')) { + int iters = argument[1] - '0'; + BMK_setNbIterations(iters); + argument++; + } + break; + + // Pause at the end (hidden option) + case 'p': BMK_setPause(); break; + + // Unknown command + default : badusage(exename); return 1; + } + } + continue; + } + + // first provided filename is input + if (!input_filename) { input_filename=argument; filenamesStart=i; continue; } + + } + + // No input filename ==> Error + if(!input_filename) { badusage(exename); return 1; } + + return fullSpeedBench(argv+filenamesStart, argc-filenamesStart); + +} diff --git a/contrib/lizard/tests/fuzzer.c b/contrib/lizard/tests/fuzzer.c new file mode 100644 index 00000000000..90864dc6d02 --- /dev/null +++ b/contrib/lizard/tests/fuzzer.c @@ -0,0 +1,1086 @@ +/* + fuzzer.c - Fuzzer test tool for Lizard + Copyright (C) Yann Collet 2012-2016 + Copyright (C) Przemyslaw Skibinski 2016-2017 + + GPL v2 License + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + + You can contact the author at : + - Lizard source repo : https://github.com/inikep/lizard +*/ + +/*-************************************ +* Compiler options +**************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define _CRT_SECURE_NO_WARNINGS /* fgets */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4146) /* disable: C4146: minus unsigned expression */ +# pragma warning(disable : 4310) /* disable: C4310: constant char value > 127 */ +#endif + +/* S_ISREG & gettimeofday() are not supported by MSVC */ +#if defined(_MSC_VER) || defined(_WIN32) +# define FUZ_LEGACY_TIMER 1 +#endif + + +/*-************************************ +* Includes +**************************************/ +#include +#include /* fgets, sscanf */ +#include /* strcmp */ +#include /* clock_t, clock, CLOCKS_PER_SEC */ +#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ +#include "lizard_decompress.h" +#include "lizard_common.h" +#define XXH_STATIC_LINKING_ONLY +#include "xxhash/xxhash.h" + + + +/*-************************************ +* Constants +**************************************/ +#define NB_ATTEMPTS (1<<16) +#define COMPRESSIBLE_NOISE_LENGTH (1 << 21) +#define FUZ_MAX_BLOCK_SIZE (1 << 17) +#define FUZ_MAX_DICT_SIZE (1 << 15) +#define FUZ_COMPRESSIBILITY_DEFAULT 60 +#define PRIME1 2654435761U +#define PRIME2 2246822519U +#define PRIME3 3266489917U + + +/*-*************************************** +* Macros +*****************************************/ +#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) +#define DISPLAYLEVEL(l, ...) if (g_displayLevel>=l) { DISPLAY(__VA_ARGS__); } +static int g_displayLevel = 2; +static const clock_t g_refreshRate = CLOCKS_PER_SEC * 25 / 100; +static clock_t g_time = 0; + + +/*-******************************************************* +* Fuzzer functions +*********************************************************/ +static clock_t FUZ_GetClockSpan(clock_t clockStart) +{ + return clock() - clockStart; /* works even if overflow; max span ~ 30mn */ +} + +static U32 FUZ_rotl32(U32 u32, U32 nbBits) +{ + return ((u32 << nbBits) | (u32 >> (32 - nbBits))); +} + +static U32 FUZ_rand(U32* src) +{ + U32 rand32 = *src; + rand32 *= PRIME1; + rand32 ^= PRIME2; + rand32 = FUZ_rotl32(rand32, 13); + *src = rand32; + return rand32; +} + + +#define FUZ_RAND15BITS ((FUZ_rand(seed) >> 3) & 32767) +#define FUZ_RANDLENGTH ( ((FUZ_rand(seed) >> 7) & 3) ? (FUZ_rand(seed) % 15) : (FUZ_rand(seed) % 510) + 15) +static void FUZ_fillCompressibleNoiseBuffer(void* buffer, size_t bufferSize, double proba, U32* seed) +{ + BYTE* BBuffer = (BYTE*)buffer; + size_t pos = 0; + U32 P32 = (U32)(32768 * proba); + + /* First Bytes */ + while (pos < 20) + BBuffer[pos++] = (BYTE)(FUZ_rand(seed)); + + while (pos < bufferSize) + { + /* Select : Literal (noise) or copy (within 64K) */ + if (FUZ_RAND15BITS < P32) + { + /* Copy (within 64K) */ + size_t match, d; + size_t length = FUZ_RANDLENGTH + 4; + size_t offset = FUZ_RAND15BITS + 1; + while (offset > pos) offset >>= 1; + d = pos + length; + while (d > bufferSize) d = bufferSize; + match = pos - offset; + while (pos < d) BBuffer[pos++] = BBuffer[match++]; + } + else + { + /* Literal (noise) */ + size_t d; + size_t length = FUZ_RANDLENGTH; + d = pos + length; + if (d > bufferSize) d = bufferSize; + while (pos < d) BBuffer[pos++] = (BYTE)(FUZ_rand(seed) >> 5); + } + } +} + + +#define MAX_NB_BUFF_I134 150 +#define BLOCKSIZE_I134 (32 MB) +/*! FUZ_AddressOverflow() : +* Aggressively pushes memory allocation limits, +* and generates patterns which create address space overflow. +* only possible in 32-bits mode */ +static int FUZ_AddressOverflow(U32* seed) +{ + char* buffers[MAX_NB_BUFF_I134]; + int i, nbBuff=0; + int highAddress = 0; + + DISPLAY("Overflow tests : "); + + /* Only possible in 32-bits */ + /* if (sizeof(void*)==8) + { + DISPLAY("64 bits mode : no overflow \n"); + fflush(stdout); + return 0; + }*/ + + buffers[0] = (char*)malloc(BLOCKSIZE_I134); + buffers[1] = (char*)malloc(BLOCKSIZE_I134); + if ((!buffers[0]) || (!buffers[1])) { + DISPLAY("not enough memory for tests \n"); + return 0; + } + + for (nbBuff=2; nbBuff < MAX_NB_BUFF_I134; nbBuff++) { + DISPLAY("%3i \b\b\b\b", nbBuff); + buffers[nbBuff] = (char*)malloc(BLOCKSIZE_I134); + if (buffers[nbBuff]==NULL) goto _endOfTests; + + if (((size_t)buffers[nbBuff] > (size_t)0x80000000) && (!highAddress)) { + DISPLAY("high address detected : "); + fflush(stdout); + highAddress=1; + } + + { int const nbOf255 = 1 + (FUZ_rand(seed) % (BLOCKSIZE_I134-1)); + char* const input = buffers[nbBuff-1]; + char* output = buffers[nbBuff]; + int r; + BYTE cLevel = LIZARD_MIN_CLEVEL + (FUZ_rand(seed) % (1+LIZARD_MAX_CLEVEL-LIZARD_MIN_CLEVEL)); + for(i = 5; i < nbOf255; i++) input[i] = (char)0xff; + for(i = 5; i < nbOf255; i+=(FUZ_rand(seed) % 128)) input[i] = (BYTE)(FUZ_rand(seed)%256); + + input[0] = (char)cLevel; /* Compression Level */ + input[1] = (char)0xF0; /* Literal length overflow */ + input[2] = (char)0xFF; + input[3] = (char)0xFF; + input[4] = (char)0xFF; + r = Lizard_decompress_safe(input, output, nbOf255, BLOCKSIZE_I134); + if (r>0 && r0 && r0 && r0 && r g_refreshRate) | (g_displayLevel>=3)) { + g_time = clock(); + DISPLAY("\r%5u ", testNb); + if (g_displayLevel>=3) fflush(stdout); + } +} + + +/*! FUZ_findDiff() : +* find the first different byte between buff1 and buff2. +* presumes buff1 != buff2. +* presumes a difference exists before end of either buffer. +* Typically invoked after a checksum mismatch. +*/ +static void FUZ_findDiff(const void* buff1, const void* buff2) +{ + const BYTE* const b1 = (const BYTE*)buff1; + const BYTE* const b2 = (const BYTE*)buff2; + size_t i=0; + while (b1[i]==b2[i]) i++; + DISPLAY("Wrong Byte at position %u\n", (unsigned)i); +} + + +static int FUZ_test(U32 seed, U32 nbCycles, const U32 startCycle, const double compressibility, U32 duration_s) +{ + unsigned long long bytes = 0; + unsigned long long cbytes = 0; + unsigned long long hcbytes = 0; + unsigned long long ccbytes = 0; + void* CNBuffer; + char* compressedBuffer; + char* decodedBuffer; +# define FUZ_max LIZARD_COMPRESSBOUND(LEN) + int ret; + unsigned cycleNb; +# define FUZ_CHECKTEST(cond, ...) if (cond) { printf("Test %u : ", testNb); printf(__VA_ARGS__); \ + printf(" (seed %u, cycle %u) \n", seed, cycleNb); goto _output_error; } +# define FUZ_DISPLAYTEST { testNb++; g_displayLevel<3 ? 0 : printf("%2u\b\b", testNb); if (g_displayLevel==4) fflush(stdout); } + void* stateLizard = malloc(Lizard_sizeofState_MinLevel()); + void* stateLizardHC = malloc(Lizard_sizeofState(0)); + Lizard_stream_t* Lizarddict; + Lizard_stream_t* Lizard_streamHCPtr; + U32 crcOrig, crcCheck; + U32 coreRandState = seed; + U32 randState = coreRandState ^ PRIME3; + int result = 0; + clock_t const clockStart = clock(); + clock_t const clockDuration = (clock_t)duration_s * CLOCKS_PER_SEC; + + + /* init */ + Lizard_streamHCPtr = Lizard_createStream(0); + Lizarddict = Lizard_createStream_MinLevel(); + + /* Create compressible test buffer */ + CNBuffer = malloc(COMPRESSIBLE_NOISE_LENGTH); + compressedBuffer = (char*)malloc(Lizard_compressBound(FUZ_MAX_BLOCK_SIZE)); + decodedBuffer = (char*)malloc(FUZ_MAX_DICT_SIZE + FUZ_MAX_BLOCK_SIZE); + + if (!stateLizard || !stateLizardHC || !Lizarddict || !Lizard_streamHCPtr || !CNBuffer || !compressedBuffer || !decodedBuffer) goto _output_error; + + FUZ_fillCompressibleNoiseBuffer(CNBuffer, COMPRESSIBLE_NOISE_LENGTH, compressibility, &randState); + + /* move to startCycle */ + for (cycleNb = 0; cycleNb < startCycle; cycleNb++) { + (void)FUZ_rand(&coreRandState); + + if (0) { /* some problems can be related to dictionary re-use; in this case, enable this loop */ + int dictSize, blockSize, blockStart; + char* dict; + char* block; + FUZ_displayUpdate(cycleNb); + randState = coreRandState ^ PRIME3; + blockSize = FUZ_rand(&randState) % FUZ_MAX_BLOCK_SIZE; + blockStart = FUZ_rand(&randState) % (COMPRESSIBLE_NOISE_LENGTH - blockSize); + dictSize = FUZ_rand(&randState) % FUZ_MAX_DICT_SIZE; + if (dictSize > blockStart) dictSize = blockStart; + block = ((char*)CNBuffer) + blockStart; + dict = block - dictSize; + Lizard_loadDict(Lizarddict, dict, dictSize); + Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + Lizard_loadDict(Lizarddict, dict, dictSize); + Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + Lizard_loadDict(Lizarddict, dict, dictSize); + Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + } } + + /* Main test loop */ + for (cycleNb = startCycle; (cycleNb < nbCycles) || (FUZ_GetClockSpan(clockStart) < clockDuration) ; cycleNb++) { + U32 testNb = 0; + char* dict; + char* block; + int dictSize, blockSize, blockStart, compressedSize, HCcompressedSize; + int blockContinueCompressedSize; + + FUZ_displayUpdate(cycleNb); + (void)FUZ_rand(&coreRandState); + randState = coreRandState ^ PRIME3; + + /* Select block to test */ + blockSize = (FUZ_rand(&randState) % (FUZ_MAX_BLOCK_SIZE-1)) + 1; + blockStart = FUZ_rand(&randState) % (COMPRESSIBLE_NOISE_LENGTH - blockSize); + dictSize = FUZ_rand(&randState) % FUZ_MAX_DICT_SIZE; + if (dictSize > blockStart) dictSize = blockStart; + block = ((char*)CNBuffer) + blockStart; + dict = block - dictSize; + + /* Compression tests */ + + /* Test compression HC */ + FUZ_DISPLAYTEST; + ret = Lizard_compress(block, compressedBuffer, blockSize, Lizard_compressBound(blockSize), 0); + FUZ_CHECKTEST(ret==0, "Lizard_compress() failed"); + HCcompressedSize = ret; + + /* Test compression HC using external state */ + FUZ_DISPLAYTEST; + ret = Lizard_compress_extState(stateLizardHC, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize), 0); + FUZ_CHECKTEST(ret==0, "Lizard_compress_extState() failed"); + + /* Test compression using external state */ + FUZ_DISPLAYTEST; + ret = Lizard_compress_extState_MinLevel(stateLizard, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + FUZ_CHECKTEST(ret==0, "Lizard_compress_extState_MinLevel(1) failed"); + + /* Test compression */ + FUZ_DISPLAYTEST; + ret = Lizard_compress_MinLevel(block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + FUZ_CHECKTEST(ret==0, "Lizard_compress_MinLevel() failed"); + compressedSize = ret; + + /* Decompression tests */ + + crcOrig = XXH32(block, blockSize, 0); + + /* Test decoding with output size exactly what's necessary => must work */ + FUZ_DISPLAYTEST; + decodedBuffer[blockSize] = 0; + ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize); + FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe failed despite sufficient space"); + FUZ_CHECKTEST(ret!=blockSize, "Lizard_decompress_safe did not regenerate original data"); + FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe overrun specified output buffer size"); + crcCheck = XXH32(decodedBuffer, blockSize, 0); + FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe corrupted decoded data"); + + // Test decoding with more than enough output size => must work + FUZ_DISPLAYTEST; + decodedBuffer[blockSize] = 0; + decodedBuffer[blockSize+1] = 0; + ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize+1); + FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe failed despite amply sufficient space"); + FUZ_CHECKTEST(ret!=blockSize, "Lizard_decompress_safe did not regenerate original data"); + //FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe wrote more than (unknown) target size"); // well, is that an issue ? + FUZ_CHECKTEST(decodedBuffer[blockSize+1], "Lizard_decompress_safe overrun specified output buffer size"); + crcCheck = XXH32(decodedBuffer, blockSize, 0); + FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe corrupted decoded data"); + + // Test decoding with output size being one byte too short => must fail + FUZ_DISPLAYTEST; + decodedBuffer[blockSize-1] = 0; + ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize-1); + FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to Output Size being one byte too short"); + FUZ_CHECKTEST(decodedBuffer[blockSize-1], "Lizard_decompress_safe overrun specified output buffer size"); + + // Test decoding with output size being 10 bytes too short => must fail + FUZ_DISPLAYTEST; + if (blockSize>10) + { + decodedBuffer[blockSize-10] = 0; + ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize-10); + FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to Output Size being 10 bytes too short"); + FUZ_CHECKTEST(decodedBuffer[blockSize-10], "Lizard_decompress_safe overrun specified output buffer size"); + } + + // Test decoding with input size being one byte too short => must fail + FUZ_DISPLAYTEST; + ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize-1, blockSize); + FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to input size being one byte too short (blockSize=%i, ret=%i, compressedSize=%i)", blockSize, ret, compressedSize); + + // Test decoding with input size being one byte too large => must fail + FUZ_DISPLAYTEST; + decodedBuffer[blockSize] = 0; + compressedBuffer[compressedSize] = 0; /* valgrind */ + ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize+1, blockSize); + FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to input size being too large"); + FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe overrun specified output buffer size"); + + // Test partial decoding with target output size being max/2 => must work + FUZ_DISPLAYTEST; + ret = Lizard_decompress_safe_partial(compressedBuffer, decodedBuffer, compressedSize, blockSize/2, blockSize); + FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe_partial failed despite sufficient space"); + + // Test partial decoding with target output size being just below max => must work + FUZ_DISPLAYTEST; + ret = Lizard_decompress_safe_partial(compressedBuffer, decodedBuffer, compressedSize, blockSize-3, blockSize); + FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe_partial failed despite sufficient space"); + + /* Test Compression with limited output size */ + + /* Test compression with output size being exactly what's necessary (should work) */ + FUZ_DISPLAYTEST; + ret = Lizard_compress_MinLevel(block, compressedBuffer, blockSize, compressedSize); + FUZ_CHECKTEST(ret==0, "Lizard_compress_MinLevel() failed despite sufficient space"); + + /* Test compression with output size being exactly what's necessary and external state (should work) */ + FUZ_DISPLAYTEST; + ret = Lizard_compress_extState_MinLevel(stateLizard, block, compressedBuffer, blockSize, compressedSize); + FUZ_CHECKTEST(ret==0, "Lizard_compress_extState_MinLevel() failed despite sufficient space"); + + /* Test HC compression with output size being exactly what's necessary (should work) */ + FUZ_DISPLAYTEST; + ret = Lizard_compress(block, compressedBuffer, blockSize, HCcompressedSize, 0); + FUZ_CHECKTEST(ret==0, "Lizard_compress() (limitedOutput) failed despite sufficient space"); + + /* Test HC compression with output size being exactly what's necessary (should work) */ + FUZ_DISPLAYTEST; + ret = Lizard_compress_extState(stateLizardHC, block, compressedBuffer, blockSize, HCcompressedSize, 0); + FUZ_CHECKTEST(ret==0, "Lizard_compress_extState() failed despite sufficient space"); + + /* Test compression with missing bytes into output buffer => must fail */ + FUZ_DISPLAYTEST; + { int missingBytes = (FUZ_rand(&randState) % 0x3F) + 1; + if (missingBytes >= compressedSize) missingBytes = compressedSize-1; + missingBytes += !missingBytes; /* avoid special case missingBytes==0 */ + compressedBuffer[compressedSize-missingBytes] = 0; + ret = Lizard_compress_MinLevel(block, compressedBuffer, blockSize, compressedSize-missingBytes); + FUZ_CHECKTEST(ret, "Lizard_compress_MinLevel should have failed (output buffer too small by %i byte)", missingBytes); + FUZ_CHECKTEST(compressedBuffer[compressedSize-missingBytes], "Lizard_compress_MinLevel overran output buffer ! (%i missingBytes)", missingBytes) + } + + /* Test HC compression with missing bytes into output buffer => must fail */ + FUZ_DISPLAYTEST; + { int missingBytes = (FUZ_rand(&randState) % 0x3F) + 1; + if (missingBytes >= HCcompressedSize) missingBytes = HCcompressedSize-1; + missingBytes += !missingBytes; /* avoid special case missingBytes==0 */ + compressedBuffer[HCcompressedSize-missingBytes] = 0; + ret = Lizard_compress(block, compressedBuffer, blockSize, HCcompressedSize-missingBytes, 0); + FUZ_CHECKTEST(ret, "Lizard_compress(limitedOutput) should have failed (output buffer too small by %i byte)", missingBytes); + FUZ_CHECKTEST(compressedBuffer[HCcompressedSize-missingBytes], "Lizard_compress overran output buffer ! (%i missingBytes)", missingBytes) + } + + + /*-******************/ + /* Dictionary tests */ + /*-******************/ + + /* Compress using dictionary */ + FUZ_DISPLAYTEST; + { Lizard_stream_t* Lizard_stream = Lizard_createStream_MinLevel(); + FUZ_CHECKTEST(Lizard_stream==NULL, "Lizard_createStream_MinLevel() allocation failed"); + Lizard_stream = Lizard_resetStream_MinLevel(Lizard_stream); + FUZ_CHECKTEST(Lizard_stream==NULL, "Lizard_resetStream_MinLevel() failed"); + Lizard_compress_continue (Lizard_stream, dict, compressedBuffer, dictSize, Lizard_compressBound(dictSize)); /* Just to fill hash tables */ + blockContinueCompressedSize = Lizard_compress_continue (Lizard_stream, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + FUZ_CHECKTEST(blockContinueCompressedSize==0, "Lizard_compress_continue failed"); + Lizard_freeStream(Lizard_stream); + } + + /* Compress using External dictionary */ + FUZ_DISPLAYTEST; + dict -= (FUZ_rand(&randState) & 0xF) + 1; /* Separation, so it is an ExtDict */ + if (dict < (char*)CNBuffer) dict = (char*)CNBuffer; + Lizard_loadDict(Lizarddict, dict, dictSize); + blockContinueCompressedSize = Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + FUZ_CHECKTEST(blockContinueCompressedSize==0, "Lizard_compress_continue failed"); + + FUZ_DISPLAYTEST; + Lizard_loadDict(Lizarddict, dict, dictSize); + ret = Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, blockContinueCompressedSize-1); + FUZ_CHECKTEST(ret>0, "Lizard_compress_continue using ExtDict should fail : one missing byte for output buffer : %i written, %i buffer", ret, blockContinueCompressedSize); + + FUZ_DISPLAYTEST; + Lizard_loadDict(Lizarddict, dict, dictSize); + ret = Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, blockContinueCompressedSize); + FUZ_CHECKTEST(ret!=blockContinueCompressedSize, "Lizard_compress_limitedOutput_compressed size is different (%i != %i)", ret, blockContinueCompressedSize); + FUZ_CHECKTEST(ret<=0, "Lizard_compress_continue should work : enough size available within output buffer"); + + FUZ_DISPLAYTEST; + decodedBuffer[blockSize] = 0; + ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize, dict, dictSize); + FUZ_CHECKTEST(ret!=blockSize, "2Lizard_decompress_safe_usingDict did not regenerate original data ret[%d]!=blockSize[%d]", (int)ret, (int)blockSize); + FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe_usingDict overrun specified output buffer size") + crcCheck = XXH32(decodedBuffer, blockSize, 0); + FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe_usingDict corrupted decoded data"); + + FUZ_DISPLAYTEST; + decodedBuffer[blockSize-1] = 0; + ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize-1, dict, dictSize); + FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe_usingDict should have failed : not enough output size (-1 byte)"); + FUZ_CHECKTEST(decodedBuffer[blockSize-1], "Lizard_decompress_safe_usingDict overrun specified output buffer size"); + + FUZ_DISPLAYTEST; + { U32 const missingBytes = (FUZ_rand(&randState) & 0xF) + 2; + if ((U32)blockSize > missingBytes) { + decodedBuffer[blockSize-missingBytes] = 0; + ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize-missingBytes, dict, dictSize); + FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe_usingDict should have failed : output buffer too small (-%u byte)", missingBytes); + FUZ_CHECKTEST(decodedBuffer[blockSize-missingBytes], "Lizard_decompress_safe_usingDict overrun specified output buffer size (-%u byte) (blockSize=%i)", missingBytes, blockSize); + } } + + /* Compress HC using External dictionary */ + FUZ_DISPLAYTEST; + dict -= (FUZ_rand(&randState) & 7); /* even bigger separation */ + if (dict < (char*)CNBuffer) dict = (char*)CNBuffer; + Lizard_streamHCPtr = Lizard_resetStream (Lizard_streamHCPtr, FUZ_rand(&randState) & 0x7); + FUZ_CHECKTEST(Lizard_streamHCPtr==NULL, "Lizard_resetStream failed"); + Lizard_loadDict(Lizard_streamHCPtr, dict, dictSize); + blockContinueCompressedSize = Lizard_compress_continue(Lizard_streamHCPtr, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); + FUZ_CHECKTEST(blockContinueCompressedSize==0, "Lizard_compress_continue failed"); + + FUZ_DISPLAYTEST; + Lizard_loadDict(Lizard_streamHCPtr, dict, dictSize); + ret = Lizard_compress_continue(Lizard_streamHCPtr, block, compressedBuffer, blockSize, blockContinueCompressedSize-1); + FUZ_CHECKTEST(ret>0, "Lizard_compress_continue using ExtDict should fail : one missing byte for output buffer"); + + FUZ_DISPLAYTEST; + Lizard_loadDict(Lizard_streamHCPtr, dict, dictSize); + ret = Lizard_compress_continue(Lizard_streamHCPtr, block, compressedBuffer, blockSize, blockContinueCompressedSize); + FUZ_CHECKTEST(ret!=blockContinueCompressedSize, "Lizard_compress_limitedOutput_compressed size is different (%i != %i)", ret, blockContinueCompressedSize); + FUZ_CHECKTEST(ret<=0, "Lizard_compress_continue should work : enough size available within output buffer"); + + FUZ_DISPLAYTEST; + decodedBuffer[blockSize] = 0; + ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize, dict, dictSize); + FUZ_CHECKTEST(ret!=blockSize, "3Lizard_decompress_safe_usingDict did not regenerate original data ret[%d]!=blockSize[%d]", (int)ret, (int)blockSize); + FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe_usingDict overrun specified output buffer size") + crcCheck = XXH32(decodedBuffer, blockSize, 0); + if (crcCheck!=crcOrig) + FUZ_findDiff(block, decodedBuffer); + FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe_usingDict corrupted decoded data"); + + /* ***** End of tests *** */ + /* Fill stats */ + bytes += blockSize; + cbytes += compressedSize; + hcbytes += HCcompressedSize; + ccbytes += blockContinueCompressedSize; + } + + if (nbCycles<=1) nbCycles = cycleNb; /* end by time */ + bytes += !bytes; /* avoid division by 0 */ + printf("\r%7u /%7u - ", cycleNb, nbCycles); + printf("all tests completed successfully \n"); + printf("compression ratio: %0.3f%%\n", (double)cbytes/bytes*100); + printf("HC compression ratio: %0.3f%%\n", (double)hcbytes/bytes*100); + printf("ratio with dict: %0.3f%%\n", (double)ccbytes/bytes*100); + + /* release memory */ + { +_exit: + free(CNBuffer); + free(compressedBuffer); + free(decodedBuffer); + free(stateLizard); + free(stateLizardHC); + Lizard_freeStream(Lizard_streamHCPtr); + Lizard_freeStream(Lizarddict); + return result; + +_output_error: + result = 1; + goto _exit; + } +} + + +#define testInputSize (192 KB) +#define testCompressedSize (128 KB) +#define ringBufferSize (8 KB) + +static void FUZ_unitTests(U32 seed) +{ + const unsigned testNb = 0; + const unsigned cycleNb= 0; + char testInput[testInputSize]; + char testCompressed[testCompressedSize]; + char testVerify[testInputSize]; + char ringBuffer[ringBufferSize]; + U32 randState = seed ^ PRIME3; + + /* Init */ + FUZ_fillCompressibleNoiseBuffer(testInput, testInputSize, 0.50, &randState); + + /* 32-bits address space overflow test */ + FUZ_AddressOverflow(&randState); + + /* Lizard streaming tests */ + { Lizard_stream_t* statePtr; + Lizard_stream_t* streamingState; + U64 crcOrig; + U64 crcNew; + int result; + + /* Allocation test */ + statePtr = Lizard_createStream_MinLevel(); + FUZ_CHECKTEST(statePtr==NULL, "Lizard_createStream_MinLevel() allocation failed"); + Lizard_freeStream(statePtr); + + streamingState = Lizard_createStream_MinLevel(); + FUZ_CHECKTEST(streamingState==NULL, "Lizard_createStream_MinLevel() allocation failed"); + + /* simple compression test */ + crcOrig = XXH64(testInput, testCompressedSize, 0); + streamingState = Lizard_resetStream_MinLevel(streamingState); + FUZ_CHECKTEST(streamingState==NULL, "Lizard_resetStream_MinLevel() failed"); + result = Lizard_compress_continue(streamingState, testInput, testCompressed, testCompressedSize, testCompressedSize-1); + FUZ_CHECKTEST(result==0, "Lizard_compress_continue() compression failed"); + + result = Lizard_decompress_safe(testCompressed, testVerify, result, testCompressedSize); + FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe() decompression failed Level 1 (result=%d testCompressedSize=%d)", (int)result, (int)testCompressedSize); + crcNew = XXH64(testVerify, testCompressedSize, 0); + FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() decompression corruption"); + + /* ring buffer test */ + { XXH64_state_t xxhOrig; + XXH64_state_t xxhNew; + Lizard_streamDecode_t decodeState; + const U32 maxMessageSizeLog = 10; + const U32 maxMessageSizeMask = (1< ringBufferSize) rNext = 0; + if (dNext + messageSize > dBufferSize) dNext = 0; + } + } + Lizard_freeStream(streamingState); + } + + /* Lizard streaming tests */ + { Lizard_stream_t* streamPtr; + U64 crcOrig; + U64 crcNew; + int result; + + /* Allocation test */ + streamPtr = Lizard_createStream(0); + FUZ_CHECKTEST(streamPtr==NULL, "Lizard_createStream() allocation failed"); + + /* simple HC compression test */ + crcOrig = XXH64(testInput, testCompressedSize, 0); + streamPtr = Lizard_resetStream(streamPtr, 0); + FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); + result = Lizard_compress_continue(streamPtr, testInput, testCompressed, testCompressedSize, testCompressedSize-1); + FUZ_CHECKTEST(result==0, "Lizard_compress_continue() compression failed"); + + result = Lizard_decompress_safe(testCompressed, testVerify, result, testCompressedSize); + FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe() decompression failed Level 0 (result=%d testCompressedSize=%d)", (int)result, (int)testCompressedSize); + crcNew = XXH64(testVerify, testCompressedSize, 0); + FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() decompression corruption"); + + /* simple dictionary HC compression test */ + crcOrig = XXH64(testInput + testInputSize - testCompressedSize, testCompressedSize, 0); + streamPtr = Lizard_resetStream(streamPtr, 0); + FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); + Lizard_loadDict(streamPtr, testInput, testInputSize - testCompressedSize); + result = Lizard_compress_continue(streamPtr, testInput + testInputSize - testCompressedSize, testCompressed, testCompressedSize, testCompressedSize-1); + FUZ_CHECKTEST(result==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result); + + result = Lizard_decompress_safe_usingDict(testCompressed, testVerify, result, testCompressedSize, testInput, testInputSize - testCompressedSize); + FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe() simple dictionary decompression test failed"); + crcNew = XXH64(testVerify, testCompressedSize, 0); + FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() simple dictionary decompression test : corruption"); + + /* multiple HC compression test with dictionary */ + { int result1, result2; + int segSize = testCompressedSize / 2; + crcOrig = XXH64(testInput + segSize, testCompressedSize, 0); + streamPtr = Lizard_resetStream(streamPtr, 0); + FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); + Lizard_loadDict(streamPtr, testInput, segSize); + result1 = Lizard_compress_continue(streamPtr, testInput + segSize, testCompressed, segSize, segSize -1); + FUZ_CHECKTEST(result1==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result1); + result2 = Lizard_compress_continue(streamPtr, testInput + 2*segSize, testCompressed+result1, segSize, segSize-1); + FUZ_CHECKTEST(result2==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result2); + + result = Lizard_decompress_safe_usingDict(testCompressed, testVerify, result1, segSize, testInput, segSize); + FUZ_CHECKTEST(result!=segSize, "Lizard_decompress_safe() dictionary decompression part 1 failed"); + result = Lizard_decompress_safe_usingDict(testCompressed+result1, testVerify+segSize, result2, segSize, testInput, 2*segSize); + FUZ_CHECKTEST(result!=segSize, "Lizard_decompress_safe() dictionary decompression part 2 failed"); + crcNew = XXH64(testVerify, testCompressedSize, 0); + FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() dictionary decompression corruption"); + } + + /* remote dictionary HC compression test */ + crcOrig = XXH64(testInput + testInputSize - testCompressedSize, testCompressedSize, 0); + streamPtr = Lizard_resetStream(streamPtr, 0); + FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); + Lizard_loadDict(streamPtr, testInput, 32 KB); + result = Lizard_compress_continue(streamPtr, testInput + testInputSize - testCompressedSize, testCompressed, testCompressedSize, testCompressedSize-1); + FUZ_CHECKTEST(result==0, "Lizard_compress_continue() remote dictionary failed : result = %i", result); + + result = Lizard_decompress_safe_usingDict(testCompressed, testVerify, result, testCompressedSize, testInput, 32 KB); + FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe_usingDict() decompression failed following remote dictionary HC compression test"); + crcNew = XXH64(testVerify, testCompressedSize, 0); + FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe_usingDict() decompression corruption"); + + /* multiple HC compression with ext. dictionary */ + { XXH64_state_t crcOrigState; + XXH64_state_t crcNewState; + const char* dict = testInput + 3; + int dictSize = (FUZ_rand(&randState) & 8191); + char* dst = testVerify; + + size_t segStart = dictSize + 7; + int segSize = (FUZ_rand(&randState) & 8191); + int segNb = 1; + + streamPtr = Lizard_resetStream(streamPtr, 0); + FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); + Lizard_loadDict(streamPtr, dict, dictSize); + + XXH64_reset(&crcOrigState, 0); + XXH64_reset(&crcNewState, 0); + + while (segStart + segSize < testInputSize) { + XXH64_update(&crcOrigState, testInput + segStart, segSize); + crcOrig = XXH64_digest(&crcOrigState); + result = Lizard_compress_continue(streamPtr, testInput + segStart, testCompressed, segSize, Lizard_compressBound(segSize)); + FUZ_CHECKTEST(result==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result); + + result = Lizard_decompress_safe_usingDict(testCompressed, dst, result, segSize, dict, dictSize); + FUZ_CHECKTEST(result!=segSize, "Lizard_decompress_safe_usingDict() dictionary decompression part %i failed", segNb); + XXH64_update(&crcNewState, dst, segSize); + crcNew = XXH64_digest(&crcNewState); + if (crcOrig!=crcNew) { + size_t c=0; + while (dst[c] == testInput[segStart+c]) c++; + DISPLAY("Bad decompression at %u / %u \n", (U32)c, (U32)segSize); + } + FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe_usingDict() part %i corruption", segNb); + + dict = dst; + //dict = testInput + segStart; + dictSize = segSize; + + dst += segSize + 1; + segNb ++; + + segStart += segSize + (FUZ_rand(&randState) & 0xF) + 1; + segSize = (FUZ_rand(&randState) & 8191); + } + } + + /* ring buffer test */ + { XXH64_state_t xxhOrig; + XXH64_state_t xxhNew; + Lizard_streamDecode_t decodeState; + const U32 maxMessageSizeLog = 10; + const U32 maxMessageSizeMask = (1< ringBufferSize) rNext = 0; + if (dNext + messageSize > dBufferSize) dNext = 0; + } + } + + /* small decoder-side ring buffer test */ + { XXH64_state_t xxhOrig; + XXH64_state_t xxhNew; + Lizard_streamDecode_t decodeState; + const U32 maxMessageSizeLog = 12; + const U32 maxMessageSizeMask = (1< dBufferSize) dNext = 0; + } + } + + Lizard_freeStream(streamPtr); + } + + printf("All unit tests completed successfully \n"); + return; +_output_error: + exit(1); +} + + +static int FUZ_usage(char* programName) +{ + DISPLAY( "Usage :\n"); + DISPLAY( " %s [args]\n", programName); + DISPLAY( "\n"); + DISPLAY( "Arguments :\n"); + DISPLAY( " -i# : Nb of tests (default:%i) \n", NB_ATTEMPTS); + DISPLAY( " -T# : Duration of tests, in seconds (default: use Nb of tests) \n"); + DISPLAY( " -s# : Select seed (default:prompt user)\n"); + DISPLAY( " -t# : Select starting test number (default:0)\n"); + DISPLAY( " -P# : Select compressibility in %% (default:%i%%)\n", FUZ_COMPRESSIBILITY_DEFAULT); + DISPLAY( " -v : verbose\n"); + DISPLAY( " -p : pause at the end\n"); + DISPLAY( " -h : display help and exit\n"); + return 0; +} + + +int main(int argc, char** argv) +{ + U32 seed=0; + int seedset=0; + int argNb; + int nbTests = NB_ATTEMPTS; + int testNb = 0; + int proba = FUZ_COMPRESSIBILITY_DEFAULT; + int pause = 0; + char* programName = argv[0]; + U32 duration = 0; + + /* Check command line */ + for(argNb=1; argNb='0') && (*argument<='9')) { + nbTests *= 10; + nbTests += *argument - '0'; + argument++; + } + break; + + case 'T': + argument++; + nbTests = 0; duration = 0; + for (;;) { + switch(*argument) + { + case 'm': duration *= 60; argument++; continue; + case 's': + case 'n': argument++; continue; + case '0': + case '1': + case '2': + case '3': + case '4': + case '5': + case '6': + case '7': + case '8': + case '9': duration *= 10; duration += *argument++ - '0'; continue; + } + break; + } + break; + + case 's': + argument++; + seed=0; seedset=1; + while ((*argument>='0') && (*argument<='9')) { + seed *= 10; + seed += *argument - '0'; + argument++; + } + break; + + case 't': /* select starting test nb */ + argument++; + testNb=0; + while ((*argument>='0') && (*argument<='9')) { + testNb *= 10; + testNb += *argument - '0'; + argument++; + } + break; + + case 'P': /* change probability */ + argument++; + proba=0; + while ((*argument>='0') && (*argument<='9')) { + proba *= 10; + proba += *argument - '0'; + argument++; + } + if (proba<0) proba=0; + if (proba>100) proba=100; + break; + default: ; + } + } + } + } + + printf("Starting Lizard fuzzer (%i-bits, v%s)\n", (int)(sizeof(size_t)*8), LIZARD_VERSION_STRING); + + if (!seedset) { + time_t const t = time(NULL); + U32 const h = XXH32(&t, sizeof(t), 1); + seed = h % 10000; + } + printf("Seed = %u\n", seed); + + if (proba!=FUZ_COMPRESSIBILITY_DEFAULT) printf("Compressibility : %i%%\n", proba); + + if (testNb==0) FUZ_unitTests(seed); + + if (nbTests<=0) nbTests=1; + + { int const result = FUZ_test(seed, nbTests, testNb, ((double)proba) / 100, duration); + if (pause) { + DISPLAY("press enter ... \n"); + (void)getchar(); + } + return result; + } +} diff --git a/contrib/lizard/visual/.gitignore b/contrib/lizard/visual/.gitignore new file mode 100644 index 00000000000..538cd1a21c8 --- /dev/null +++ b/contrib/lizard/visual/.gitignore @@ -0,0 +1,11 @@ +# Visual C++ +.vs/ +*Copy +*.db +*.opensdf +*.sdf +*.suo +*.user +*.opendb + +VS2010/bin/ diff --git a/contrib/lizard/visual/README.md b/contrib/lizard/visual/README.md new file mode 100644 index 00000000000..527e4be1874 --- /dev/null +++ b/contrib/lizard/visual/README.md @@ -0,0 +1,53 @@ +Projects for various integrated development environments (IDE) +============================================================== + +#### Included projects + +The following projects are included with the lizard distribution: +- `VS2010` - Visual Studio 2010 project (which also works well with Visual Studio 2012, 2013, 2015) + + +#### How to compile lizard with Visual Studio + +1. Install Visual Studio e.g. VS 2015 Community Edition (it's free). +2. Download the latest version of lizard from https://github.com/inikep/lizard/releases +3. Decompress ZIP archive. +4. Go to decompressed directory then to `visual` then `VS2010` and open `lizard.sln` +5. Visual Studio will ask about converting VS2010 project to VS2015 and you should agree. +6. Change `Debug` to `Release` and if you have 64-bit Windows change also `Win32` to `x64`. +7. Press F7 on keyboard or select `BUILD` from the menu bar and choose `Build Solution`. +8. If compilation will be fine a compiled executable will be in `visual\VS2010\bin\x64_Release\lizard.exe` + + +#### Projects available within lizard.sln + +The Visual Studio solution file `lizard.sln` contains many projects that will be compiled to the +`visual\VS2010\bin\$(Platform)_$(Configuration)` directory. For example `lizard` set to `x64` and +`Release` will be compiled to `visual\VS2010\bin\x64_Release\lizard.exe`. The solution file contains the +following projects: + +- `lizard` : Command Line Utility, supporting gzip-like arguments +- `datagen` : Synthetic and parametrable data generator, for tests +- `frametest` : Test tool that checks lizard_frame integrity on target platform +- `fullbench` : Precisely measure speed for each lizard inner functions +- `fuzzer` : Test tool, to check lizard integrity on target platform +- `liblizard` : A static Lizard library compiled to `liblizard_static.lib` +- `liblizard-dll` : A dynamic Lizard library (DLL) compiled to `liblizard.dll` with the import library `liblizard.lib` +- `fullbench-dll` : The fullbench program compiled with the import library; the executable requires Lizard DLL + + +#### Using Lizard DLL with Microsoft Visual C++ project + +The header files `lib\lizard.h`, `lib\lizardhc.h`, `lib\lizard_frame.h` and the import library +`visual\VS2010\bin\$(Platform)_$(Configuration)\liblizard.lib` are required to compile a +project using Visual C++. + +1. The path to header files should be added to `Additional Include Directories` that can + be found in Project Properties of Visual Studio IDE in the `C/C++` Property Pages on the `General` page. +2. The import library has to be added to `Additional Dependencies` that can + be found in Project Properties in the `Linker` Property Pages on the `Input` page. + If one will provide only the name `liblizard.lib` without a full path to the library + then the directory has to be added to `Linker\General\Additional Library Directories`. + +The compiled executable will require Lizard DLL which is available at +`visual\VS2010\bin\$(Platform)_$(Configuration)\liblizard.dll`. diff --git a/contrib/lizard/visual/VS2010/datagen/datagen.vcxproj b/contrib/lizard/visual/VS2010/datagen/datagen.vcxproj new file mode 100644 index 00000000000..7f7581d01b8 --- /dev/null +++ b/contrib/lizard/visual/VS2010/datagen/datagen.vcxproj @@ -0,0 +1,165 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {D745AE2F-596A-403A-9B91-81A8C6779243} + Win32Proj + datagen + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + Application + true + Unicode + + + Application + true + Unicode + + + Application + false + true + Unicode + + + Application + false + true + Unicode + + + + + + + + + + + + + + + + + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + + + Console + true + + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + false + + + Console + true + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + true + true + + + + + + + + + + + + + \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/frametest/frametest.vcxproj b/contrib/lizard/visual/VS2010/frametest/frametest.vcxproj new file mode 100644 index 00000000000..b753d651be1 --- /dev/null +++ b/contrib/lizard/visual/VS2010/frametest/frametest.vcxproj @@ -0,0 +1,177 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7} + Win32Proj + frametest + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + Application + true + Unicode + + + Application + true + Unicode + + + Application + false + true + Unicode + + + Application + false + true + Unicode + + + + + + + + + + + + + + + + + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + + + Console + true + + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + false + + + Console + true + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + true + true + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj b/contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj new file mode 100644 index 00000000000..f05bb298055 --- /dev/null +++ b/contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj @@ -0,0 +1,175 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E} + Win32Proj + fullbench + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + Application + true + Unicode + + + Application + true + Unicode + + + Application + false + true + Unicode + + + Application + false + true + Unicode + + + + + + + + + + + + + + + + + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + + + Console + true + + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + + + Console + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + false + + + Console + true + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + false + + + Console + true + true + true + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj b/contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj new file mode 100644 index 00000000000..7fa0fec022f --- /dev/null +++ b/contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj @@ -0,0 +1,174 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {18B9F1A7-9C66-4352-898B-30804DADE0FD} + Win32Proj + fuzzer + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + Application + true + Unicode + + + Application + true + Unicode + + + Application + false + true + Unicode + + + Application + false + true + Unicode + + + + + + + + + + + + + + + + + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + + + Console + true + + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + false + + + Console + true + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + true + true + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc b/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc new file mode 100644 index 00000000000..79110101f3b --- /dev/null +++ b/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc @@ -0,0 +1,51 @@ +// Microsoft Visual C++ generated resource script. +// + +#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ +#define APSTUDIO_READONLY_SYMBOLS +#include "verrsrc.h" +#undef APSTUDIO_READONLY_SYMBOLS + + +#if !defined(AFX_RESOURCE_DLL) || defined(AFX_TARG_ENU) +LANGUAGE 9, 1 + +///////////////////////////////////////////////////////////////////////////// +// +// Version +// + +VS_VERSION_INFO VERSIONINFO + FILEVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 + PRODUCTVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 + FILEFLAGSMASK VS_FFI_FILEFLAGSMASK +#ifdef _DEBUG + FILEFLAGS VS_FF_DEBUG +#else + FILEFLAGS 0x0L +#endif + FILEOS VOS_NT_WINDOWS32 + FILETYPE VFT_DLL + FILESUBTYPE VFT2_UNKNOWN +BEGIN + BLOCK "StringFileInfo" + BEGIN + BLOCK "040904B0" + BEGIN + VALUE "CompanyName", "Yann Collet, Przemyslaw Skibinski" + VALUE "FileDescription", "Extremely fast compression" + VALUE "FileVersion", LIZARD_VERSION_STRING + VALUE "InternalName", "liblizard.dll" + VALUE "LegalCopyright", "Copyright (C) 2013-2017, Yann Collet, Przemyslaw Skibinski" + VALUE "OriginalFilename", "liblizard.dll" + VALUE "ProductName", "Lizard" + VALUE "ProductVersion", LIZARD_VERSION_STRING + END + END + BLOCK "VarFileInfo" + BEGIN + VALUE "Translation", 0x0409, 1200 + END +END + +#endif diff --git a/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj b/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj new file mode 100644 index 00000000000..09cdaa16180 --- /dev/null +++ b/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj @@ -0,0 +1,179 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {00000000-3E71-41B3-BF68-4A7BDD8A5476} + Win32Proj + liblizard-dll + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + DynamicLibrary + true + Unicode + + + DynamicLibrary + true + Unicode + + + DynamicLibrary + false + true + Unicode + + + DynamicLibrary + false + true + Unicode + + + + + + + + + + + + + + + + + + + true + liblizard + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + liblizard + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + liblizard + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + liblizard + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + true + false + + + true + + + + + + + Level4 + Disabled + WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + true + true + /analyze:stacksize295252 %(AdditionalOptions) + + + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + false + false + + + true + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + false + true + /analyze:stacksize295252 %(AdditionalOptions) + + + true + true + true + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj b/contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj new file mode 100644 index 00000000000..15b74523b96 --- /dev/null +++ b/contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj @@ -0,0 +1,176 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476} + Win32Proj + liblizard + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + StaticLibrary + true + Unicode + + + StaticLibrary + true + Unicode + + + StaticLibrary + false + true + Unicode + + + StaticLibrary + false + true + Unicode + + + + + + + + + + + + + + + + + + + true + liblizard_static + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + liblizard_static + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + liblizard_static + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + liblizard_static + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + true + false + + + true + + + + + + + Level4 + Disabled + WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + true + true + /analyze:stacksize295252 %(AdditionalOptions) + + + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + false + false + + + true + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) + false + true + /analyze:stacksize295252 %(AdditionalOptions) + + + true + true + true + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/contrib/lizard/visual/VS2010/lizard.sln b/contrib/lizard/visual/VS2010/lizard.sln new file mode 100644 index 00000000000..72d68335c1e --- /dev/null +++ b/contrib/lizard/visual/VS2010/lizard.sln @@ -0,0 +1,86 @@ + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio Express 2012 for Windows Desktop +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "lizard", "lizard\lizard.vcxproj", "{E30329AC-0057-4FE0-8FDA-7F650D398C4C}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "liblizard", "liblizard\liblizard.vcxproj", "{9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "liblizard-dll", "liblizard-dll\liblizard-dll.vcxproj", "{00000000-3E71-41B3-BF68-4A7BDD8A5476}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "fuzzer", "fuzzer\fuzzer.vcxproj", "{18B9F1A7-9C66-4352-898B-30804DADE0FD}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "fullbench", "fullbench\fullbench.vcxproj", "{6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "frametest", "frametest\frametest.vcxproj", "{39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "datagen", "datagen\datagen.vcxproj", "{D745AE2F-596A-403A-9B91-81A8C6779243}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Win32 = Debug|Win32 + Debug|x64 = Debug|x64 + Release|Win32 = Release|Win32 + Release|x64 = Release|x64 + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|Win32.ActiveCfg = Debug|Win32 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|Win32.Build.0 = Debug|Win32 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|x64.ActiveCfg = Debug|x64 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|x64.Build.0 = Debug|x64 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|Win32.ActiveCfg = Release|Win32 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|Win32.Build.0 = Release|Win32 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|x64.ActiveCfg = Release|x64 + {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|x64.Build.0 = Release|x64 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.ActiveCfg = Debug|Win32 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.Build.0 = Debug|Win32 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.ActiveCfg = Debug|x64 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.Build.0 = Debug|x64 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.ActiveCfg = Release|Win32 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.Build.0 = Release|Win32 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.ActiveCfg = Release|x64 + {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.Build.0 = Release|x64 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.ActiveCfg = Debug|Win32 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.Build.0 = Debug|Win32 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.ActiveCfg = Debug|x64 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.Build.0 = Debug|x64 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.ActiveCfg = Release|Win32 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.Build.0 = Release|Win32 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.ActiveCfg = Release|x64 + {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.Build.0 = Release|x64 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|Win32.ActiveCfg = Debug|Win32 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|Win32.Build.0 = Debug|Win32 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|x64.ActiveCfg = Debug|x64 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|x64.Build.0 = Debug|x64 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|Win32.ActiveCfg = Release|Win32 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|Win32.Build.0 = Release|Win32 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|x64.ActiveCfg = Release|x64 + {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|x64.Build.0 = Release|x64 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|Win32.ActiveCfg = Debug|Win32 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|Win32.Build.0 = Debug|Win32 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|x64.ActiveCfg = Debug|x64 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|x64.Build.0 = Debug|x64 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|Win32.ActiveCfg = Release|Win32 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|Win32.Build.0 = Release|Win32 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|x64.ActiveCfg = Release|x64 + {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|x64.Build.0 = Release|x64 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|Win32.ActiveCfg = Debug|Win32 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|Win32.Build.0 = Debug|Win32 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|x64.ActiveCfg = Debug|x64 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|x64.Build.0 = Debug|x64 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|Win32.ActiveCfg = Release|Win32 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|Win32.Build.0 = Release|Win32 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|x64.ActiveCfg = Release|x64 + {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|x64.Build.0 = Release|x64 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|Win32.ActiveCfg = Debug|Win32 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|Win32.Build.0 = Debug|Win32 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|x64.ActiveCfg = Debug|x64 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|x64.Build.0 = Debug|x64 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|Win32.ActiveCfg = Release|Win32 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|Win32.Build.0 = Release|Win32 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|x64.ActiveCfg = Release|x64 + {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|x64.Build.0 = Release|x64 + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection +EndGlobal diff --git a/contrib/lizard/visual/VS2010/lizard/lizard.rc b/contrib/lizard/visual/VS2010/lizard/lizard.rc new file mode 100644 index 00000000000..76c3c7f947e --- /dev/null +++ b/contrib/lizard/visual/VS2010/lizard/lizard.rc @@ -0,0 +1,51 @@ +// Microsoft Visual C++ generated resource script. +// + +#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ +#define APSTUDIO_READONLY_SYMBOLS +#include "verrsrc.h" +#undef APSTUDIO_READONLY_SYMBOLS + + +#if !defined(AFX_RESOURCE_DLL) || defined(AFX_TARG_ENU) +LANGUAGE 9, 1 + +///////////////////////////////////////////////////////////////////////////// +// +// Version +// + +VS_VERSION_INFO VERSIONINFO + FILEVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 + PRODUCTVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 + FILEFLAGSMASK VS_FFI_FILEFLAGSMASK +#ifdef _DEBUG + FILEFLAGS VS_FF_DEBUG +#else + FILEFLAGS 0x0L +#endif + FILEOS VOS_NT_WINDOWS32 + FILETYPE VFT_DLL + FILESUBTYPE VFT2_UNKNOWN +BEGIN + BLOCK "StringFileInfo" + BEGIN + BLOCK "040904B0" + BEGIN + VALUE "CompanyName", "Yann Collet, Przemyslaw Skibinski" + VALUE "FileDescription", "Extremely fast compression" + VALUE "FileVersion", LIZARD_VERSION_STRING + VALUE "InternalName", "lizard.exe" + VALUE "LegalCopyright", "Copyright (C) 2013-2017, Yann Collet, Przemyslaw Skibinski" + VALUE "OriginalFilename", "lizard.exe" + VALUE "ProductName", "Lizard" + VALUE "ProductVersion", LIZARD_VERSION_STRING + END + END + BLOCK "VarFileInfo" + BEGIN + VALUE "Translation", 0x0409, 1200 + END +END + +#endif diff --git a/contrib/lizard/visual/VS2010/lizard/lizard.vcxproj b/contrib/lizard/visual/VS2010/lizard/lizard.vcxproj new file mode 100644 index 00000000000..98be4d599db --- /dev/null +++ b/contrib/lizard/visual/VS2010/lizard/lizard.vcxproj @@ -0,0 +1,190 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {E30329AC-0057-4FE0-8FDA-7F650D398C4C} + Win32Proj + lizard + $(SolutionDir)bin\$(Platform)_$(Configuration)\ + $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ + + + + Application + true + MultiByte + + + Application + true + MultiByte + + + Application + false + true + MultiByte + + + Application + false + true + MultiByte + + + + + + + + + + + + + + + + + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + true + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + + + false + $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); + true + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + false + + + Console + true + setargv.obj;%(AdditionalDependencies) + + + + + + + Level4 + Disabled + WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) + true + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + setargv.obj;%(AdditionalDependencies) + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + false + + + Console + true + setargv.obj;%(AdditionalDependencies) + true + true + + + + + Level4 + + + MaxSpeed + true + true + WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) + false + true + /analyze:stacksize295252 %(AdditionalOptions) + + + Console + true + setargv.obj;%(AdditionalDependencies) + true + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/contrib/lzsse/.gitignore b/contrib/lzsse/.gitignore new file mode 100644 index 00000000000..379a8f7f570 --- /dev/null +++ b/contrib/lzsse/.gitignore @@ -0,0 +1,4 @@ +bin/* +gmake/* +vs2013/* +vs2015/* \ No newline at end of file diff --git a/contrib/lzsse/.travis.yml b/contrib/lzsse/.travis.yml new file mode 100644 index 00000000000..68bb59ef0b9 --- /dev/null +++ b/contrib/lzsse/.travis.yml @@ -0,0 +1,185 @@ +language: c +sudo: false +branches: + except: + - /^(wip\/)?(appveyor|msvc|mingw|windows)(\-.+)?$/ +matrix: + include: + ### + ## Linux builds using various versions of GCC. + ### + - env: C_COMPILER=gcc-7 + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - premake4 + packages: + - gcc-7 + - g++-7 + - premake4 + # - env: C_COMPILER=gcc-6 + # addons: + # apt: + # sources: + # - ubuntu-toolchain-r-test + # - premake4 + # packages: + # - gcc-6 + # - g++-6 + # - premake4 + - env: C_COMPILER=gcc-5 + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - premake4 + packages: + - gcc-5 + - g++-5 + - premake4 + # - env: C_COMPILER=gcc-4.9 + # addons: + # apt: + # sources: + # - ubuntu-toolchain-r-test + # - premake4 + # packages: + # - gcc-4.9 + # - g++-4.9 + # - premake4 + - env: C_COMPILER=gcc-4.8 + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - premake4 + packages: + - gcc-4.8 + - g++-4.8 + - premake4 + # - env: C_COMPILER=gcc-4.7 + # addons: + # apt: + # sources: + # - ubuntu-toolchain-r-test + # - premake4 + # packages: + # - gcc-4.7 + # - g++-4.7 + # - premake4 + - env: C_COMPILER=gcc-4.6 + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - premake4 + packages: + - gcc-4.6 + - g++-4.6 + - premake4 + # - os: linux + # env: C_COMPILER=gcc-4.5 + # addons: + # apt: + # sources: + # - ubuntu-toolchain-r-test + # - premake4 + # packages: + # - gcc-4.5 + # - g++-4.5 + # - premake4 + - env: C_COMPILER=gcc-4.4 + addons: + apt: + sources: + - ubuntu-toolchain-r-test + - premake4 + packages: + - gcc-4.4 + - g++-4.4 + - premake4 + + ### + ## clang on Linux + ### + - env: C_COMPILER=clang-3.9 + addons: + apt: + sources: + - llvm-toolchain-precise-3.9 + - ubuntu-toolchain-r-test + - premake4 + packages: + - clang-3.9 + - premake4 + # - env: C_COMPILER=clang-3.8 + # addons: + # apt: + # sources: + # - llvm-toolchain-precise-3.8 + # - ubuntu-toolchain-r-test + # - premake4 + # packages: + # - clang-3.8 + # - premake4 + - env: C_COMPILER=clang-3.7 + addons: + apt: + sources: + - llvm-toolchain-precise-3.7 + - ubuntu-toolchain-r-test + - premake4 + packages: + - clang-3.7 + - premake4 + # - env: C_COMPILER=clang-3.6 + # addons: + # apt: + # sources: + # - llvm-toolchain-precise-3.6 + # - ubuntu-toolchain-r-test + # - premake4 + # packages: + # - clang-3.6 + # - premake4 + - env: C_COMPILER=clang-3.5 + addons: + apt: + sources: + - llvm-toolchain-precise-3.5 + - ubuntu-toolchain-r-test + - premake4 + packages: + - clang-3.5 + - premake4 + + ### + ## PGI + ### + - env: C_COMPILER=pgcc ENABLE_OPENMP=y + addons: + apt: + sources: + - premake4 + packages: + - premake4 + + ### + ## OS X + ### + # - os: osx + +before_install: +### +## If we use the matrix to set CC/CXX Travis, overwrites the values, +## so instead we use C/CXX_COMPILER, then copy the values to CC/CXX +## here (after Travis has set CC/CXX). +### +- if [ "${C_COMPILER}" = "pgcc" ]; then wget -q -O /dev/stdout 'https://raw.githubusercontent.com/nemequ/pgi-travis/master/install-pgi.sh' | /bin/sh; fi +- if [ -n "${C_COMPILER}" ]; then export CC="${C_COMPILER}"; fi +- premake4 gmake + +script: +- cd gmake && make + diff --git a/contrib/lzsse/LICENSE b/contrib/lzsse/LICENSE new file mode 100644 index 00000000000..d13dc62210f --- /dev/null +++ b/contrib/lzsse/LICENSE @@ -0,0 +1,23 @@ +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/lzsse/README.md b/contrib/lzsse/README.md new file mode 100644 index 00000000000..6b777eecb52 --- /dev/null +++ b/contrib/lzsse/README.md @@ -0,0 +1,15 @@ +# LZSSE +[LZSS](https://en.wikipedia.org/wiki/Lempel%E2%80%93Ziv%E2%80%93Storer%E2%80%93Szymanski) designed for a branchless SSE decompression implementation. + +Three variants: +- LZSSE2, for high compression files with small literal runs. +- LZSSE4, for a more balanced mix of literals and matches. +- LZSSE8, for lower compression data with longer runs of matches. + +All three variants have an optimal parser implementation, which uses a quite strong match finder (very similar to LzFind) combined with a Storer-Szymanski style parse. LZSSE4 and LZSSE8 have "fast" compressor implementations, which use a simple hash table based matching and a greedy parse. + +Currently LZSSE8 is the recommended variant to use in the general case, as it generally performs well in most cases (and you have the option of both optimal parse and fast compression). LZSSE2 is recommended if you are only using text, especially heavily compressible text, but is slow/doesn't compress as well on less compressible data and binaries. + +The code is approaching production readiness and LZSSE2 and LZSSE8 have received a reasonable amount of testing. + +See these blog posts [An LZ Codec Designed for SSE Decompression](http://conorstokes.github.io/compression/2016/02/15/an-LZ-codec-designed-for-SSE-decompression) and [Compressor Improvements and LZSSE2 vs LZSSE8](http://conorstokes.github.io/compression/2016/02/24/compressor-improvements-and-lzsse2-vs-lzsse8) for a description of how the compression algorithm and implementation function. There are also benchmarks, but these may not be upto date (in particular the figures in the initial blog post no longer represent compression performance). diff --git a/contrib/lzsse/example/main.cpp b/contrib/lzsse/example/main.cpp new file mode 100644 index 00000000000..9287f683c7d --- /dev/null +++ b/contrib/lzsse/example/main.cpp @@ -0,0 +1,389 @@ +// LZSSE.cpp : Defines the entry point for the console application. +// +#define _CRT_SECURE_NO_WARNINGS 1 + +#include +#include +#include +#include +#include "../lzsse2/lzsse2.h" +#include "../lzsse4/lzsse4.h" +#include "../lzsse8/lzsse8.h" + +static const uint32_t MAGIC_NUMBER = 0x28F19732; + +void DisplayUsage() +{ + printf( "Usage:\n" ); + printf( " lzsse [args] input_file output_file\n" ); + printf( "\n" ); + printf( "Arguments:\n" ); + printf( " -2 Compress in lzsse2 mode (default)\n" ); + printf( " -4 Compress in lzsse4 mode\n" ); + printf( " -8 Compress in lzsse8 mode\n" ); + printf( " -f Optimal parse (default)\n" ); + printf( " -o Fast parse (not available for lzsse2)\n" ); + printf( " -d Decompress\n" ); + printf( " -lN Compression level for optimal parse, where N is 1 to 17 (default 16)\n" ); + printf( " -bN Block size in KiB, default 131,072\n" ); + printf( "\n" ); +} + +static size_t CompressorFastLZSSE4( LZSSE4_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int ) +{ + return LZSSE4_CompressFast( state, input, inputLength, output, outputLength ); +} + +static size_t CompressorFastLZSSE8( LZSSE8_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int ) +{ + return LZSSE8_CompressFast( state, input, inputLength, output, outputLength ); +} + +template +void Compress( FILE* inputFile, FILE* outputFile, uint64_t blockSize, uint8_t mode, unsigned int level, State* state, size_t (*compressor)( State*, const void*, size_t, void*, size_t, unsigned int ) ) +{ + if ( state == nullptr ) + { + printf( "Couldn't allocate parse state\n" ); + exit( 1 ); + } + + if ( fwrite( &MAGIC_NUMBER, sizeof( uint32_t ), 1, outputFile ) == 0 ) + { + printf( "Couldn't write magic number\n" ); + exit( 1 ); + } + + if ( fwrite( &mode, sizeof( uint8_t ), 1, outputFile ) == 0 ) + { + printf( "Couldn't write stream type\n" ); + exit( 1 ); + } + + if ( fwrite( &blockSize, sizeof( uint64_t ), 1, outputFile ) == 0 ) + { + printf( "Couldn't write block size\n" ); + exit( 1 ); + } + + size_t typedBlockSize = static_cast< size_t >( blockSize ); + uint8_t* inputBuffer = reinterpret_cast< uint8_t* >( malloc( typedBlockSize ) ); + uint8_t* outputBuffer = reinterpret_cast< uint8_t* >( malloc( typedBlockSize ) ); + + if ( inputBuffer == nullptr || outputBuffer == nullptr ) + { + printf( "Couldn't allocate buffer memory\n" ); + exit( 1 ); + } + + for ( ;; ) + { + size_t readSize = fread( inputBuffer, 1, blockSize, inputFile ); + + if ( readSize == 0 ) + { + break; + } + + size_t compressedSize = compressor( state, inputBuffer, readSize, outputBuffer, typedBlockSize, level ); + + if ( compressedSize == 0 ) + { + printf( "Compression function failed\n" ); + exit( 1 ); + } + + uint32_t compressedLength = static_cast< uint32_t >( compressedSize ); + uint32_t uncompressedLength = static_cast< uint32_t >( readSize ); + + if ( fwrite( &uncompressedLength, sizeof( uint32_t ), 1, outputFile ) < 1 ) + { + break; + } + + if ( fwrite( &compressedLength, sizeof( uint32_t ), 1, outputFile ) < 1 ) + { + printf( "Error writing compressed length from block\n" ); + exit( 1 ); + } + + if ( fwrite( outputBuffer, 1, compressedLength, outputFile ) != compressedLength ) + { + printf( "Error writing block\n" ); + exit( 1 ); + } + } + + free( inputBuffer ); + free( outputBuffer ); +} + +void Decompress( FILE* inputFile, FILE* outputFile ) +{ + uint32_t magicNumber; + uint64_t blockSize = 128 * 1024 * 1024; + + if ( fread( &magicNumber, sizeof( uint32_t ), 1, inputFile ) < 1 || magicNumber != MAGIC_NUMBER ) + { + printf( "Couldn't read magic number, or magic number incorrect\n" ); + exit( 1 ); + } + + uint8_t streamType; + + if ( fread( &streamType, sizeof( uint8_t ), 1, inputFile ) < 1 ) + { + printf( "Couldn't read stream type\n" ); + exit( 1 ); + } + + if ( fread( &blockSize, sizeof( uint64_t ), 1, inputFile ) < 1 ) + { + printf( "Couldn't read block size\n" ); + exit( 1 ); + } + + uint8_t* inputBuffer = reinterpret_cast< uint8_t* >( malloc( static_cast< size_t >( blockSize ) ) ); + uint8_t* outputBuffer = reinterpret_cast< uint8_t* >( malloc( static_cast< size_t >( blockSize ) ) ); + + if ( inputBuffer == nullptr || outputBuffer == nullptr ) + { + printf( "Couldn't allocate buffer memory\n" ); + exit( 1 ); + } + + size_t( *decompressor )( const void*, size_t, void*, size_t ); + + switch ( streamType ) + { + case 2: + + decompressor = LZSSE2_Decompress; + break; + + case 4: + + decompressor = LZSSE4_Decompress; + break; + + case 8: + + decompressor = LZSSE8_Decompress; + break; + + default: + + printf( "Invalid stream type\n" ); + exit( 1 ); + + } + + memset( inputBuffer, 0, blockSize ); + memset( outputBuffer, 0, blockSize ); + + for ( ;; ) + { + uint32_t compressedLength; + uint32_t uncompressedLength; + + if ( fread( &uncompressedLength, sizeof( uint32_t ), 1, inputFile ) < 1 ) + { + break; + } + + if ( fread( &compressedLength, sizeof( uint32_t ), 1, inputFile ) < 1 ) + { + printf( "Error reading compressed length from block\n" ); + exit( 1 ); + } + + if ( fread( inputBuffer, 1, compressedLength, inputFile ) != compressedLength ) + { + printf( "Error reading block\n" ); + exit( 1 ); + } + + size_t decompressedSize = 0; + + decompressedSize = + decompressor( inputBuffer, + compressedLength, + outputBuffer, + uncompressedLength ); + + if ( decompressedSize != size_t( uncompressedLength ) ) + { + printf( "Error in decompression stream\n" ); + exit( 1 ); + } + + if ( fwrite( outputBuffer, 1, uncompressedLength, outputFile ) != uncompressedLength ) + { + printf( "Couldn't write block to output file\n" ); + exit( 1 ); + } + } + + free( inputBuffer ); + free( outputBuffer ); +} + +int main( int argc, const char** argv ) +{ + bool decompression = false; + bool optimal = true; + uint64_t blockSize = 128 * 1024 * 1024; + uint8_t mode = 2; + unsigned int level = 16; + + if ( argc < 3 ) + { + DisplayUsage(); + exit( 1 ); + } + + for ( int argIndex = 1; argIndex < argc - 2; ++argIndex ) + { + const char* arg = argv[ argIndex ]; + + if ( arg[ 0 ] == '-' ) + { + switch ( arg[ 1 ] ) + { + case 'd': + + decompression = true; + break; + + case '2': + + mode = 2; + break; + + case '4': + + mode = 4; + break; + + case '8': + + mode = 8; + break; + + case 'l': + + level = static_cast< unsigned int >( strtoul( arg + 2, nullptr, 10 ) ); + break; + + case 'b': + + blockSize = strtoull( arg + 2, nullptr, 10 ) * 1024; + break; + + case 'o': + + optimal = true; + break; + + case 'f': + + optimal = false; + break; + + } + } + } + + FILE* inputFile = fopen( argv[ argc - 2 ], "rb" ); + + if ( inputFile == nullptr ) + { + perror( argv[ argc - 2 ] ); + exit( 1 ); + } + + FILE* outputFile = fopen( argv[ argc - 1 ], "wb+" ); + + if ( outputFile == nullptr ) + { + perror( argv[ argc - 2 ] ); + exit( 1 ); + } + + if ( decompression ) + { + Decompress( inputFile, outputFile ); + } + else + { + switch ( mode ) + { + case 2: + { + LZSSE2_OptimalParseState* state = LZSSE2_MakeOptimalParseState( static_cast< size_t >( blockSize ) ); + + Compress( inputFile, outputFile, blockSize, mode, level, state, LZSSE2_CompressOptimalParse ); + + LZSSE2_FreeOptimalParseState( state ); + + break; + } + + case 4: + { + if ( optimal ) + { + LZSSE4_OptimalParseState* state = LZSSE4_MakeOptimalParseState( static_cast( blockSize ) ); + + Compress( inputFile, outputFile, blockSize, mode, level, state, LZSSE4_CompressOptimalParse ); + + LZSSE4_FreeOptimalParseState( state ); + } + else + { + LZSSE4_FastParseState* state = LZSSE4_MakeFastParseState(); + + Compress( inputFile, outputFile, blockSize, mode, level, state, CompressorFastLZSSE4 ); + + LZSSE4_FreeFastParseState( state ); + } + + break; + } + + case 8: + { + if ( optimal ) + { + LZSSE8_OptimalParseState* state = LZSSE8_MakeOptimalParseState( static_cast( blockSize ) ); + + Compress( inputFile, outputFile, blockSize, mode, level, state, LZSSE8_CompressOptimalParse ); + + LZSSE8_FreeOptimalParseState( state ); + } + else + { + LZSSE8_FastParseState* state = LZSSE8_MakeFastParseState(); + + Compress( inputFile, outputFile, blockSize, mode, level, state, CompressorFastLZSSE8 ); + + LZSSE8_FreeFastParseState( state ); + } + + break; + } + + default: + + printf( "Invalid stream type\n" ); + exit( 1 ); + + } + } + + fclose( inputFile ); + fclose( outputFile ); + + return 0; +} + diff --git a/contrib/lzsse/lzsse2/lzsse2.cpp b/contrib/lzsse/lzsse2/lzsse2.cpp new file mode 100644 index 00000000000..d521a84553a --- /dev/null +++ b/contrib/lzsse/lzsse2/lzsse2.cpp @@ -0,0 +1,1080 @@ +/* +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#include +#include +#include +#include + +#include "lzsse2_platform.h" +#include "lzsse2.h" + +#pragma warning ( disable : 4127 ) + +namespace +{ + // Constants - most of these should not be changed without corresponding code changes because it will break many things in unpredictable ways. + const uint32_t WINDOW_BITS = 16; + const uint32_t MIN_MATCH_LENGTH = 3; + const uint32_t LZ_WINDOW_SIZE = 1 << WINDOW_BITS; + const uint32_t LZ_WINDOW_MASK = LZ_WINDOW_SIZE - 1; + const uint32_t OPTIMAL_HASH_BITS = 20; + const uint32_t OPTIMAL_BUCKETS_COUNT = 1 << OPTIMAL_HASH_BITS; + const uint32_t OPTIMAL_HASH_MASK = OPTIMAL_BUCKETS_COUNT - 1; + const uint32_t MIN_COMPRESSION_SIZE = 32; + const uint32_t END_PADDING_LITERALS = 16; + const int32_t NO_MATCH = -1; + const int32_t EMPTY_NODE = -1; + const uint32_t MIN_LITERAL_COUNT = 8; + const uint32_t HASH_MULTIPLIER = 4013; + const uint32_t HASH_REMOVAL_MULTIPLIER = HASH_MULTIPLIER * HASH_MULTIPLIER; + const uint32_t CONTROL_BITS = 4; + const uint32_t LITERAL_BITS = 8; + const uint32_t OFFSET_BITS = 16; + const uint32_t BASE_MATCH_BITS = OFFSET_BITS + CONTROL_BITS; + const uint32_t SINGLE_LITERAL_COST = CONTROL_BITS + LITERAL_BITS; + const uint32_t DOUBLE_LITERAL_COST = SINGLE_LITERAL_COST + LITERAL_BITS; + const uint32_t EXTENDED_MATCH_BOUND = ( 1 << CONTROL_BITS ) - 1; + const uint32_t CONTROL_BLOCK_SIZE = sizeof( __m128i ); + const uint32_t CONTROLS_PER_BLOCK = 32; + const uint32_t LITERALS_PER_CONTROL = 2; + const uint32_t MAX_INPUT_PER_CONTROL = 2; + const size_t OUTPUT_BUFFER_SAFE = EXTENDED_MATCH_BOUND * CONTROLS_PER_BLOCK; + const size_t INPUT_BUFFER_SAFE = MAX_INPUT_PER_CONTROL * CONTROLS_PER_BLOCK; + const uint16_t INITIAL_OFFSET = MIN_MATCH_LENGTH; + const size_t SKIP_MATCH_LENGTH = 128; + const uint32_t NO_SKIP_LEVEL = 17; +} + +struct Arrival +{ + size_t cost; + int32_t from; + int32_t to; + uint16_t offset; +}; + +struct TreeNode +{ + int32_t children[ 2 ]; +}; + +struct LZSSE2_OptimalParseState +{ + // Note, we should really replace this with a BST, hash chaining works but is *slooooooooooooooow* for optimal parse. + int32_t roots[ OPTIMAL_BUCKETS_COUNT ]; + + TreeNode window[ LZ_WINDOW_SIZE ]; + + Arrival* arrivals; + + size_t bufferSize; +}; + + +LZSSE2_OptimalParseState* LZSSE2_MakeOptimalParseState( size_t bufferSize ) +{ + if ( bufferSize > 0 && ( SIZE_MAX / sizeof( Arrival ) ) < bufferSize ) + { + return nullptr; + } + + LZSSE2_OptimalParseState* result = reinterpret_cast< LZSSE2_OptimalParseState* >( ::malloc( sizeof( LZSSE2_OptimalParseState ) ) ); + + result->bufferSize = bufferSize; + + if ( result != nullptr ) + { + result->arrivals = reinterpret_cast< Arrival* >( ::malloc( sizeof( Arrival ) * bufferSize ) ); + + if ( result->arrivals == nullptr ) + { + LZSSE2_FreeOptimalParseState( result ); + + result = nullptr; + } + } + + return result; +} + + +void LZSSE2_FreeOptimalParseState( LZSSE2_OptimalParseState* toFree ) +{ + ::free( toFree->arrivals ); + + toFree->arrivals = nullptr; + + ::free( toFree ); +} + + +inline uint32_t CalculateHash( const uint8_t* inputCursor ) +{ + return ( uint32_t( inputCursor[ 0 ] ) * HASH_MULTIPLIER * HASH_MULTIPLIER + uint32_t( inputCursor[ 1 ] ) * HASH_MULTIPLIER + uint32_t( inputCursor[ 2 ] ) ) & OPTIMAL_HASH_MASK; +} + + +struct Match +{ + size_t length; + int32_t position; + uint16_t offset; +}; + + +inline Match SearchAndUpdateFinder( LZSSE2_OptimalParseState& state, const uint8_t* input, const uint8_t* inputCursor, const uint8_t* inputEnd, uint32_t cutOff ) +{ + Match result; + + int32_t position = static_cast( inputCursor - input ); + + result.position = NO_MATCH; + result.length = MIN_MATCH_LENGTH; + result.offset = 0; + + size_t lengthToEnd = inputEnd - inputCursor; + int32_t lastPosition = position - ( LZ_WINDOW_SIZE - 1 ); + uint32_t hash = CalculateHash( inputCursor ); + + lastPosition = lastPosition > 0 ? lastPosition : 0; + + int32_t treeCursor = state.roots[ hash ]; + + state.roots[ hash ] = position; + + int32_t* left = &state.window[ position & LZ_WINDOW_MASK ].children[ 1 ]; + int32_t* right = &state.window[ position & LZ_WINDOW_MASK ].children[ 0 ]; + size_t leftLength = 0; + size_t rightLength = 0; + + for ( ;; ) + { + if ( cutOff-- == 0 || treeCursor < lastPosition ) + { + *left = *right = EMPTY_NODE; + break; + } + + TreeNode& currentNode = state.window[ treeCursor & LZ_WINDOW_MASK ]; + const uint8_t* key = input + treeCursor; + size_t matchLength = leftLength < rightLength ? leftLength : rightLength; + + uint16_t matchOffset = static_cast( position - treeCursor ); + size_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; + + while ( matchLength < lengthToEnd ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); + + unsigned long matchBytes; + + _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != 16 ) + { + break; + } + } + + matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; + + size_t truncatedMatchLength = matchLength < maxLength ? matchLength : maxLength; + + if ( truncatedMatchLength >= result.length ) + { + result.length = truncatedMatchLength; + result.offset = matchOffset; + result.position = treeCursor; + } + + if ( matchLength == lengthToEnd ) + { + *left = currentNode.children[ 1 ]; + *right = currentNode.children[ 0 ]; + break; + } + + if ( inputCursor[ matchLength ] < key[ matchLength ] || ( matchLength == lengthToEnd ) ) + { + *left = treeCursor; + left = currentNode.children; + treeCursor = *left; + leftLength = matchLength; + } + else + { + *right = treeCursor; + right = currentNode.children + 1; + treeCursor = *right; + rightLength = matchLength; + } + } + + // Special RLE overlapping match case, the LzFind style match above doesn't work very well with our + // restriction of overlapping matches having offsets of at least 16. + // Suffix array seems like a better option to handling this. + { + // Note, we're detecting long RLE here, but if we have an offset too close, we'll sacrifice a fair + // amount of decompression performance to load-hit-stores. + int32_t matchPosition = position - ( sizeof( __m128i ) * 2 ); + + if ( matchPosition >= 0 ) + { + uint16_t matchOffset = static_cast( position - matchPosition ); + const uint8_t* key = input + matchPosition; + size_t matchLength = 0; + + while ( matchLength < lengthToEnd ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); + + unsigned long matchBytes; + + _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != 16 ) + { + break; + } + + } + + matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; + + if ( matchLength >= result.length ) + { + result.length = matchLength; + result.offset = matchOffset; + result.position = matchPosition; + } + } + } + + return result; +} + + +size_t LZSSE2_CompressOptimalParse( LZSSE2_OptimalParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength, unsigned int level ) +{ + if ( outputLength < inputLength || state->bufferSize < inputLength ) + { + // error case, output buffer not large enough. + return 0; + } + + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + if ( inputLength < MIN_COMPRESSION_SIZE ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + const uint8_t* inputEnd = input + inputLength; + Arrival* arrivalWatermark = state->arrivals; + Arrival* arrival = state->arrivals; + uint32_t cutOff = 1 << level; + + for ( int32_t* rootCursor = state->roots, *end = rootCursor + OPTIMAL_BUCKETS_COUNT; rootCursor < end; rootCursor += 4 ) + { + rootCursor[ 0 ] = EMPTY_NODE; + rootCursor[ 1 ] = EMPTY_NODE; + rootCursor[ 2 ] = EMPTY_NODE; + rootCursor[ 3 ] = EMPTY_NODE; + } + + for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) + { + /*Match dummy = */ SearchAndUpdateFinder( *state, input, inputCursor, inputEnd - END_PADDING_LITERALS, cutOff ); + + ++inputCursor; + } + + arrival->cost = LITERAL_BITS * MIN_MATCH_LENGTH; + arrival->from = -1; + arrival->offset = 0; + + // loop through each character and project forward the matches at that character to calculate the cheapest + // path of arrival for each individual character. + for ( const uint8_t* earlyEnd = inputEnd - END_PADDING_LITERALS; inputCursor < earlyEnd; ++inputCursor, ++arrival ) + { + uint32_t lengthToEnd = static_cast< uint32_t >( earlyEnd - inputCursor ); + int32_t currentPosition = static_cast< int32_t >( inputCursor - input ); + Arrival* literalFirst = arrival + 1; + Arrival* literalSecond = arrival + 2; + size_t arrivalCost = arrival->cost; + + // NOTE - we currently assume only 2 literals filled in here, because the minimum match length is 3. + // If we wanted to go with a higher minimum match length, we would need to fill in more literals before hand. + // Also, because there is a maximum of 2 literals per control block assumed. + + // project forward the cost of a single literal + if ( literalFirst > arrivalWatermark || literalFirst->cost > ( arrival->cost + SINGLE_LITERAL_COST ) ) + { + literalFirst->cost = arrival->cost + SINGLE_LITERAL_COST; + literalFirst->from = currentPosition; + literalFirst->offset = 0; + + arrivalWatermark = literalFirst > arrivalWatermark ? literalFirst : arrivalWatermark; + } + + // project forward the cost of two literals + if ( lengthToEnd > 1 ) + { + if ( literalSecond > arrivalWatermark || literalFirst->cost > ( arrival->cost + DOUBLE_LITERAL_COST ) ) + { + literalSecond->cost = arrival->cost + DOUBLE_LITERAL_COST; + literalSecond->from = currentPosition; + literalSecond->offset = 0; + + arrivalWatermark = literalSecond > arrivalWatermark ? literalSecond : arrivalWatermark; + } + } + else + { + continue; + } + + Match match = SearchAndUpdateFinder( *state, input, inputCursor, earlyEnd, cutOff ); + + if ( match.position != NO_MATCH ) + { + for ( size_t matchedLength = MIN_MATCH_LENGTH, end = match.length + 1; matchedLength < end; ++matchedLength ) + { + Arrival* matchArrival = arrival + matchedLength; + size_t matchCost = arrivalCost + BASE_MATCH_BITS; + + if ( matchedLength > EXTENDED_MATCH_BOUND ) + { + matchCost += ( ( matchedLength - 1 ) / EXTENDED_MATCH_BOUND ) * CONTROL_BITS; + } + + if ( matchArrival > arrivalWatermark || matchArrival->cost > matchCost ) + { + matchArrival->cost = matchCost; + matchArrival->from = currentPosition; + matchArrival->offset = match.offset; + + arrivalWatermark = matchArrival > arrivalWatermark ? matchArrival : arrivalWatermark; + } + } + + if ( match.length > SKIP_MATCH_LENGTH && level < NO_SKIP_LEVEL ) + { + arrival += match.length - LITERALS_PER_CONTROL; + inputCursor += match.length - LITERALS_PER_CONTROL; + } + } + } + + // If this would cost more to encode than it would if it were just literals, encode it with no control blocks, + // just literals + if ( ( arrivalWatermark->cost + END_PADDING_LITERALS * LITERAL_BITS + CONTROLS_PER_BLOCK * CONTROL_BITS ) > ( inputLength * LITERAL_BITS ) ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + Arrival* previousPathNode; + + // now trace the actual optimal parse path back, connecting the nodes in the other direction. + for ( const Arrival* pathNode = arrivalWatermark; pathNode->from > 0; pathNode = previousPathNode ) + { + previousPathNode = state->arrivals + ( pathNode->from - MIN_MATCH_LENGTH ); + + previousPathNode->to = static_cast( ( pathNode - state->arrivals ) + MIN_MATCH_LENGTH ); + } + + uint8_t* outputCursor = output; + + memcpy( outputCursor, input, MIN_MATCH_LENGTH ); + + outputCursor += MIN_MATCH_LENGTH; + + uint8_t* currentControlBlock = outputCursor; + uint32_t currentControlCount = 0; + uint32_t totalControlCount = 0; + + outputCursor += CONTROL_BLOCK_SIZE; + + Arrival* nextPathNode; + + size_t totalPathLength = MIN_MATCH_LENGTH; + uint16_t previousOffset = INITIAL_OFFSET; + + bool lastControlIsNop = false; + + // Now walk forwards again and actually write out the data. + for ( const Arrival* pathNode = state->arrivals; pathNode < arrivalWatermark; pathNode = nextPathNode ) + { + int32_t currentPosition = static_cast< int32_t >( ( pathNode - state->arrivals ) + MIN_MATCH_LENGTH ); + + nextPathNode = state->arrivals + ( pathNode->to - MIN_MATCH_LENGTH ); + + size_t pathDistance = nextPathNode - pathNode; + + totalPathLength += pathDistance; + + lastControlIsNop = false; + + if ( pathDistance < MIN_MATCH_LENGTH ) + { + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + currentControlCount = 0; + } + + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + ( static_cast( pathDistance ) - 1 ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + ( static_cast< uint8_t >( pathDistance ) - 1 ) << CONTROL_BITS; + } + + // output the literals. + for ( int32_t where = 0; where < pathDistance; ++where ) + { + const uint8_t* currentInput = input + currentPosition + where; + + outputCursor[ where ] = *currentInput ^ *( currentInput - previousOffset ); + } + + outputCursor += pathDistance; + + ++totalControlCount; + ++currentControlCount; + } + else + { + size_t toEncode = pathDistance - 1; //note, we always subtract one here, because the first control block of the match encodes this way + + // make sure the control block for the first part of the match has been allocated + // note, this is idempontent if we have not actually incremented the control count and we try this again. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + currentControlCount = 0; + } + + // output the offset (after control block containing the first control nibble for this match). + *reinterpret_cast< uint16_t* >( outputCursor ) = nextPathNode->offset ^ previousOffset; + + previousOffset = nextPathNode->offset; + + outputCursor += sizeof( uint16_t ); + + for ( ;; ) + { + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + currentControlCount = 0; + } + + if ( toEncode >= EXTENDED_MATCH_BOUND ) + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( EXTENDED_MATCH_BOUND ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; + } + + toEncode -= EXTENDED_MATCH_BOUND; + + ++totalControlCount; + ++currentControlCount; + + } + else + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( toEncode ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( toEncode ) << CONTROL_BITS; + } + + if ( toEncode == 0 && currentControlCount == 0 ) + { + lastControlIsNop = true; + } + + ++totalControlCount; + ++currentControlCount; + + break; + } + } + } + } + + if ( lastControlIsNop ) + { + outputCursor -= CONTROL_BLOCK_SIZE; + } + + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy remaining literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + + return outputCursor - output; +} + + +size_t LZSSE2_Decompress( const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) +{ + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + // Length it not work compressing, just copy initial values + if ( outputLength == inputLength ) + { + memcpy( output, input, outputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + uint8_t* outputCursor = output; + + // The offset starts off as the minimum match length. We actually need it at least two + // characters back because we need them to be set to xor out the literals from the match data. + size_t offset = INITIAL_OFFSET; + __m128i previousCarryHi = _mm_setzero_si128(); + + *( outputCursor++ ) = *( inputCursor++ ); + *( outputCursor++ ) = *( inputCursor++ ); + *( outputCursor++ ) = *( inputCursor++ ); + + // What these macros do: + // Decode a single literal run or match run for a single control nibble. + // How they do it: + // - Read the *unaligned* input (in the case of LZSSE-F - twice, for LZSSE-O we read once) - one goes into an SSE register, + // because it could either be literals or an offset (or nothing at all). The low byte of streamBytesRead controls how much we advance + // the input cursor. + // - Used a contived set of casts to sign extend the "read offset" control mask and then use it to mask the input word, + // which is then xor'd against the offset, for a "branchless" conditional move into the offset which + // has been carried over from the previous literal/match block. Note, this ends up doing better than a cmov on most + // modern processors. But we need to pre-xor the input offset. + // - We then load the match data from output buffer (offset back from the current output point). Unconditional load here. + // - We broadcast the "from literal" control mask from the current least significant byte of the SSE register using a shuffle epi-8 + // - We mask the literals with that SSE register wide mask. + // - The literals have been pre-xor'd with the data read in as match data, so we use an xor to branchlessly choose between the two. + // In this case, it ends up a better option than a blendv on most processors. + // - Store the block. We store all 16 bytes of the SSE register (due to some constraints in the format of the data, we won't + // go past the end of the buffer), but we may overlap this. + // - bytesOut controls how much we advance the output cursor. + // - We use 8 bit shifts to advance all the controls up to the next byte. There is some variable sized register trickery that + // x86/x64 is great for as long as we don't anger the register renamer. + +#define DECODE_STEP( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + \ + uint64_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_cvtsi64_si128( inputWord ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + readOffsetHalf##HILO >>= 8; \ + \ + const uint8_t* matchPointer = outputCursor - offset; \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( fromLiteral, literals ); \ + \ + fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + bytesOutHalf##HILO >>= 8; \ + streamBytesReadHalf##HILO >>= 8; \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_HALF( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast< const uint16_t* >( inputCursor ); \ + __m128i literals = _mm_cvtsi64_si128( inputWord ); \ + \ + offset ^= static_cast< size_t >( static_cast< ptrdiff_t >( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + const uint8_t* matchPointer = outputCursor - offset; \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( fromLiteral, literals ); \ + \ + fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_END( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast< const uint16_t* >( inputCursor ); \ + __m128i literals = _mm_cvtsi64_si128( inputWord ); \ + \ + offset ^= static_cast< size_t >( static_cast< ptrdiff_t >( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + const uint8_t* matchPointer = outputCursor - offset; \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( fromLiteral, literals ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Hi, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HALF_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HALF_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Hi, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_END_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_END_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Hi, CHECKMATCH, CHECKBUFFERS ) + + __m128i nibbleMask = _mm_set1_epi8( 0xF ); + __m128i literalsPerControl = _mm_set1_epi8( LITERALS_PER_CONTROL ); + + // Note, we use this block here because it allows the "fake" inputEarlyEnd/outputEarlyEnd not to cause register spills + // in the decompression loops. And yes, that did actually happen. + { +#pragma warning ( push ) +#pragma warning ( disable : 4101 ) + + const uint8_t* inputEarlyEnd; //= ( input + inputLength ) - END_PADDING_LITERALS; + uint8_t* outputEarlyEnd;// = ( output + outputLength ) - END_PADDING_LITERALS; + +#pragma warning ( pop ) + + // "Safe" ends to the buffer, before the input/output cursors hit these, we can loop without overflow checks. + const uint8_t* inputSafeEnd = ( input + inputLength ) - INPUT_BUFFER_SAFE; + uint8_t* outputSafeEnd = ( output + outputLength ) - OUTPUT_BUFFER_SAFE; + + // Decoding loop with offset output buffer underflow test, but no buffer overflow tests, assumed to end at a safe distance + // from overflows + while ( ( outputCursor - output ) < LZ_WINDOW_SIZE && outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) + { + // load the control block + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + + // split the control block into high and low nibbles. + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation + // this will essentially be ignored later on. + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + // Work out the carry for the low nibbles (which will be used with the high controls to put them into + // match without offset read mode). + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + + // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save + // the calculated carry to use that byte next iteration. + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); + + previousCarryHi = carryHi; + + // I want 128 set bits please. + __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + // Calcualting the bytes to output to the stream. Basically, we are subtracting negative one from the control value if the + // carry is not set. This works because the masks produced by comparisons are the equivalent to negative one, which + // make this a conditional increment. + __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_xor_si128( shiftedCarryHi, allSet ) ); + __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_xor_si128( carryLo, allSet ) ); + + // Calculate the number of bytes to read per control. + // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_min_epi8( literalsPerControl, bytesOutLo ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_min_epi8( literalsPerControl, bytesOutHi ) ); + + // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); + + // Masks whether we are reading literals - set if the carry is not set and these are literals. + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + // Advance the input past the control block. + inputCursor += CONTROL_BLOCK_SIZE; + + { + // Pull out the bottom halves off the SSE registers from before - we want these + // things in GPRs for the more linear logic. + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_HALF_LO( true, false ); + DECODE_STEP_HALF_HI( true, false ); + } + + { + // Now the top halves. + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_END_LO( true, false ); + DECODE_STEP_END_HI( true, false ); + } + } + + // Decoding loop with no buffer checks, but will end at a safe distance from the end of the buffers. + // Note, when we get here we have already reached the point in the output buffer which is *past* where we can underflow + // due to a bad match offset. + while ( outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) + { + // This code is the same as the loop above, see comments there + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); // where we take the carry from the previous hi values + + previousCarryHi = carryHi; + + __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_xor_si128( shiftedCarryHi, neg1 ) ); + __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_xor_si128( carryLo, neg1 ) ); + + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_min_epi8( literalsPerControl, bytesOutLo ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_min_epi8( literalsPerControl, bytesOutHi ) ); + + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); + + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + inputCursor += CONTROL_BLOCK_SIZE; + + { + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_HALF_LO( false, false ); + DECODE_STEP_HALF_HI( false, false ); + } + + { + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_END_LO( false, false ); + DECODE_STEP_END_HI( false, false ); + } + } + } + + // Decoding loop with all buffer checks. + { + const uint8_t* inputEarlyEnd; + uint8_t* outputEarlyEnd; + inputEarlyEnd = (( input + inputLength ) - END_PADDING_LITERALS); + outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; + + while ( outputCursor < outputEarlyEnd && inputCursor < inputEarlyEnd ) + { + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); // where we take the carry from the previous hi values + + previousCarryHi = carryHi; + + __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_xor_si128( shiftedCarryHi, neg1 ) ); + __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_xor_si128( carryLo, neg1 ) ); + + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_min_epi8( literalsPerControl, bytesOutLo ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_min_epi8( literalsPerControl, bytesOutHi ) ); + + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); + + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + inputCursor += CONTROL_BLOCK_SIZE; + + if ( inputCursor > inputEarlyEnd ) + goto BUFFER_END; + + { + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_HALF_LO( true, true ); + DECODE_STEP_HALF_HI( true, true ); + } + + { + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_END_LO( true, true ); + DECODE_STEP_END_HI( true, true ); + } + } + +BUFFER_END: + + // When we get here, we have either advanced the right amount on both cursors + // or something bad happened, so leave it as is, so we can tell where + // the error happened. + if ( inputCursor == inputEarlyEnd && outputCursor == outputEarlyEnd ) + { + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy any trailing literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + } + } + +MATCH_UNDERFLOW: + + return outputCursor - output; +} diff --git a/contrib/lzsse/lzsse2/lzsse2.h b/contrib/lzsse/lzsse2/lzsse2.h new file mode 100644 index 00000000000..ea908b85fb5 --- /dev/null +++ b/contrib/lzsse/lzsse2/lzsse2.h @@ -0,0 +1,90 @@ +/* +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#ifndef LZSSE2_H__ +#define LZSSE2_H__ + +#pragma once + +/* LZSSE2 - x64/SSE targeted codec for better performance with high compression ratio data/more optimal compressors. + * Supports minimum 3 byte matches, maximum 16 bytes of match per control word and 2 byte literal runs per control word. + */ + +#ifdef __cplusplus +extern "C" +{ +#endif + + /* Re-usable parse state object for compression. */ +typedef struct LZSSE2_OptimalParseState LZSSE2_OptimalParseState; + +/* Allocate the parse state for compression - returns null on failure. Note + Buffersize has to be greater or equal to any inputLength used with LZSSE2_CompressOptimalParse */ +LZSSE2_OptimalParseState* LZSSE2_MakeOptimalParseState( size_t bufferSize ); + +/* De-allocate the parse state for compression */ +void LZSSE2_FreeOptimalParseState( LZSSE2_OptimalParseState* toFree ); + +/* "Optimal" compression routine. +* Will compress data into LZSSE2 format, uses hash BST matching to find matches and run an optimal parse (high relative memory usage). Requires SSE 4.1. +* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. +* input : Buffer containing uncompressed data to be compressed. May not be null. +* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. +* output : Buffer that will receive the compressed output. +* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, +* The compressed data should never be longer than inputLength, as in this case the data is stored raw. +* level : The compression level to use for this file 1->17, 17 is highest compression, 0 is least +* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state +* concurrently. +* +* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). +*/ +size_t LZSSE2_CompressOptimalParse( LZSSE2_OptimalParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int level ); + +/* Decompression routine. +* This routine will decompress data in the LZSSE2 format and currently requires SSE 4.1 and is targeted at x64. +* It will perform poorly on x86 due to hunger for registers. +* input : Buffer containing compressed input block. May not be null. +* inputLength : Length of the compressed data in the input buffer - note, this should be under 2GB +* output : Buffer that will received the de-compressed output. Note, that this needs to be at least outputLength long. +* May not be null. +* outputLength : The length of the compressed output - note, this should be under 2GB +* +* Provided that input and output are valid pointers to buffers of at least their specified size, this routine +* should be memory safe - both match pointer checks and input/output buffer checks exist. +* +* Returns the size of the decompressed data, which will be less than outputLength in the event of an error (number of bytes +* will indicate where in the output stream the error occured). +* +* Note that this data is not hash verified, errors that occur are either from a misformed stream or bad buffer sizes. +* Remember, corrupt data can still be valid to decompress. +*/ +size_t LZSSE2_Decompress( const void* input, size_t inputLength, void* output, size_t outputLength ); + +#ifdef __cplusplus +} +#endif + +#endif /* -- LZSSE2_H__ */ diff --git a/contrib/lzsse/lzsse2/lzsse2_platform.h b/contrib/lzsse/lzsse2/lzsse2_platform.h new file mode 100644 index 00000000000..5cbab6b0600 --- /dev/null +++ b/contrib/lzsse/lzsse2/lzsse2_platform.h @@ -0,0 +1,73 @@ +/* +Copyright (c) 2016, Brian Marshall +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#ifndef LZSSE2_PLATFORM_H__ +#define LZSSE2_PLATFORM_H__ + +#pragma once + +/* + Compiler/Platform detection based on the table from: + https://blogs.msdn.microsoft.com/vcblog/2015/12/04/clang-with-microsoft-codegen-in-vs-2015-update-1/ +*/ + +#ifdef _MSC_VER + +/* + Microsoft Visual Studio Support. + C1xx/C2, Clang/C2 and Clang/LLVM all support the Microsoft header files and _BitScanForward + + Note: if you receive errors with the intrinsics make sure that you have SSE4.1 support enabled. + For example with Clang include "-msse4.1" on the command line +*/ +#include + +#else /* _MSC_VER */ + +#ifdef __GNUC__ + +/* + GCC +*/ + +/* + Note: including just would be sufficient, but including x86intrin is a better match to intrin.h on Visual Studio as + both include all intrinsics for the enabled processor, rather than just SSE4.1. +*/ +#include +/* _BitScanForward is Visual Studio specific. */ +#define _BitScanForward(x, m) *(x) = __builtin_ctz(m) + +#else + +/* +If you hit the error below, then add detection for your compiler/platform to this header file. +*/ +#error Platform not supported + +#endif /* __GNUC__ */ +#endif /* _MSC_VER */ + +#endif /* -- LZSSE2_PLATFORM_H__ */ diff --git a/contrib/lzsse/lzsse4/lzsse4.cpp b/contrib/lzsse/lzsse4/lzsse4.cpp new file mode 100644 index 00000000000..cf9f6fc6e60 --- /dev/null +++ b/contrib/lzsse/lzsse4/lzsse4.cpp @@ -0,0 +1,1499 @@ +/* +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#include +#include +#include +#include + +#include "lzsse4_platform.h" +#include "lzsse4.h" + +#pragma warning ( disable : 4127 ) + +namespace +{ + // Constants - most of these should not be changed without corresponding code changes because it will break many things in unpredictable ways. + const uint32_t WINDOW_BITS = 16; + const uint32_t MIN_MATCH_LENGTH = 4; + const uint32_t LZ_WINDOW_SIZE = 1 << WINDOW_BITS; + const uint32_t LZ_WINDOW_MASK = LZ_WINDOW_SIZE - 1; + const uint32_t FAST_HASH_BITS = 20; // You can change this - more bits = more matches, less bits = more cache hits + const uint32_t FAST_BUCKETS_COUNT = 1 << FAST_HASH_BITS; + const uint32_t FAST_HASH_MASK = FAST_BUCKETS_COUNT - 1; + const uint32_t MIN_COMPRESSION_SIZE = 32; + const uint32_t END_PADDING_LITERALS = 16; + const int32_t NO_MATCH = -1; + const int32_t EMPTY_NODE = -1; + const uint32_t MIN_LITERAL_COUNT = 8; + const uint32_t CONTROL_BITS = 4; + const uint32_t LITERAL_BITS = 8; + const uint32_t OFFSET_BITS = 16; + const uint32_t BASE_MATCH_BITS = OFFSET_BITS + CONTROL_BITS; + const uint32_t OFFSET_SIZE = 2; + const uint32_t EXTENDED_MATCH_BOUND = ( 1 << CONTROL_BITS ) - 1; + const uint32_t CONTROL_BLOCK_SIZE = sizeof( __m128i ); + const uint32_t CONTROLS_PER_BLOCK = 32; + const uint32_t LITERALS_PER_CONTROL = 4; + const uint32_t MAX_INPUT_PER_CONTROL = 4; + const size_t OUTPUT_BUFFER_SAFE = EXTENDED_MATCH_BOUND * CONTROLS_PER_BLOCK; + const size_t INPUT_BUFFER_SAFE = MAX_INPUT_PER_CONTROL * CONTROLS_PER_BLOCK; + const uint16_t INITIAL_OFFSET = MIN_MATCH_LENGTH; + const uint32_t OPTIMAL_HASH_BITS = 20; + const uint32_t OPTIMAL_BUCKETS_COUNT = 1 << OPTIMAL_HASH_BITS; + const uint32_t OPTIMAL_HASH_MASK = OPTIMAL_BUCKETS_COUNT - 1; + const size_t SKIP_MATCH_LENGTH = 128; + const uint32_t NO_SKIP_LEVEL = 17; +} + + +struct LZSSE4_FastParseState +{ + int32_t buckets[ FAST_BUCKETS_COUNT ]; // stores the first matching position, we can then look at the rest of the matches by tracing through the window. +}; + + +LZSSE4_FastParseState* LZSSE4_MakeFastParseState() +{ + return new LZSSE4_FastParseState(); +} + + +void LZSSE4_FreeFastParseState( LZSSE4_FastParseState* toFree ) +{ + delete toFree; +} + + +inline void SetHash( LZSSE4_FastParseState* state, uint32_t hash, const uint8_t* input, const uint8_t* inputCursor ) +{ + int32_t position = static_cast( inputCursor - input ); + + state->buckets[ hash & FAST_HASH_MASK ] = position; +} + +// Simple fast hash function - actually what is used in snappy and derivatives +// There's probably better, but I haven't spent time focusing on this area yet. +inline uint32_t HashFast( const uint8_t* inputCursor ) +{ + return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - FAST_HASH_BITS ); +} + +size_t LZSSE4_CompressFast( LZSSE4_FastParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) +{ + if ( outputLength < inputLength ) + { + // error case, output buffer not large enough. + return 0; + } + + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + if ( inputLength < MIN_COMPRESSION_SIZE ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + const uint8_t* inputEnd = input + inputLength; + const uint8_t* inputEarlyEnd = inputEnd - END_PADDING_LITERALS; + uint8_t* outputCursor = output; + uint8_t* outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; + uint32_t hash = 0; + + // initialize hash to empty + for ( int32_t* where = state->buckets, *end = state->buckets + FAST_BUCKETS_COUNT; where < end; where += 4 ) + { + where[ 0 ] = -1; + where[ 1 ] = -1; + where[ 2 ] = -1; + where[ 3 ] = -1; + } + + // initial literals that wont be compressed + for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) + { + hash = HashFast( inputCursor ); + + SetHash( state, hash, input, inputCursor ); + + *( outputCursor++ ) = *( inputCursor++ ); + } + + uint8_t* currentControlBlock = outputCursor; + uint32_t currentControlCount = 0; + uint16_t previousOffset = INITIAL_OFFSET; + size_t literalsToFlush = 0; + + outputCursor += CONTROL_BLOCK_SIZE; + + bool lastControlIsNop = false; + + // Loop through the data until we hit the end of one of the buffers (minus the end padding literals) + while ( inputCursor < inputEarlyEnd && outputCursor <= outputEarlyEnd ) + { + lastControlIsNop = false; + + hash = HashFast( inputCursor ); + + int matchPosition = state->buckets[ hash & FAST_HASH_MASK ]; + int currentPosition = static_cast< int32_t >( inputCursor - input ); + uint32_t matchLength = 0; + uint16_t matchOffset = static_cast< uint16_t >( currentPosition - matchPosition ); + + // If we had a hit in the hash and it wasn't outside the window. + if ( matchPosition >= 0 && ( currentPosition - matchPosition ) < ( LZ_WINDOW_SIZE - 1 ) ) + { + const uint8_t* matchCandidate = input + matchPosition; + uint32_t lengthToEnd = static_cast< uint32_t >( inputEarlyEnd - inputCursor ); + // Here we limit the hash length to prevent overlap matches with offset less than 16 bytes + uint32_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; + + // Find how long the match is 16 bytes at a time. + while ( matchLength < maxLength ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( matchCandidate + matchLength ) ); + + unsigned long matchBytes; + + // Finds the number of equal bytes at the start of the 16 + _BitScanForward( &matchBytes, ( static_cast< unsigned long >( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != sizeof( __m128i ) ) + { + break; + } + } + + matchLength = matchLength < maxLength ? matchLength : maxLength; + } + + // If we have at least the minimum match length (4 bytes) + if ( matchLength >= MIN_MATCH_LENGTH ) + { + // Do we have literals to flush before the match? + if ( literalsToFlush > 0 ) + { + // Start a new control block if we need one. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + // Would be larger than compressed size, get out! + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); + + // flush the literals. + // note the xor against the data that would be read in the match. + for ( uint32_t where = 0; where < literalsToFlush; ++where ) + { + const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); + + *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); + } + + ++currentControlCount; + + literalsToFlush = 0; + + // Would be larger than compressed size, get out! + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + // Start a new control block if the previous one is full. + // Note this is done before the offset is written out - the offset + // is always written after the control block containing the first + // control in the match. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + // The match length value we are encoding. + size_t toEncode = matchLength; + + // Write the offset out - note the xor with the previous offset. + *reinterpret_cast< uint16_t* >( outputCursor ) = matchOffset ^ previousOffset; + + previousOffset = matchOffset; + outputCursor += sizeof( uint16_t ); + + for ( ;; ) + { + // Check if we need to start a new control block + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + // If the encode size is greater than we can hold in a control, write out a full match length + // control, subtract full control value from the amount to encode and loop around again. + if ( toEncode >= EXTENDED_MATCH_BOUND ) + { + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( EXTENDED_MATCH_BOUND ) << 4 ); + + toEncode -= EXTENDED_MATCH_BOUND; + + ++currentControlCount; + } + else // Write out the remaining match length control. Could potentially be zero. + { + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( toEncode ) << 4 ); + + if ( currentControlCount == 0 && toEncode == 0 ) + { + lastControlIsNop = true; + } + + ++currentControlCount; + + break; + } + } + + // Update the value into the hash for future matches. + SetHash( state, hash, input, inputCursor ); + + ++inputCursor; + + // Hash all the other values in the match too. + for ( const uint8_t* nextArrival = inputCursor + matchLength - 1; inputCursor < nextArrival; ++inputCursor ) + { + hash = HashFast( inputCursor ); + SetHash( state, hash, input, inputCursor ); + } + } + else + { + // One more literal to write out. + ++literalsToFlush; + + // If we have reached the maximum number of literals allowed in the control, flush them out. + if ( literalsToFlush == LITERALS_PER_CONTROL ) + { + // Check if the control block is full and we need start a new one. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( ( static_cast( LITERALS_PER_CONTROL - 1 ) ) << 4 ); + + ++currentControlCount; + + *reinterpret_cast< uint32_t* >( outputCursor ) = + *reinterpret_cast< const uint32_t* >( inputCursor - 3 ) ^ + *reinterpret_cast< const uint32_t* >( ( inputCursor - 3 ) - previousOffset ); + + outputCursor += 4; + + //*( outputCursor++ ) = *( inputCursor - 3 ) ^ *( ( inputCursor - 3 ) - previousOffset ); + //*( outputCursor++ ) = *( inputCursor - 2 ) ^ *( ( inputCursor - 2 ) - previousOffset ); + //*( outputCursor++ ) = *( inputCursor - 1 ) ^ *( ( inputCursor - 1 ) - previousOffset ); + //*( outputCursor++ ) = *inputCursor ^ *( inputCursor - previousOffset ); + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + + literalsToFlush = 0; + } + + // Update the hash with this byte + SetHash( state, hash, input, inputCursor ); + + ++inputCursor; + } + } + + // If we would create a compression output bigger than or equal to the input, just copy the input to the output and return equal size. + if ( ( ( outputCursor + literalsToFlush + ( currentControlCount == CONTROLS_PER_BLOCK ? CONTROL_BLOCK_SIZE : 0 ) ) ) >= output + inputLength - END_PADDING_LITERALS ) + { + memcpy( output, input, inputLength ); + + outputCursor = output + inputLength; + } + else + { + // Flush any remaining literals. + if ( literalsToFlush > 0 ) + { + lastControlIsNop = false; + + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + } + + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); + + for ( uint32_t where = 0; where < literalsToFlush; ++where ) + { + const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); + + *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); + } + + ++currentControlCount; + } + + // Need to finish off shifting the final control block into the low nibble if there is no second nibble + if ( ( currentControlCount & 1 ) > 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] >>= 4; + } + + if ( lastControlIsNop ) + { + outputCursor -= CONTROL_BLOCK_SIZE; + } + + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy remaining literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + } + + // Return the size of the compressed data. + return outputCursor - output; +} + + +struct Arrival +{ + size_t cost; + int32_t from; + int32_t to; + uint16_t offset; +}; + +struct TreeNode +{ + int32_t children[ 2 ]; +}; + +struct LZSSE4_OptimalParseState +{ + // Note, we should really replace this with a BST, hash chaining works but is *slooooooooooooooow* for optimal parse. + int32_t roots[ OPTIMAL_BUCKETS_COUNT ]; + + TreeNode window[ LZ_WINDOW_SIZE ]; + + Arrival* arrivals; + + size_t bufferSize; +}; + + +LZSSE4_OptimalParseState* LZSSE4_MakeOptimalParseState( size_t bufferSize ) +{ + if ( bufferSize > 0 && ( SIZE_MAX / sizeof( Arrival ) ) < bufferSize ) + { + return nullptr; + } + + LZSSE4_OptimalParseState* result = reinterpret_cast< LZSSE4_OptimalParseState* >( ::malloc( sizeof( LZSSE4_OptimalParseState ) ) ); + + result->bufferSize = bufferSize; + + if ( result != nullptr ) + { + result->arrivals = reinterpret_cast< Arrival* >( ::malloc( sizeof( Arrival ) * bufferSize ) ); + + if ( result->arrivals == nullptr ) + { + LZSSE4_FreeOptimalParseState( result ); + + result = nullptr; + } + } + + return result; +} + + +void LZSSE4_FreeOptimalParseState( LZSSE4_OptimalParseState* toFree ) +{ + ::free( toFree->arrivals ); + + toFree->arrivals = nullptr; + + ::free( toFree ); +} + + +inline uint32_t HashOptimal( const uint8_t* inputCursor ) +{ + return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - OPTIMAL_HASH_BITS ); +} + + +struct Match +{ + size_t length; + int32_t position; + uint16_t offset; +}; + + +inline Match SearchAndUpdateFinder( LZSSE4_OptimalParseState& state, const uint8_t* input, const uint8_t* inputCursor, const uint8_t* inputEnd, uint32_t cutOff ) +{ + Match result; + + int32_t position = static_cast( inputCursor - input ); + + result.position = NO_MATCH; + result.length = MIN_MATCH_LENGTH; + result.offset = 0; + + size_t lengthToEnd = inputEnd - inputCursor; + int32_t lastPosition = position - ( LZ_WINDOW_SIZE - 1 ); + uint32_t hash = HashOptimal( inputCursor ); + + lastPosition = lastPosition > 0 ? lastPosition : 0; + + int32_t treeCursor = state.roots[ hash ]; + + state.roots[ hash ] = position; + + int32_t* left = &state.window[ position & LZ_WINDOW_MASK ].children[ 1 ]; + int32_t* right = &state.window[ position & LZ_WINDOW_MASK ].children[ 0 ]; + size_t leftLength = 0; + size_t rightLength = 0; + + for ( ;; ) + { + if ( cutOff-- == 0 || treeCursor < lastPosition ) + { + *left = *right = EMPTY_NODE; + break; + } + + TreeNode& currentNode = state.window[ treeCursor & LZ_WINDOW_MASK ]; + const uint8_t* key = input + treeCursor; + size_t matchLength = leftLength < rightLength ? leftLength : rightLength; + + uint16_t matchOffset = static_cast< uint16_t >( position - treeCursor ); + size_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; + + while ( matchLength < lengthToEnd ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); + + unsigned long matchBytes; + + _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != 16 ) + { + break; + } + } + + matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; + + size_t truncatedMatchLength = matchLength < maxLength ? matchLength : maxLength; + + if ( truncatedMatchLength >= result.length && matchOffset >= LITERALS_PER_CONTROL ) + { + result.length = truncatedMatchLength; + result.offset = matchOffset; + result.position = treeCursor; + } + + if ( matchLength == lengthToEnd ) + { + *left = currentNode.children[ 1 ]; + *right = currentNode.children[ 0 ]; + break; + } + + if ( inputCursor[ matchLength ] < key[ matchLength ] || ( matchLength == lengthToEnd ) ) + { + *left = treeCursor; + left = currentNode.children; + treeCursor = *left; + leftLength = matchLength; + } + else + { + *right = treeCursor; + right = currentNode.children + 1; + treeCursor = *right; + rightLength = matchLength; + } + } + + // Special RLE overlapping match case, the LzFind style match above doesn't work very well with our + // restriction of overlapping matches having offsets of at least 16. + // Suffix array seems like a better option to handling this. + { + // Note, we're detecting long RLE here, but if we have an offset too close, we'll sacrifice a fair + // amount of decompression performance to load-hit-stores. + int32_t matchPosition = position - ( sizeof( __m128i ) * 2 ); + + if ( matchPosition >= 0 ) + { + uint16_t matchOffset = static_cast( position - matchPosition ); + const uint8_t* key = input + matchPosition; + size_t matchLength = 0; + + while ( matchLength < lengthToEnd ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); + + unsigned long matchBytes; + + _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != 16 ) + { + break; + } + + } + + matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; + + if ( matchLength >= result.length ) + { + result.length = matchLength; + result.offset = matchOffset; + result.position = matchPosition; + } + } + } + + return result; +} + + +size_t LZSSE4_CompressOptimalParse( LZSSE4_OptimalParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength, unsigned int level ) +{ + if ( outputLength < inputLength || state->bufferSize < inputLength ) + { + // error case, output buffer not large enough. + return 0; + } + + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + if ( inputLength < MIN_COMPRESSION_SIZE ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + const uint8_t* inputEnd = input + inputLength; + Arrival* arrivalWatermark = state->arrivals; + Arrival* arrival = state->arrivals; + uint32_t cutOff = 1 << level; + + for ( int32_t* rootCursor = state->roots, *end = rootCursor + OPTIMAL_BUCKETS_COUNT; rootCursor < end; rootCursor += 4 ) + { + rootCursor[ 0 ] = EMPTY_NODE; + rootCursor[ 1 ] = EMPTY_NODE; + rootCursor[ 2 ] = EMPTY_NODE; + rootCursor[ 3 ] = EMPTY_NODE; + } + + for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) + { + SearchAndUpdateFinder( *state, input, inputCursor, inputEnd - END_PADDING_LITERALS, cutOff ); + + ++inputCursor; + } + + arrival->cost = LITERAL_BITS * LITERALS_PER_CONTROL; + arrival->from = -1; + arrival->offset = 0; + + // loop through each character and project forward the matches at that character to calculate the cheapest + // path of arrival for each individual character. + for ( const uint8_t* earlyEnd = inputEnd - END_PADDING_LITERALS; inputCursor < earlyEnd; ++inputCursor, ++arrival ) + { + uint32_t lengthToEnd = static_cast< uint32_t >( earlyEnd - inputCursor ); + int32_t currentPosition = static_cast< int32_t >( inputCursor - input ); + size_t literalsForward = LITERALS_PER_CONTROL < lengthToEnd ? LITERALS_PER_CONTROL : lengthToEnd; + size_t arrivalCost = arrival->cost; + + // NOTE - we currently assume only 2 literals filled in here, because the minimum match length is 3. + // If we wanted to go with a higher minimum match length, we would need to fill in more literals before hand. + // Also, because there is a maximum of 2 literals per control block assumed. + + // project forward the cost of a single literal + + for ( size_t where = 1; where <= literalsForward; ++where ) + { + Arrival* literalArrival = arrival + where; + size_t literalCost = arrivalCost + CONTROL_BITS + ( where * LITERAL_BITS ); + + if ( literalArrival > arrivalWatermark || literalArrival->cost > literalCost ) + { + literalArrival->cost = literalCost; + literalArrival->from = currentPosition; + literalArrival->offset = 0; + + arrivalWatermark = literalArrival > arrivalWatermark ? literalArrival : arrivalWatermark; + } + } + + Match match = SearchAndUpdateFinder( *state, input, inputCursor, earlyEnd, cutOff ); + + if ( match.position != NO_MATCH ) + { + for ( size_t matchedLength = MIN_MATCH_LENGTH, end = match.length + 1; matchedLength < end; ++matchedLength ) + { + Arrival* matchArrival = arrival + matchedLength; + size_t matchCost = arrivalCost + BASE_MATCH_BITS; + + if ( matchedLength >= EXTENDED_MATCH_BOUND ) + { + matchCost += ( matchedLength / EXTENDED_MATCH_BOUND ) * CONTROL_BITS; + } + + if ( matchArrival > arrivalWatermark || matchArrival->cost > matchCost ) + { + matchArrival->cost = matchCost; + matchArrival->from = currentPosition; + matchArrival->offset = match.offset; + + arrivalWatermark = matchArrival > arrivalWatermark ? matchArrival : arrivalWatermark; + } + } + + if ( match.length > SKIP_MATCH_LENGTH && level < NO_SKIP_LEVEL ) + { + arrival += match.length - LITERALS_PER_CONTROL; + inputCursor += match.length - LITERALS_PER_CONTROL; + } + } + } + + // If this would cost more to encode than it would if it were just literals, encode it with no control blocks, + // just literals + if ( ( arrivalWatermark->cost + END_PADDING_LITERALS * LITERAL_BITS + CONTROLS_PER_BLOCK * CONTROL_BITS ) > ( inputLength * LITERAL_BITS ) ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + Arrival* previousPathNode; + + // now trace the actual optimal parse path back, connecting the nodes in the other direction. + for ( const Arrival* pathNode = arrivalWatermark; pathNode->from > 0; pathNode = previousPathNode ) + { + previousPathNode = state->arrivals + ( pathNode->from - LITERALS_PER_CONTROL ); + + previousPathNode->to = static_cast( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); + } + + uint8_t* outputCursor = output; + + memcpy( outputCursor, input, MIN_MATCH_LENGTH ); + + outputCursor += MIN_MATCH_LENGTH; + + uint8_t* currentControlBlock = outputCursor; + uint32_t currentControlCount = 0; + uint32_t totalControlCount = 0; + + outputCursor += CONTROL_BLOCK_SIZE; + + Arrival* nextPathNode; + + size_t totalPathLength = MIN_MATCH_LENGTH; + uint16_t previousOffset = INITIAL_OFFSET; + + bool lastControlIsNop = false; + + // Now walk forwards again and actually write out the data. + for ( const Arrival* pathNode = state->arrivals; pathNode < arrivalWatermark; pathNode = nextPathNode ) + { + int32_t currentPosition = static_cast< int32_t >( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); + + nextPathNode = state->arrivals + ( pathNode->to - LITERALS_PER_CONTROL ); + + size_t pathDistance = nextPathNode - pathNode; + + totalPathLength += pathDistance; + + lastControlIsNop = false; + + if ( nextPathNode->offset == 0 ) + { + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + currentControlCount = 0; + } + + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + ( static_cast( pathDistance ) - 1 ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + ( static_cast< uint8_t >( pathDistance ) - 1 ) << CONTROL_BITS; + } + + // output the literals. + for ( int32_t where = 0; where < pathDistance; ++where ) + { + const uint8_t* currentInput = input + currentPosition + where; + + outputCursor[ where ] = *currentInput ^ *( currentInput - previousOffset ); + } + + outputCursor += pathDistance; + + ++totalControlCount; + ++currentControlCount; + } + else + { + // Check if we need to start a new control block + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + } + + // Write the offset out - note the xor with the previous offset. + *reinterpret_cast< uint16_t* >( outputCursor ) = nextPathNode->offset ^ previousOffset; + + previousOffset = nextPathNode->offset; + outputCursor += sizeof( uint16_t ); + + if ( pathDistance < EXTENDED_MATCH_BOUND ) + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( pathDistance ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( pathDistance ) << CONTROL_BITS; + } + + ++currentControlCount; + } + else + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( EXTENDED_MATCH_BOUND ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; + } + + ++currentControlCount; + + size_t toEncode = pathDistance - EXTENDED_MATCH_BOUND; + + for ( ;; ) + { + // Check if we need to start a new control block + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + } + + // If the encode size is greater than we can hold in a control, write out a full match length + // control, subtract full control value from the amount to encode and loop around again. + if ( toEncode >= EXTENDED_MATCH_BOUND ) + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( EXTENDED_MATCH_BOUND ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; + } + + toEncode -= EXTENDED_MATCH_BOUND; + + ++currentControlCount; + } + else // Write out the remaining match length control. Could potentially be zero. + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( toEncode ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( toEncode ) << CONTROL_BITS; + } + + if ( toEncode == 0 && currentControlCount == 0 ) + { + lastControlIsNop = true; + } + + ++currentControlCount; + + break; + } + } + } + } + } + + if ( lastControlIsNop ) + { + outputCursor -= CONTROL_BLOCK_SIZE; + } + + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy remaining literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + + return outputCursor - output; +} + + +size_t LZSSE4_Decompress( const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) +{ + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + // Data was not compressible, just copy initial values + if ( outputLength == inputLength ) + { + memcpy( output, input, outputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + uint8_t* outputCursor = output; + + // The offset starts off as the minimum match length. We actually need it least four + // characters back because we need them to be set to xor out the literals from the match data. + size_t offset = INITIAL_OFFSET; + __m128i previousCarryHi = _mm_setzero_si128(); + + // Copy the initial literals to the output. + for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) + { + *( outputCursor++ ) = *( inputCursor++ ); + } + + // Let me be clear, I am usually anti-macro, but they work for this particular (very unusual) case. + // DECODE_STEP is a regular decoding step, DECODE_STEP_HALF and DECODE_STEP_END are because the compiler couldn't + // seem to remove some of the dead code where values were updated and then never used. + + // What these macros do: + // Decode a single literal run or match run for a single control nibble. + // How they do it: + // - Read the *unaligned* input (in the case of LZSSE-F - twice), it goes into both a regular variable and an SSE register, + // because it could either be literals or an offset (or nothing at all). The low byte of streamBytesRead controls how much we advance + // the input cursor. + // - Used a contived set of casts to sign extend the "read offset" control mask and then use it to mask the input word, + // which is then xor'd against the offset, for a "branchless" conditional move into the offset which + // has been carried over from the previous literal/match block. Note, this ends up doing better than a cmov on most + // modern processors. But we need to pre-xor the input offset. + // - We then load the match data from output buffer (offset back from the current output point). Unconditional load here. + // - We broadcast the "from literal" control mask from the current least significant byte of the SSE register using a shuffle epi-8 + // - We mask the literals with that SSE register wide mask. + // - The literals have been pre-xor'd with the data read in as match data, so we use an xor to branchlessly choose between the two. + // In this case, it ends up a better option than a blendv on most processors. + // - Store the block. We store all 16 bytes of the SSE register (due to some constraints in the format of the data, we won't + // go past the end of the buffer), but we may overlap this. + // - bytesOut controls how much we advance the output cursor. + // - We use 8 bit shifts to advance all the controls up to the next byte. There is some variable sized register trickery that + // x86/x64 is great for as long as we don't anger the register renamer. + +#define DECODE_STEP( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + readOffsetHalf##HILO >>= 8; \ + \ + const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( literals, fromLiteral ); \ + \ + fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + bytesOutHalf##HILO >>= 8; \ + streamBytesReadHalf##HILO >>= 8; \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_HALF( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( literals, fromLiteral ); \ + \ + fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_END( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( literals, fromLiteral ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Hi, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HALF_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HALF_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Hi, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_END_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_END_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Hi, CHECKMATCH, CHECKBUFFERS ) + + __m128i nibbleMask = _mm_set1_epi8( 0xF ); + __m128i offsetSize = _mm_set1_epi8( OFFSET_SIZE ); + + // Note, we use this block here because it allows the "fake" inputEarlyEnd/outputEarlyEnd not to cause register spills + // in the decompression loops. And yes, that did actually happen. + { + +#pragma warning ( push ) +#pragma warning ( disable : 4101 ) + + // These variables are not actually ever used in this block, because we use + // a constant conditional expression to take out the branches that would hit them. + // But unfortunately, we need them to compile. + const uint8_t* inputEarlyEnd; + uint8_t* outputEarlyEnd; + +#pragma warning ( pop ) + + // "Safe" ends to the buffer, before the input/output cursors hit these, we can loop without overflow checks. + const uint8_t* inputSafeEnd = ( input + inputLength ) - INPUT_BUFFER_SAFE; + uint8_t* outputSafeEnd = ( output + outputLength ) - OUTPUT_BUFFER_SAFE; + + // Decoding loop with offset output buffer underflow test, but no buffer overflow tests, assumed to end at a safe distance + // from overflows + while ( ( outputCursor - output ) < LZ_WINDOW_SIZE && outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) + { + // load the control block + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + + // split the control block into high and low nibbles + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will + // be an extension of the current match operation. + + // Work out the carry for the low nibbles (which will be used with the high controls to put them into + // match without offset read mode). + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + + // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save + // the calculated carry to use that byte next iteration. + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); + + previousCarryHi = carryHi; + + // We make the implicit assumption that the maximum number of literals to controls here is twice the offset size (4 vs 2), + // we are doing this here to save keeping the value around (spilling or fetching it each time) + __m128i literalsPerControl = _mm_add_epi8( offsetSize, offsetSize ); + + // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation + // this will essentially be ignored later on. + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + // Here we're calculating the number of bytes that will be output, we are actually subtracting negative one from the control + // (handy trick where comparison result masks are negative one) if carry is not set and it is a literal. + __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_andnot_si128( shiftedCarryHi, isLiteralLo ) ); + __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_andnot_si128( carryLo, isLiteralHi ) ); + + // Calculate the number of bytes to read per control. + // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_blendv_epi8( offsetSize, bytesOutLo, isLiteralLo ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_blendv_epi8( offsetSize, bytesOutHi, isLiteralHi ) ); + + // I want 128 set bits please. + __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); + + // Masks whether we are reading literals - set if the carry is not set and these are literals. + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + // Advance the input past the control block + inputCursor += CONTROL_BLOCK_SIZE; + + { + // Pull out the bottom halves off the SSE registers from before - we want these + // things in GPRs for the more linear logic. + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_HALF_LO( true, false ); + DECODE_STEP_HALF_HI( true, false ); + } + + { + // Now the top halves. + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_END_LO( true, false ); + DECODE_STEP_END_HI( true, false ); + } + } + + // Decoding loop with no buffer checks, but will end at a safe distance from the end of the buffers. + // Note, when we get here we have already reached the point in the output buffer which is *past* where we can underflow + // due to a bad match offset. + while ( outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) + { + // This code is the same as the loop above, see comments there + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); // where we take the carry from the previous hi values + + previousCarryHi = carryHi; + + __m128i literalsPerControl = _mm_add_epi8( offsetSize, offsetSize ); + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_andnot_si128( shiftedCarryHi, isLiteralLo ) ); + __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_andnot_si128( carryLo, isLiteralHi ) ); + + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_blendv_epi8( offsetSize, bytesOutLo, isLiteralLo ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_blendv_epi8( offsetSize, bytesOutHi, isLiteralHi ) ); + + __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); + + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + inputCursor += CONTROL_BLOCK_SIZE; + + { + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_HALF_LO( false, false ); + DECODE_STEP_HALF_HI( false, false ); + } + + { + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_END_LO( false, false ); + DECODE_STEP_END_HI( false, false ); + } + } + } + + // Decoding loop with all buffer checks. + { + const uint8_t* inputEarlyEnd; + uint8_t* outputEarlyEnd; + inputEarlyEnd = ( input + inputLength ) - END_PADDING_LITERALS; + outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; + + while ( outputCursor < outputEarlyEnd && inputCursor < inputEarlyEnd ) + { + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); + + previousCarryHi = carryHi; + + __m128i literalsPerControl = _mm_add_epi8( offsetSize, offsetSize ); + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_andnot_si128( shiftedCarryHi, isLiteralLo ) ); + __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_andnot_si128( carryLo, isLiteralHi ) ); + + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_blendv_epi8( offsetSize, bytesOutLo, isLiteralLo ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_blendv_epi8( offsetSize, bytesOutHi, isLiteralHi ) ); + + __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); + + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + inputCursor += CONTROL_BLOCK_SIZE; + + if ( inputCursor > inputEarlyEnd ) + goto BUFFER_END; + + { + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_HALF_LO( true, true ); + DECODE_STEP_HALF_HI( true, true ); + } + + { + // Now the top halves. + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_END_LO( true, true ); + DECODE_STEP_END_HI( true, true ); + } + } + +BUFFER_END: + + // When we get here, we have either advanced the right amount on both cursors + // or something bad happened, so leave it as is, so we can tell where + // the error happened. + if ( inputCursor == inputEarlyEnd && outputCursor == outputEarlyEnd ) + { + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy any trailing literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + } + } + +MATCH_UNDERFLOW: + + return outputCursor - output; +} diff --git a/contrib/lzsse/lzsse4/lzsse4.h b/contrib/lzsse/lzsse4/lzsse4.h new file mode 100644 index 00000000000..0fa5e6c8ed0 --- /dev/null +++ b/contrib/lzsse/lzsse4/lzsse4.h @@ -0,0 +1,117 @@ +/* +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#ifndef LZSSE4_H__ +#define LZSSE4_H__ + +#pragma once + +/* LZSSE4 - x64/SSE targeted codec for better performance with lower compression ratio data/less optimal compressors. + * Supports minimum 4 byte matches, maximum 15 bytes of match per control word and 4 byte literal runs per control word. + */ + +#ifdef __cplusplus +extern "C" +{ +#endif + +/* Re-usable parse state object for compression. */ +typedef struct LZSSE4_FastParseState LZSSE4_FastParseState; + +/* Allocate the parse state for compression - returns null on failure */ +LZSSE4_FastParseState* LZSSE4_MakeFastParseState(); + +/* De-allocate the parse state for compression */ +void LZSSE4_FreeFastParseState( LZSSE4_FastParseState* toFree ); + +/* Re-usable parse state object for compression. */ +typedef struct LZSSE4_OptimalParseState LZSSE4_OptimalParseState; + +/* Allocate the parse state for compression - returns null on failure. Note + Buffersize has to be greater or equal to any inputLength used with LZSSE2_CompressOptimalParse */ +LZSSE4_OptimalParseState* LZSSE4_MakeOptimalParseState( size_t bufferSize ); + +/* De-allocate the parse state for compression */ +void LZSSE4_FreeOptimalParseState( LZSSE4_OptimalParseState* toFree ); + + +/* "Optimal" compression routine. +* Will compress data into LZSSE4 format, uses hash BST matching to find matches and run an optimal parse (high relative memory usage). Requires SSE 4.1. +* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. +* input : Buffer containing uncompressed data to be compressed. May not be null. +* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. +* output : Buffer that will receive the compressed output. +* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, +* The compressed data should never be longer than inputLength, as in this case the data is stored raw. +* level : The compression level to use for this file 1->17, 17 delivers the highest compression, 1 delivers the least. +* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state +* concurrently. +* +* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). +*/ +size_t LZSSE4_CompressOptimalParse( LZSSE4_OptimalParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int level ); + + +/* "Fast" compression routine. + * Will compress data into LZSSE4 format, uses a simple single entry hash/greedy matching to find matches. Requires SSE 4.1. + * state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. + * input : Buffer containing uncompressed data to be compressed. May not be null. + * inputLength : Length of the compressed data in the input buffer - note should be under 2GB. + * output : Buffer that will receive the compressed output. + * outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, + * The compressed data should never be longer than inputLength, as in this case the data is stored raw. + * + * Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state + * concurrently. + * + * Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). + */ +size_t LZSSE4_CompressFast( LZSSE4_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength ); + +/* Decompression routine. + * This routine will decompress data in the LZSSE4 format and currently requires SSE 4.1 and is targeted at x64. + * It will perform poorly on x86 due to hunger for registers. + * input : Buffer containing compressed input block. May not be null. + * inputLength : Length of the compressed data in the input buffer - note, this should be under 2GB + * output : Buffer that will received the de-compressed output. Note, that this needs to be at least outputLength long. + * May not be null. + * outputLength : The length of the compressed output - note, this should be under 2GB + * + * Provided that input and output are valid pointers to buffers of at least their specified size, this routine + * should be memory safe - both match pointer checks and input/output buffer checks exist. + * + * Returns the size of the decompressed data, which will be less than outputLength in the event of an error (number of bytes + * will indicate where in the output stream the error occured). + * + * Note that this data is not hash verified, errors that occur are either from a misformed stream or bad buffer sizes. + * Remember, corrupt data can still be valid to decompress. + */ +size_t LZSSE4_Decompress( const void* input, size_t inputLength, void* output, size_t outputLength ); + +#ifdef __cplusplus +} +#endif + +#endif /* -- LZSSE4_H__ */ diff --git a/contrib/lzsse/lzsse4/lzsse4_platform.h b/contrib/lzsse/lzsse4/lzsse4_platform.h new file mode 100644 index 00000000000..d1d3f1cced2 --- /dev/null +++ b/contrib/lzsse/lzsse4/lzsse4_platform.h @@ -0,0 +1,73 @@ +/* +Copyright (c) 2016, Brian Marshall +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#ifndef LZSSE4_PLATFORM_H__ +#define LZSSE4_PLATFORM_H__ + +#pragma once + +/* + Compiler/Platform detection based on the table from: + https://blogs.msdn.microsoft.com/vcblog/2015/12/04/clang-with-microsoft-codegen-in-vs-2015-update-1/ +*/ + +#ifdef _MSC_VER + +/* + Microsoft Visual Studio Support. + C1xx/C2, Clang/C2 and Clang/LLVM all support the Microsoft header files and _BitScanForward + + Note: if you receive errors with the intrinsics make sure that you have SSE4.1 support enabled. + For example with Clang include "-msse4.1" on the command line +*/ +#include + +#else /* _MSC_VER */ + +#ifdef __GNUC__ + +/* + GCC +*/ + +/* + Note: including just would be sufficient, but including x86intrin is a better match to intrin.h on Visual Studio as + both include all intrinsics for the enabled processor, rather than just SSE4.1. +*/ +#include +/* _BitScanForward is Visual Studio specific. */ +#define _BitScanForward(x, m) *(x) = __builtin_ctz(m) + +#else + +/* +If you hit the error below, then add detection for your compiler/platform to this header file. +*/ +#error Platform not supported + +#endif /* __GNUC__ */ +#endif /* _MSC_VER */ + +#endif /* -- LZSSE4_PLATFORM_H__ */ diff --git a/contrib/lzsse/lzsse8/lzsse8.cpp b/contrib/lzsse/lzsse8/lzsse8.cpp new file mode 100644 index 00000000000..a5682a1de81 --- /dev/null +++ b/contrib/lzsse/lzsse8/lzsse8.cpp @@ -0,0 +1,1568 @@ +/* +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#include +#include +#include +#include + +#include "lzsse8_platform.h" +#include "lzsse8.h" + +#pragma warning ( disable : 4127 ) + +namespace +{ + // Constants - most of these should not be changed without corresponding code changes because it will break many things in unpredictable ways. + const uint32_t WINDOW_BITS = 16; + const uint32_t MIN_MATCH_LENGTH = 4; + const uint32_t LZ_WINDOW_SIZE = 1 << WINDOW_BITS; + const uint32_t LZ_WINDOW_MASK = LZ_WINDOW_SIZE - 1; + const uint32_t FAST_HASH_BITS = 20; // You can change this - more bits = more matches, less bits = more cache hits + const uint32_t FAST_BUCKETS_COUNT = 1 << FAST_HASH_BITS; + const uint32_t FAST_HASH_MASK = FAST_BUCKETS_COUNT - 1; + const uint32_t MIN_COMPRESSION_SIZE = 32; + const uint32_t END_PADDING_LITERALS = 16; + const int32_t NO_MATCH = -1; + const int32_t EMPTY_NODE = -1; + const uint32_t CONTROL_BITS = 4; + const uint32_t OFFSET_SIZE = 2; + const uint32_t EXTENDED_MATCH_BOUND = ( 1 << CONTROL_BITS ) - 1; + const uint32_t CONTROL_BLOCK_SIZE = sizeof( __m128i ); + const uint32_t CONTROLS_PER_BLOCK = 32; + const uint32_t LITERALS_PER_CONTROL = 8; + const uint32_t MAX_INPUT_PER_CONTROL = 8; + const uint32_t INITIAL_MATCH_BOUND = 11; + const size_t OUTPUT_BUFFER_SAFE = EXTENDED_MATCH_BOUND * CONTROLS_PER_BLOCK; + const size_t INPUT_BUFFER_SAFE = MAX_INPUT_PER_CONTROL * CONTROLS_PER_BLOCK; + const uint16_t INITIAL_OFFSET = LITERALS_PER_CONTROL; + const uint32_t OFFSET_BITS = 16; + const uint32_t BASE_MATCH_BITS = OFFSET_BITS + CONTROL_BITS; + const uint32_t OPTIMAL_HASH_BITS = 20; + const uint32_t OPTIMAL_BUCKETS_COUNT = 1 << OPTIMAL_HASH_BITS; + const uint32_t OPTIMAL_HASH_MASK = OPTIMAL_BUCKETS_COUNT - 1; + const uint32_t LITERAL_BITS = 8; + const size_t SKIP_MATCH_LENGTH = 128; + const uint32_t NO_SKIP_LEVEL = 17; + +} + + +struct LZSSE8_FastParseState +{ + int32_t buckets[ FAST_BUCKETS_COUNT ]; // stores the first matching position, we can then look at the rest of the matches by tracing through the window. +}; + + +LZSSE8_FastParseState* LZSSE8_MakeFastParseState() +{ + return new LZSSE8_FastParseState(); +} + + +void LZSSE8_FreeFastParseState( LZSSE8_FastParseState* toFree ) +{ + delete toFree; +} + + +inline void SetHash( LZSSE8_FastParseState* state, uint32_t hash, const uint8_t* input, const uint8_t* inputCursor ) +{ + int32_t position = static_cast( inputCursor - input ); + + state->buckets[ hash & FAST_HASH_MASK ] = position; +} + +// Simple fast hash function - actually what is used in snappy and derivatives +// There's probably better, but I haven't spent time focusing on this area yet. +inline uint32_t HashFast( const uint8_t* inputCursor ) +{ + return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - FAST_HASH_BITS ); +} + +struct Arrival +{ + size_t cost; + int32_t from; + int32_t to; + uint16_t offset; +}; + +struct TreeNode +{ + int32_t children[ 2 ]; +}; + +struct LZSSE8_OptimalParseState +{ + int32_t roots[ OPTIMAL_BUCKETS_COUNT ]; + + TreeNode window[ LZ_WINDOW_SIZE ]; + + Arrival* arrivals; + + size_t bufferSize; +}; + + +LZSSE8_OptimalParseState* LZSSE8_MakeOptimalParseState( size_t bufferSize ) +{ + if ( bufferSize > 0 && ( SIZE_MAX / sizeof( Arrival ) ) < bufferSize ) + { + return nullptr; + } + + LZSSE8_OptimalParseState* result = reinterpret_cast< LZSSE8_OptimalParseState* >( ::malloc( sizeof( LZSSE8_OptimalParseState ) ) ); + + result->bufferSize = bufferSize; + + if ( result != nullptr ) + { + result->arrivals = reinterpret_cast< Arrival* >( ::malloc( sizeof( Arrival ) * bufferSize ) ); + + if ( result->arrivals == nullptr ) + { + LZSSE8_FreeOptimalParseState( result ); + + result = nullptr; + } + } + + return result; +} + + +void LZSSE8_FreeOptimalParseState( LZSSE8_OptimalParseState* toFree ) +{ + ::free( toFree->arrivals ); + + toFree->arrivals = nullptr; + + ::free( toFree ); +} + + +inline uint32_t HashOptimal( const uint8_t* inputCursor ) +{ + return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - OPTIMAL_HASH_BITS ); +} + + +struct Match +{ + size_t length; + int32_t position; + uint16_t offset; +}; + + +inline Match SearchAndUpdateFinder( LZSSE8_OptimalParseState& state, const uint8_t* input, const uint8_t* inputCursor, const uint8_t* inputEnd, uint32_t cutOff ) +{ + Match result; + + int32_t position = static_cast( inputCursor - input ); + + result.position = NO_MATCH; + result.length = MIN_MATCH_LENGTH; + result.offset = 0; + + size_t lengthToEnd = inputEnd - inputCursor; + int32_t lastPosition = position - ( LZ_WINDOW_SIZE - 1 ); + uint32_t hash = HashOptimal( inputCursor ); + + lastPosition = lastPosition > 0 ? lastPosition : 0; + + int32_t treeCursor = state.roots[ hash ]; + + state.roots[ hash ] = position; + + int32_t* left = &state.window[ position & LZ_WINDOW_MASK ].children[ 1 ]; + int32_t* right = &state.window[ position & LZ_WINDOW_MASK ].children[ 0 ]; + size_t leftLength = 0; + size_t rightLength = 0; + + for ( ;; ) + { + if ( cutOff-- == 0 || treeCursor < lastPosition ) + { + *left = *right = EMPTY_NODE; + break; + } + + TreeNode& currentNode = state.window[ treeCursor & LZ_WINDOW_MASK ]; + const uint8_t* key = input + treeCursor; + size_t matchLength = leftLength < rightLength ? leftLength : rightLength; + + uint16_t matchOffset = static_cast( position - treeCursor ); + size_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; + + while ( matchLength < lengthToEnd ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); + + unsigned long matchBytes; + + _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != 16 ) + { + break; + } + } + + matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; + + size_t truncatedMatchLength = matchLength < maxLength ? matchLength : maxLength; + + if ( truncatedMatchLength >= result.length && matchOffset >= LITERALS_PER_CONTROL ) + { + result.length = truncatedMatchLength; + result.offset = matchOffset; + result.position = treeCursor; + } + + if ( matchLength == lengthToEnd ) + { + *left = currentNode.children[ 1 ]; + *right = currentNode.children[ 0 ]; + break; + } + + if ( inputCursor[ matchLength ] < key[ matchLength ] || ( matchLength == lengthToEnd ) ) + { + *left = treeCursor; + left = currentNode.children; + treeCursor = *left; + leftLength = matchLength; + } + else + { + *right = treeCursor; + right = currentNode.children + 1; + treeCursor = *right; + rightLength = matchLength; + } + } + + // Special RLE overlapping match case, the LzFind style match above doesn't work very well with our + // restriction of overlapping matches having offsets of at least 16. + // Suffix array seems like a better option to handling this. + { + // Note, we're detecting long RLE here, but if we have an offset too close, we'll sacrifice a fair + // amount of decompression performance to load-hit-stores. + int32_t matchPosition = position - ( sizeof( __m128i ) * 2 ); + + if ( matchPosition >= 0 ) + { + uint16_t matchOffset = static_cast( position - matchPosition ); + const uint8_t* key = input + matchPosition; + size_t matchLength = 0; + + while ( matchLength < lengthToEnd ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); + + unsigned long matchBytes; + + _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != 16 ) + { + break; + } + + } + + matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; + + if ( matchLength >= result.length ) + { + result.length = matchLength; + result.offset = matchOffset; + result.position = matchPosition; + } + } + } + + return result; +} + + +size_t LZSSE8_CompressOptimalParse( LZSSE8_OptimalParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength, unsigned int level ) +{ + if ( outputLength < inputLength || state->bufferSize < inputLength ) + { + // error case, output buffer not large enough. + return 0; + } + + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + if ( inputLength < MIN_COMPRESSION_SIZE ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + const uint8_t* inputEnd = input + inputLength; + Arrival* arrivalWatermark = state->arrivals; + Arrival* arrival = state->arrivals; + uint32_t cutOff = 1 << level; + + for ( int32_t* rootCursor = state->roots, *end = rootCursor + OPTIMAL_BUCKETS_COUNT; rootCursor < end; rootCursor += 4 ) + { + rootCursor[ 0 ] = EMPTY_NODE; + rootCursor[ 1 ] = EMPTY_NODE; + rootCursor[ 2 ] = EMPTY_NODE; + rootCursor[ 3 ] = EMPTY_NODE; + } + + for ( uint32_t where = 0; where < LITERALS_PER_CONTROL; ++where ) + { + SearchAndUpdateFinder( *state, input, inputCursor, inputEnd - END_PADDING_LITERALS, cutOff ); + + ++inputCursor; + } + + arrival->cost = LITERAL_BITS * LITERALS_PER_CONTROL; + arrival->from = -1; + arrival->offset = 0; + + // loop through each character and project forward the matches at that character to calculate the cheapest + // path of arrival for each individual character. + for ( const uint8_t* earlyEnd = inputEnd - END_PADDING_LITERALS; inputCursor < earlyEnd; ++inputCursor, ++arrival ) + { + uint32_t lengthToEnd = static_cast< uint32_t >( earlyEnd - inputCursor ); + int32_t currentPosition = static_cast< int32_t >( inputCursor - input ); + size_t literalsForward = LITERALS_PER_CONTROL < lengthToEnd ? LITERALS_PER_CONTROL : lengthToEnd; + size_t arrivalCost = arrival->cost; + + // NOTE - we currently assume only 2 literals filled in here, because the minimum match length is 3. + // If we wanted to go with a higher minimum match length, we would need to fill in more literals before hand. + // Also, because there is a maximum of 2 literals per control block assumed. + + // project forward the cost of a single literal + + for ( size_t where = 1; where <= literalsForward; ++where ) + { + Arrival* literalArrival = arrival + where; + size_t literalCost = arrivalCost + CONTROL_BITS + ( where * LITERAL_BITS ); + + if ( literalArrival > arrivalWatermark || literalArrival->cost > literalCost ) + { + literalArrival->cost = literalCost; + literalArrival->from = currentPosition; + literalArrival->offset = 0; + + arrivalWatermark = literalArrival > arrivalWatermark ? literalArrival : arrivalWatermark; + } + } + + Match match = SearchAndUpdateFinder( *state, input, inputCursor, earlyEnd, cutOff ); + + if ( match.position != NO_MATCH ) + { + for ( size_t matchedLength = MIN_MATCH_LENGTH, end = match.length + 1; matchedLength < end; ++matchedLength ) + { + Arrival* matchArrival = arrival + matchedLength; + size_t matchCost = arrivalCost + BASE_MATCH_BITS; + + if ( matchedLength >= INITIAL_MATCH_BOUND ) + { + matchCost += ( ( ( matchedLength - INITIAL_MATCH_BOUND ) / EXTENDED_MATCH_BOUND ) + 1 ) * CONTROL_BITS; + } + + if ( matchArrival > arrivalWatermark || matchArrival->cost > matchCost ) + { + matchArrival->cost = matchCost; + matchArrival->from = currentPosition; + matchArrival->offset = match.offset; + + arrivalWatermark = matchArrival > arrivalWatermark ? matchArrival : arrivalWatermark; + } + } + + if ( match.length > SKIP_MATCH_LENGTH && level < NO_SKIP_LEVEL ) + { + arrival += match.length - LITERALS_PER_CONTROL; + inputCursor += match.length - LITERALS_PER_CONTROL; + } + } + } + + // If this would cost more to encode than it would if it were just literals, encode it with no control blocks, + // just literals + if ( ( arrivalWatermark->cost + END_PADDING_LITERALS * LITERAL_BITS + CONTROLS_PER_BLOCK * CONTROL_BITS ) > ( inputLength * LITERAL_BITS ) ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + Arrival* previousPathNode; + + // now trace the actual optimal parse path back, connecting the nodes in the other direction. + for ( const Arrival* pathNode = arrivalWatermark; pathNode->from > 0; pathNode = previousPathNode ) + { + previousPathNode = state->arrivals + ( pathNode->from - LITERALS_PER_CONTROL ); + + previousPathNode->to = static_cast( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); + } + + uint8_t* outputCursor = output; + + memcpy( outputCursor, input, LITERALS_PER_CONTROL ); + + outputCursor += LITERALS_PER_CONTROL; + + uint8_t* currentControlBlock = outputCursor; + uint32_t currentControlCount = 0; + uint32_t totalControlCount = 0; + + outputCursor += CONTROL_BLOCK_SIZE; + + Arrival* nextPathNode; + + size_t totalPathLength = LITERALS_PER_CONTROL; + uint16_t previousOffset = INITIAL_OFFSET; + + bool lastControlIsNop = false; + + // Now walk forwards again and actually write out the data. + for ( const Arrival* pathNode = state->arrivals; pathNode < arrivalWatermark; pathNode = nextPathNode ) + { + int32_t currentPosition = static_cast< int32_t >( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); + + nextPathNode = state->arrivals + ( pathNode->to - LITERALS_PER_CONTROL ); + + size_t pathDistance = nextPathNode - pathNode; + + totalPathLength += pathDistance; + + lastControlIsNop = false; + + if ( nextPathNode->offset == 0 ) + { + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + currentControlCount = 0; + } + + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + ( static_cast( pathDistance ) - 1 ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + ( static_cast< uint8_t >( pathDistance ) - 1 ) << CONTROL_BITS; + } + + // output the literals. + for ( int32_t where = 0; where < pathDistance; ++where ) + { + const uint8_t* currentInput = input + currentPosition + where; + + outputCursor[ where ] = *currentInput ^ *( currentInput - previousOffset ); + } + + outputCursor += pathDistance; + + ++totalControlCount; + ++currentControlCount; + } + else + { + // Check if we need to start a new control block + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + } + + // Write the offset out - note the xor with the previous offset. + *reinterpret_cast< uint16_t* >( outputCursor ) = nextPathNode->offset ^ previousOffset; + + previousOffset = nextPathNode->offset; + outputCursor += sizeof( uint16_t ); + + if ( pathDistance < INITIAL_MATCH_BOUND ) + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( pathDistance + MIN_MATCH_LENGTH ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( pathDistance + MIN_MATCH_LENGTH ) << CONTROL_BITS; + } + + ++currentControlCount; + } + else + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( EXTENDED_MATCH_BOUND ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; + } + + ++currentControlCount; + + size_t toEncode = pathDistance - INITIAL_MATCH_BOUND; + + for ( ;; ) + { + // Check if we need to start a new control block + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + } + + // If the encode size is greater than we can hold in a control, write out a full match length + // control, subtract full control value from the amount to encode and loop around again. + if ( toEncode >= EXTENDED_MATCH_BOUND ) + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( EXTENDED_MATCH_BOUND ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; + } + + toEncode -= EXTENDED_MATCH_BOUND; + + ++currentControlCount; + } + else // Write out the remaining match length control. Could potentially be zero. + { + if ( ( currentControlCount & 1 ) == 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] = + static_cast( toEncode ); + } + else + { + currentControlBlock[ currentControlCount >> 1 ] |= + static_cast< uint8_t >( toEncode ) << CONTROL_BITS; + } + + if ( toEncode == 0 && currentControlCount == 0 ) + { + lastControlIsNop = true; + } + + ++currentControlCount; + + break; + } + } + } + } + } + + if ( lastControlIsNop ) + { + outputCursor -= CONTROL_BLOCK_SIZE; + } + + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy remaining literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + + return outputCursor - output; +} + + +size_t LZSSE8_CompressFast( LZSSE8_FastParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) +{ + if ( outputLength < inputLength ) + { + // error case, output buffer not large enough. + return 0; + } + + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + if ( inputLength < MIN_COMPRESSION_SIZE ) + { + memcpy( output, input, inputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + const uint8_t* inputEnd = input + inputLength; + const uint8_t* inputEarlyEnd = inputEnd - END_PADDING_LITERALS; + uint8_t* outputCursor = output; + uint8_t* outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; + uint32_t hash = 0; + + // initialize hash to empty + for ( int32_t* where = state->buckets, *end = state->buckets + FAST_BUCKETS_COUNT; where < end; where += 4 ) + { + where[ 0 ] = -1; + where[ 1 ] = -1; + where[ 2 ] = -1; + where[ 3 ] = -1; + } + + // initial literals that wont be compressed + for ( uint32_t where = 0; where < LITERALS_PER_CONTROL; ++where ) + { + hash = HashFast( inputCursor ); + + SetHash( state, hash, input, inputCursor ); + + *( outputCursor++ ) = *( inputCursor++ ); + } + + uint8_t* currentControlBlock = outputCursor; + uint32_t currentControlCount = 0; + uint16_t previousOffset = INITIAL_OFFSET; + size_t literalsToFlush = 0; + + outputCursor += CONTROL_BLOCK_SIZE; + + bool lastControlIsNop = false; + + // Loop through the data until we hit the end of one of the buffers (minus the end padding literals) + while ( inputCursor < inputEarlyEnd && outputCursor <= outputEarlyEnd ) + { + lastControlIsNop = false; + + hash = HashFast( inputCursor ); + + int matchPosition = state->buckets[ hash & FAST_HASH_MASK ]; + int currentPosition = static_cast< int32_t >( inputCursor - input ); + uint32_t matchLength = 0; + uint16_t matchOffset = static_cast< uint16_t >( currentPosition - matchPosition ); + + // If we had a hit in the hash and it wasn't outside the window. + if ( matchPosition >= 0 && ( currentPosition - matchPosition ) < ( LZ_WINDOW_SIZE - 1 ) && matchOffset >= LITERALS_PER_CONTROL ) + { + const uint8_t* matchCandidate = input + matchPosition; + uint32_t lengthToEnd = static_cast< uint32_t >( inputEarlyEnd - inputCursor ); + // Here we limit the hash length to prevent overlap matches with offset less than 16 bytes + uint32_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; + + // Find how long the match is 16 bytes at a time. + while ( matchLength < maxLength ) + { + __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); + __m128i match16 = _mm_loadu_si128( reinterpret_cast( matchCandidate + matchLength ) ); + + unsigned long matchBytes; + + // Finds the number of equal bytes at the start of the 16 + _BitScanForward( &matchBytes, ( static_cast< unsigned long >( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); + + matchLength += matchBytes; + + if ( matchBytes != sizeof( __m128i ) ) + { + break; + } + } + + matchLength = matchLength < maxLength ? matchLength : maxLength; + } + + // If we have at least the minimum match length (4 bytes) + if ( matchLength >= MIN_MATCH_LENGTH ) + { + // Do we have literals to flush before the match? + if ( literalsToFlush > 0 ) + { + // Start a new control block if we need one. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + // Would be larger than compressed size, get out! + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); + + // flush the literals. + // note the xor against the data that would be read in the match. + for ( uint32_t where = 0; where < literalsToFlush; ++where ) + { + const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); + + *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); + } + + ++currentControlCount; + + literalsToFlush = 0; + + // Would be larger than compressed size, get out! + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + // Start a new control block if the previous one is full. + // Note this is done before the offset is written out - the offset + // is always written after the control block containing the first + // control in the match. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + // The match length value we are encoding. + + // Write the offset out - note the xor with the previous offset. + *reinterpret_cast< uint16_t* >( outputCursor ) = matchOffset ^ previousOffset; + + previousOffset = matchOffset; + outputCursor += sizeof( uint16_t ); + + if ( matchLength < INITIAL_MATCH_BOUND ) + { + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( matchLength + MIN_MATCH_LENGTH ) << 4 ); + + ++currentControlCount; + } + else + { + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( EXTENDED_MATCH_BOUND ) << 4 ); + + ++currentControlCount; + + size_t toEncode = matchLength - INITIAL_MATCH_BOUND; + + for ( ;; ) + { + // Check if we need to start a new control block + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + // If the encode size is greater than we can hold in a control, write out a full match length + // control, subtract full control value from the amount to encode and loop around again. + if ( toEncode >= EXTENDED_MATCH_BOUND ) + { + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( EXTENDED_MATCH_BOUND ) << 4 ); + + toEncode -= EXTENDED_MATCH_BOUND; + + ++currentControlCount; + } + else // Write out the remaining match length control. Could potentially be zero. + { + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( toEncode ) << 4 ); + + if ( currentControlBlock == 0 && toEncode == 0 ) + { + lastControlIsNop = true; + } + + ++currentControlCount; + + break; + } + } + } + + // Update the value into the hash for future matches. + SetHash( state, hash, input, inputCursor ); + + ++inputCursor; + + // Hash all the other values in the match too. + for ( const uint8_t* nextArrival = inputCursor + matchLength - 1; inputCursor < nextArrival; ++inputCursor ) + { + hash = HashFast( inputCursor ); + SetHash( state, hash, input, inputCursor ); + } + } + else + { + // One more literal to write out. + ++literalsToFlush; + + // If we have reached the maximum number of literals allowed in the control, flush them out. + if ( literalsToFlush == LITERALS_PER_CONTROL ) + { + // Check if the control block is full and we need start a new one. + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + } + + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( ( static_cast( LITERALS_PER_CONTROL - 1 ) ) << 4 ); + + ++currentControlCount; + + *reinterpret_cast< uint64_t* >( outputCursor ) = + *reinterpret_cast< const uint64_t* >( inputCursor - ( sizeof( uint64_t ) - 1 ) ) ^ + *reinterpret_cast< const uint64_t* >( ( inputCursor - ( sizeof( uint64_t ) - 1 ) ) - previousOffset ); + + outputCursor += sizeof( uint64_t ); + + //*( outputCursor++ ) = *( inputCursor - 3 ) ^ *( ( inputCursor - 3 ) - previousOffset ); + //*( outputCursor++ ) = *( inputCursor - 2 ) ^ *( ( inputCursor - 2 ) - previousOffset ); + //*( outputCursor++ ) = *( inputCursor - 1 ) ^ *( ( inputCursor - 1 ) - previousOffset ); + //*( outputCursor++ ) = *inputCursor ^ *( inputCursor - previousOffset ); + + if ( outputCursor > outputEarlyEnd ) + { + break; + } + + literalsToFlush = 0; + } + + // Update the hash with this byte + SetHash( state, hash, input, inputCursor ); + + ++inputCursor; + } + } + + // If we would create a compression output bigger than or equal to the input, just copy the input to the output and return equal size. + if ( ( ( outputCursor + literalsToFlush + ( currentControlCount == CONTROLS_PER_BLOCK ? CONTROL_BLOCK_SIZE : 0 ) ) ) >= output + inputLength - END_PADDING_LITERALS ) + { + memcpy( output, input, inputLength ); + + outputCursor = output + inputLength; + } + else + { + // Flush any remaining literals. + if ( literalsToFlush > 0 ) + { + lastControlIsNop = false; + + if ( currentControlCount == CONTROLS_PER_BLOCK ) + { + currentControlBlock = outputCursor; + outputCursor += CONTROL_BLOCK_SIZE; + + _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); + + currentControlCount = 0; + } + + currentControlBlock[ currentControlCount >> 1 ] = + ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); + + for ( uint32_t where = 0; where < literalsToFlush; ++where ) + { + const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); + + *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); + } + + ++currentControlCount; + } + + // Need to finish off shifting the final control block into the low nibble if there is no second nibble + if ( ( currentControlCount & 1 ) > 0 ) + { + currentControlBlock[ currentControlCount >> 1 ] >>= 4; + } + + if ( lastControlIsNop ) + { + outputCursor -= CONTROL_BLOCK_SIZE; + } + + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy remaining literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + } + + // Return the size of the compressed data. + return outputCursor - output; +} + + +size_t LZSSE8_Decompress( const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) +{ + const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); + uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); + + // Data was not compressible, just copy initial values + if ( outputLength == inputLength ) + { + memcpy( output, input, outputLength ); + + return inputLength; + } + + const uint8_t* inputCursor = input; + uint8_t* outputCursor = output; + + // The offset starts off as the minimum match length. We actually need it least four + // characters back because we need them to be set to xor out the literals from the match data. + size_t offset = INITIAL_OFFSET; + __m128i previousCarryHi = _mm_setzero_si128(); + + // Copy the initial literals to the output. + for ( uint32_t where = 0; where < LITERALS_PER_CONTROL; ++where ) + { + *( outputCursor++ ) = *( inputCursor++ ); + } + + // Let me be clear, I am usually anti-macro, but they work for this particular (very unusual) case. + // DECODE_STEP is a regular decoding step, DECODE_STEP_HALF and DECODE_STEP_END are because the compiler couldn't + // seem to remove some of the dead code where values were updated and then never used. + + // What these macros do: + // Decode a single literal run or match run for a single control nibble. + // How they do it: + // - Read the *unaligned* input (in the case of LZSSE-F - twice), it goes into both a regular variable and an SSE register, + // because it could either be literals or an offset (or nothing at all). The low byte of streamBytesRead controls how much we advance + // the input cursor. + // - Used a contived set of casts to sign extend the "read offset" control mask and then use it to mask the input word, + // which is then xor'd against the offset, for a "branchless" conditional move into the offset which + // has been carried over from the previous literal/match block. Note, this ends up doing better than a cmov on most + // modern processors. But we need to pre-xor the input offset. + // - We then load the match data from output buffer (offset back from the current output point). Unconditional load here. + // - We broadcast the "from literal" control mask from the current least significant byte of the SSE register using a shuffle epi-8 + // - We mask the literals with that SSE register wide mask. + // - The literals have been pre-xor'd with the data read in as match data, so we use an xor to branchlessly choose between the two. + // In this case, it ends up a better option than a blendv on most processors. + // - Store the block. We store all 16 bytes of the SSE register (due to some constraints in the format of the data, we won't + // go past the end of the buffer), but we may overlap this. + // - bytesOut controls how much we advance the output cursor. + // - We use 8 bit shifts to advance all the controls up to the next byte. There is some variable sized register trickery that + // x86/x64 is great for as long as we don't anger the register renamer. + +#define DECODE_STEP( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + readOffsetHalf##HILO >>= 8; \ + \ + const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( literals, fromLiteral ); \ + \ + fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + bytesOutHalf##HILO >>= 8; \ + streamBytesReadHalf##HILO >>= 8; \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_HALF( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( literals, fromLiteral ); \ + \ + fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_END( HILO, CHECKMATCH, CHECKBUFFERS ) \ + { \ + size_t inputWord = *reinterpret_cast( inputCursor ); \ + __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ + \ + offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ + \ + const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ + \ + if ( CHECKMATCH && matchPointer < output ) \ + goto MATCH_UNDERFLOW; \ + \ + __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ + __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ + \ + literals = _mm_and_si128( literals, fromLiteral ); \ + \ + __m128i toStore = _mm_xor_si128( matchData, literals ); \ + \ + _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ + \ + outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ + inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ + \ + if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ + goto BUFFER_END; \ + } + +#define DECODE_STEP_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Hi, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HALF_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_HALF_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Hi, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_END_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Lo, CHECKMATCH, CHECKBUFFERS ) +#define DECODE_STEP_END_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Hi, CHECKMATCH, CHECKBUFFERS ) + + __m128i nibbleMask = _mm_set1_epi8( 0xF ); + __m128i literalsPerControl = _mm_set1_epi8( static_cast< char >( LITERALS_PER_CONTROL ) ); + __m128i bytesInOutLUT = _mm_set_epi8( '\x2B', '\x2A', '\x29', '\x28', '\x27', '\x26', '\x25', '\x24', '\x88', '\x77', '\x66', '\x55', '\x44', '\x33', '\x22', '\x11' ); + + // Note, we use this block here because it allows the "fake" inputEarlyEnd/outputEarlyEnd not to cause register spills + // in the decompression loops. And yes, that did actually happen. + { + +#pragma warning ( push ) +#pragma warning ( disable : 4101 ) + + // These variables are not actually ever used in this block, because we use + // a constant conditional expression to take out the branches that would hit them. + // But unfortunately, we need them to compile. + const uint8_t* inputEarlyEnd; + uint8_t* outputEarlyEnd; + +#pragma warning ( pop ) + + // "Safe" ends to the buffer, before the input/output cursors hit these, we can loop without overflow checks. + const uint8_t* inputSafeEnd = ( input + inputLength ) - INPUT_BUFFER_SAFE; + uint8_t* outputSafeEnd = ( output + outputLength ) - OUTPUT_BUFFER_SAFE; + + // Decoding loop with offset output buffer underflow test, but no buffer overflow tests, assumed to end at a safe distance + // from overflows + while ( ( outputCursor - output ) < LZ_WINDOW_SIZE && outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) + { + // load the control block + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + + // split the control block into high and low nibbles + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will + // be an extension of the current match operation. + + // Work out the carry for the low nibbles (which will be used with the high controls to put them into + // match without offset read mode). + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + + // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save + // the calculated carry to use that byte next iteration. + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); + + previousCarryHi = carryHi; + + // We make the implicit assumption that the maximum number of literals to controls here is twice the offset size (4 vs 2), + // we are doing this here to save keeping the value around (spilling or fetching it each time) + + __m128i streamBytesLo = _mm_shuffle_epi8( bytesInOutLUT, controlLo ); + __m128i streamBytesHi = _mm_shuffle_epi8( bytesInOutLUT, controlHi ); + + // Here we're calculating the number of bytes that will be output, we are actually subtracting negative one from the control + // (handy trick where comparison result masks are negative one) if carry is not set and it is a literal. + __m128i bytesOutLo = _mm_blendv_epi8( _mm_and_si128( streamBytesLo, nibbleMask ), controlLo, shiftedCarryHi ); + __m128i bytesOutHi = _mm_blendv_epi8( _mm_and_si128( streamBytesHi, nibbleMask ), controlHi, carryLo ); + + // Calculate the number of bytes to read per control. + // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_and_si128( _mm_srli_epi32( streamBytesLo, 4 ), nibbleMask ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_and_si128( _mm_srli_epi32( streamBytesHi, 4 ), nibbleMask ) ); + + // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation + // this will essentially be ignored later on. + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + // I want 128 set bits please. + __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); + + // Masks whether we are reading literals - set if the carry is not set and these are literals. + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + // Advance the input past the control block + inputCursor += CONTROL_BLOCK_SIZE; + + { + // Pull out the bottom halves off the SSE registers from before - we want these + // things in GPRs for the more linear logic. + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_HALF_LO( true, false ); + DECODE_STEP_HALF_HI( true, false ); + } + + { + // Now the top halves. + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + + DECODE_STEP_LO( true, false ); + DECODE_STEP_HI( true, false ); + DECODE_STEP_END_LO( true, false ); + DECODE_STEP_END_HI( true, false ); + } + } + + // Decoding loop with no buffer checks, but will end at a safe distance from the end of the buffers. + // Note, when we get here we have already reached the point in the output buffer which is *past* where we can underflow + // due to a bad match offset. + while ( outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) + { + // This code is the same as the loop above, see comments there + + // load the control block + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + + // split the control block into high and low nibbles + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will + // be an extension of the current match operation. + + // Work out the carry for the low nibbles (which will be used with the high controls to put them into + // match without offset read mode). + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + + // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save + // the calculated carry to use that byte next iteration. + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); + + previousCarryHi = carryHi; + + // We make the implicit assumption that the maximum number of literals to controls here is twice the offset size (4 vs 2), + // we are doing this here to save keeping the value around (spilling or fetching it each time) + + __m128i streamBytesLo = _mm_shuffle_epi8( bytesInOutLUT, controlLo ); + __m128i streamBytesHi = _mm_shuffle_epi8( bytesInOutLUT, controlHi ); + + // Here we're calculating the number of bytes that will be output, we are actually subtracting negative one from the control + // (handy trick where comparison result masks are negative one) if carry is not set and it is a literal. + __m128i bytesOutLo = _mm_blendv_epi8( _mm_and_si128( streamBytesLo, nibbleMask ), controlLo, shiftedCarryHi ); + __m128i bytesOutHi = _mm_blendv_epi8( _mm_and_si128( streamBytesHi, nibbleMask ), controlHi, carryLo ); + + // Calculate the number of bytes to read per control. + // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_and_si128( _mm_srli_epi32( streamBytesLo, 4 ), nibbleMask ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_and_si128( _mm_srli_epi32( streamBytesHi, 4 ), nibbleMask ) ); + + // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation + // this will essentially be ignored later on. + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + // I want 128 set bits please. + __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); + + // Masks whether we are reading literals - set if the carry is not set and these are literals. + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + inputCursor += CONTROL_BLOCK_SIZE; + + { + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_HALF_LO( false, false ); + DECODE_STEP_HALF_HI( false, false ); + } + + { + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + + DECODE_STEP_LO( false, false ); + DECODE_STEP_HI( false, false ); + DECODE_STEP_END_LO( false, false ); + DECODE_STEP_END_HI( false, false ); + } + } + } + + // Decoding loop with all buffer checks. + { + const uint8_t* inputEarlyEnd; + uint8_t* outputEarlyEnd; + inputEarlyEnd = ( input + inputLength ) - END_PADDING_LITERALS; + outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; + + while ( outputCursor < outputEarlyEnd && inputCursor < inputEarlyEnd ) + { + // This code is the same as the loop above, see comments there + + // load the control block + __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); + + // split the control block into high and low nibbles + __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); + __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); + + // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will + // be an extension of the current match operation. + + // Work out the carry for the low nibbles (which will be used with the high controls to put them into + // match without offset read mode). + __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); + + // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save + // the calculated carry to use that byte next iteration. + __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); + __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); + + previousCarryHi = carryHi; + + // Note, unlike the other compressors, we are essentially doing an in register lookup table to implement the logic here. + __m128i streamBytesLo = _mm_shuffle_epi8( bytesInOutLUT, controlLo ); + __m128i streamBytesHi = _mm_shuffle_epi8( bytesInOutLUT, controlHi ); + + // Either use the value from the lookup, or in the case the carry is set, use the control value. + __m128i bytesOutLo = _mm_blendv_epi8( _mm_and_si128( streamBytesLo, nibbleMask ), controlLo, shiftedCarryHi ); + __m128i bytesOutHi = _mm_blendv_epi8( _mm_and_si128( streamBytesHi, nibbleMask ), controlHi, carryLo ); + + // Calculate the number of bytes to read per control. + // We use the value from the lookup table to . + __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_and_si128( _mm_srli_epi32( streamBytesLo, 4 ), nibbleMask ) ); + __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_and_si128( _mm_srli_epi32( streamBytesHi, 4 ), nibbleMask ) ); + + // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation + // this will essentially be ignored later on. + __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); + __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); + + // I want 128 set bits please. + __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); + + // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set + __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); + __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); + + // Masks whether we are reading literals - set if the carry is not set and these are literals. + __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); + __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); + + inputCursor += CONTROL_BLOCK_SIZE; + + if ( inputCursor > inputEarlyEnd ) + goto BUFFER_END; + + { + uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_HALF_LO( true, true ); + DECODE_STEP_HALF_HI( true, true ); + } + + { + // Now the top halves. + uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); + uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); + + uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); + uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); + + uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); + uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + + DECODE_STEP_LO( true, true ); + DECODE_STEP_HI( true, true ); + DECODE_STEP_END_LO( true, true ); + DECODE_STEP_END_HI( true, true ); + } + } + +BUFFER_END: + + // When we get here, we have either advanced the right amount on both cursors + // or something bad happened, so leave it as is, so we can tell where + // the error happened. + if ( inputCursor == inputEarlyEnd && outputCursor == outputEarlyEnd ) + { + size_t remainingLiterals = ( input + inputLength ) - inputCursor; + + // copy any trailing literals + memcpy( outputCursor, inputCursor, remainingLiterals ); + + outputCursor += remainingLiterals; + } + } + +MATCH_UNDERFLOW: + + return outputCursor - output; +} diff --git a/contrib/lzsse/lzsse8/lzsse8.h b/contrib/lzsse/lzsse8/lzsse8.h new file mode 100644 index 00000000000..dec5297a17c --- /dev/null +++ b/contrib/lzsse/lzsse8/lzsse8.h @@ -0,0 +1,117 @@ +/* +Copyright (c) 2016, Conor Stokes +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#ifndef LZSSE8_H__ +#define LZSSE8_H__ + +#pragma once + +/* LZSSE8 - x64/SSE targeted codec for better performance with lower compression ratio data/less optimal compressors. +* Supports minimum 4 byte matches, maximum 15 bytes of match per control word and 8 byte literal runs per control word. +*/ + +#ifdef __cplusplus +extern "C" +{ +#endif + +/* Re-usable parse state object for compression. */ +typedef struct LZSSE8_FastParseState LZSSE8_FastParseState; + +/* Allocate the parse state for compression - returns null on failure */ +LZSSE8_FastParseState* LZSSE8_MakeFastParseState(); + +/* De-allocate the parse state for compression */ +void LZSSE8_FreeFastParseState( LZSSE8_FastParseState* toFree ); + +/* Re-usable parse state object for compression. */ +typedef struct LZSSE8_OptimalParseState LZSSE8_OptimalParseState; + +/* Allocate the parse state for compression - returns null on failure. Note +Buffersize has to be greater or equal to any inputLength used with LZSSE2_CompressOptimalParse */ +LZSSE8_OptimalParseState* LZSSE8_MakeOptimalParseState( size_t bufferSize ); + +/* De-allocate the parse state for compression */ +void LZSSE8_FreeOptimalParseState( LZSSE8_OptimalParseState* toFree ); + + +/* "Optimal" compression routine. +* Will compress data into LZSSE8 format, uses hash BST matching to find matches and run an optimal parse (high relative memory usage). Requires SSE 4.1. +* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. +* input : Buffer containing uncompressed data to be compressed. May not be null. +* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. +* output : Buffer that will receive the compressed output. +* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, +* The compressed data should never be longer than inputLength, as in this case the data is stored raw. +* level : The compression level to use for this file 1->17, 17 delivers the highest compression, 1 delivers the least. +* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state +* concurrently. +* +* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). +*/ +size_t LZSSE8_CompressOptimalParse( LZSSE8_OptimalParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int level ); + + +/* "Fast" compression routine. +* Will compress data into LZSSE8 format, uses a simple single entry hash/greedy matching to find matches. Requires SSE 4.1. +* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. +* input : Buffer containing uncompressed data to be compressed. May not be null. +* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. +* output : Buffer that will receive the compressed output. +* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, +* The compressed data should never be longer than inputLength, as in this case the data is stored raw. +* +* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state +* concurrently. +* +* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). +*/ +size_t LZSSE8_CompressFast( LZSSE8_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength ); + +/* Decompression routine. +* This routine will decompress data in the LZSSE8 format and currently requires SSE 4.1 and is targeted at x64. +* It will perform poorly on x86 due to hunger for registers. +* input : Buffer containing compressed input block. May not be null. +* inputLength : Length of the compressed data in the input buffer - note, this should be under 2GB +* output : Buffer that will received the de-compressed output. Note, that this needs to be at least outputLength long. +* May not be null. +* outputLength : The length of the compressed output - note, this should be under 2GB +* +* Provided that input and output are valid pointers to buffers of at least their specified size, this routine +* should be memory safe - both match pointer checks and input/output buffer checks exist. +* +* Returns the size of the decompressed data, which will be less than outputLength in the event of an error (number of bytes +* will indicate where in the output stream the error occured). +* +* Note that this data is not hash verified, errors that occur are either from a misformed stream or bad buffer sizes. +* Remember, corrupt data can still be valid to decompress. +*/ +size_t LZSSE8_Decompress( const void* input, size_t inputLength, void* output, size_t outputLength ); + +#ifdef __cplusplus +} +#endif + +#endif /* -- LZSSE8_H__ */ diff --git a/contrib/lzsse/lzsse8/lzsse8_platform.h b/contrib/lzsse/lzsse8/lzsse8_platform.h new file mode 100644 index 00000000000..5ad2e0a8853 --- /dev/null +++ b/contrib/lzsse/lzsse8/lzsse8_platform.h @@ -0,0 +1,73 @@ +/* +Copyright (c) 2016, Brian Marshall +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +*/ + +#ifndef LZSSE8_PLATFORM_H__ +#define LZSSE8_PLATFORM_H__ + +#pragma once + +/* + Compiler/Platform detection based on the table from: + https://blogs.msdn.microsoft.com/vcblog/2015/12/04/clang-with-microsoft-codegen-in-vs-2015-update-1/ +*/ + +#ifdef _MSC_VER + +/* + Microsoft Visual Studio Support. + C1xx/C2, Clang/C2 and Clang/LLVM all support the Microsoft header files and _BitScanForward + + Note: if you receive errors with the intrinsics make sure that you have SSE4.1 support enabled. + For example with Clang include "-msse4.1" on the command line +*/ +#include + +#else /* _MSC_VER */ + +#ifdef __GNUC__ + +/* + GCC +*/ + +/* + Note: including just would be sufficient, but including x86intrin is a better match to intrin.h on Visual Studio as + both include all intrinsics for the enabled processor, rather than just SSE4.1. +*/ +#include +/* _BitScanForward is Visual Studio specific. */ +#define _BitScanForward(x, m) *(x) = __builtin_ctz(m) + +#else + +/* +If you hit the error below, then add detection for your compiler/platform to this header file. +*/ +#error Platform not supported + +#endif /* __GNUC__ */ +#endif /* _MSC_VER */ + +#endif /* -- LZSSE8_PLATFORM_H__ */ diff --git a/contrib/lzsse/premake4.exe b/contrib/lzsse/premake4.exe new file mode 100644 index 0000000000000000000000000000000000000000..50f4c86b528f881a6e56a76228e0fcd32ad512c7 GIT binary patch literal 518144 zcmeFaeSDME`S_nUX&WFk0RvR2TC{4lh}EJ}gP0bQ7LbxsNvKHCsVs|3863ig8J|O{ZAj+r6)dG>UeO(A17@s zxcHBgE{T6_RmF;x*L`*66<@Eo@``J(y)Ietm8&aOrmn5{+O-w4=C@XS{kp5Jo-%s$ z$jV&Pzcv?NWPNb%^8Dw$t$$kn7vA67`oZ#F>+e^V*YSPcxFgFC=yFq+|B!F>U#~2G zTjzhc`~&@N>F)(!yD~0q{huI@gdC2G3yK_5pP73}zRW&HVZo>ZkHfJXOnhH3;c)a^ zT;Oo{c<2=D1nuWShohJ``IrB81l>0BNOL-Z8+AdcCzbX0;FlyiX8pRv5z$X>$3D9F zw|_pjqce}3f6E+7=>Ol6qvr@XAOBD7<*H0cUcENS^Lf`W9)U~7Y7cd=(F%v-@+m8? zx*~an!_i(xri}dpzE=nihW-UX$P~MnWBGZccb7OE?{r`{uA7Egic_5Z>Bnq-Aw;EaPa2Pv0 zW`8U@&fPiF(bb(gLDH`~(cy^2eK*Z?$lIeQNGV^UaFOM;>t&C3INEBv+k#7%Y^S}} zXv|#imPTXhkuw|)o(f}TpZW#Q#*Q*`Z^F?U&5rZfJscNPUfOSsPo@2sx@@TR(bB-Z zgKf7j)~!c@A)E4K$N8vl_8op};-SNDB|V9=1h&bgK+CiybOt~{%15ahMap9kc`dK!cZENf5PWtWSNL2k681#6o!rE?g-azdi+tm+g z#r(SsK`g!*SmSpKLdKN?MkG6~LZC^x%{>V>RgF^}nGV;dzle6dlyqfZvWm0g{1jN4 zhiu~k#oP@+)oLyQs;dAcekU0T;QBosG8w&Hmy>!oi5dMwS7p-GccifQkJ=1M`UgYh%ozh*qQ=Z+K7i6B8&Qn`r#HHplX5XhjS=yc&yR7azl9(DnqdREBT7Edx@914XnY03{tqP9*L5g)oN@JlBNG~c7AiIM{TcEt%BBXa!h~kgYmdtjaC@#j zCt z-Lv0y_(kKLf);7BDc7dYZqqJ!2L+Apsk+mjF;3|;S0HMaSUJ>b%2nHKoo;lv94TKW z?2ZTsHGsf$yk{(P<;3c~vixjl*b$GluO}QcTLp&iodT6K7058ukD&pxpYAgA-7N^n z$+Q%sP`V_&)TRClXxrMBwwv2+IXEb_FQEx7wG)&cH&u6U%k4*~5{q|2NYYgyZ!|tk zZa3xuv*mV6t4$e&i=*nyar740?!84sMQc>u56KEGcsJcbf(UT+MrFR=KumQ3JJMB? zOX;N00Gx7-2G|;1-5cTUL;3>BL^i5^N=E!1po*$S9uoyob%%UAqw4Qtq;V;90y=^D z|1z63cGN(FXD7als;wvkGc;iK2M@p9HZj|Fa)0Wk=uFk=7hv~#Z*7HKpvm^dLiULx zQFU`(dsrU~S9Fu|po6e!!w~jOfQ-fO%OUK=3IB?)@uXtI|%(J7ZDS3*aMkUMC{k+GrHSvQQC5!D|tqHOC4ic>5SbU$(XwVrR znZGDF0(+-|t$5WYLF5rKv#A47)x?9KN@PyFM(6#NBoTi5j2)H31lj|+lIK%WsN5ff zqv7Xv_4}#A*m?TVfg9}fBNOmVfk}N{`^z^9F#ww%AZf< z&&TqoPyT!)e?F8y|ByciBAL*R_%r~HC=34O4NZI+FanjaBU0O~B~957Xts7QCt(Oe z^r$D(_2pCO01p0C>>u7CO9cE!s`}#}-M@M<0FsfEU zS~9d`X-r;!uJs=FCDJ3Nvl0%YzbJKe?dzm0!gh-)CxS;L!joeKiDg!UEe@`uYAj+S zk7+_I2z%d;*vO?t)fY&CnXN+mzd-Rz8#m}SrXxKb4O*ws#py@2vXxc_Yd=A4V@HL# zeQ?yjJbT!vJ251GOTw^GW3olH5N=g3BCLS3u+{8WPwQv1&u$r#X%S!r%ne{Uq8jX8 zc^MqeW-TM80p}z=jjie|k!U#$%M=6`s}pn3KvGn#OpUM`j94!{IYgi|fAM*m6z#q) z;zcVk}xa&E1PyF%eUN7Bs6-#i>Wc4pP%KO)feo1T&FlvuXP>r z-r7ZR<49p@MCTJ<`#;3&Q_n)=&mrf{dG0L*)6Ys)OusByKD{zoHvObjap&~~mC4dg-AQ@6Ac^K) zvK@mYO$uSr<`d*=)9}+eS`XCL^7DXFYFz9Oz4q?Ci*%m2~YaJ}YcDg%qt4 z&TexaHZ^=R13wVRytgvutX&yV)e=*WGTfty?&nj6SimiV{}l;CEP#23F$3}n7=|AD zF1UJdet%nA;5g@Xy9d;Z`PsA|2&BoWopqOaA0Cr9tKB>6&k$Y8b-`bf&dn1V?EwXL zB@4G4Ti;pdNd4Jzch;SiD&3hUl-Yq@YyOcawDJAeEf83%xieywBnpk4g)sQ9G$(nG zIRA)d9}m; z8xJC0eiXEVVCf{9-{Q^9)8>qzDWaVJ*8JN?hRMet&@A@DF!`97Jcz><+jOjbJ!XDU zcf5nOcCu!xd}V8}11oH*PsLgjS$dCaaAd6!Zp^rzkYaUZsaN8QHnZUQaIdJM4?#|A zqJY8yUAUaW$BI?B67BHjlKxM;lSAmPoS{dvQ~?K(col7^22`7E&{%IG4RXN6YO}Gp zdwdYAUq_?y6*r6Bv=gP1J$K(LjXdMXbi_B2tS*3z;G3V3kSGwl^kLk6#)Qc$5s4uZy*!pqD^9wi&WC zK8w;%$?(Nk#{U(+m;~DTt2_&CNdGg>d1yTX+?RmI43E?8094YIa9Dn7%?5%YL|MlQ zRL>qWh@umTH%q@qO~X;+yh5N#0j<@t)N0W8{6Eew-(Ba6gavG55d<@dtI@*?Q80 z@yB^%)IXJ+JvwKC)bfFLl5dyfNbgEtBM82@l zK9kiDYl+lIslJu!mRkhCJwV^qrki>Xrf2Se%L3k`rQYjRgB+_(HaG%k_R)-d@8!D@ zAyl{C)3MSF`ApYVEf7=Apf$Slvoll2OlvAgl^WfJG`tYD4ndwN_&%bpt?kJcAVP=g+UgeO+Aa`Dd>K=PfM~zQ zFQFLzJG6zdLnPM;QFRE+iyeGSG=3SmX)SToU0N?=aLT~M(7xa_dxBH}=73l-s8@9_ z(%S5cs@rZ98t59xW?AHz{n^^6Iv!Bsab!bIZmzen+LcXt#k&wyr%+n!pr{J{nHe*C z;djjz;)OUA&ABulLT&S0NF8&p`YCT>3ipG%I!upqTh)2MFD~3EGLE)D-?~ZFeKF^! zQT0bKb9nE>?w~f94&krY_GVCgJNd56g>K^pcd>19nPWR|D0U<(&3PW&e(Mf+A8sfK z?Aq`yw9@mA%jj`CpGmv73Z|EAhe$m!^9G-Bq+s3gfi;yI%FJ26^Wny!aN2D)mYcJF zGhlT4?9TMC%#bI$!j-KNG4}l>niX>CZnMcBXsSvTnZc$d+s%5H=`e#Hohkiv9qO0} zpj+486n6By=Q8JbjGjK{^MN^TZ$ppOv9%A$9O&y9jcrM`^BZn+rxlNB;T1TXjBPyf z5e1U9fj(%F(FOh0N6ethbep|B`&>r1+xu`!iE~%rbKQZrxyL%g4El`j%i)rF?o9n6 zbGF}{?J`H0vps2dZ_v#elC?jL8Qo1eNe0SP{Cjf^>t+taqmy+(y{#v!xvs zC}u`{X2YWNoJIE7?Rv=f%yi7VX4r2wy8}(`mCkgNJJ;Ae z&!a(>GmuKQG4RZW2I-y3VNXxL%h?mywxOnLcgGYnTwWb^x0xhW#$M>8*BTEQzx3izRtD9j?{SZwQ7~Q!q=jDmq}&+ z(3(OhaPfS4kcEaUN}acm7WoF^oE1*a@Ja7hG(V2LF>9! z)pjkFtz~A%te%fXTJ3yqd_Rld!fG|=`ON67o_9tDo>?=(+*aKqHJs^=U|lr025$Y{ zb?l>cjB~UlN1EYaAUtceWu3qjTF06l5#gP1#40sAF42#dSWe>wJu3)?n@yh;)quz9{Za57QJ=%V=DtrcXU~GFq0$m|FcZ z=vw1Whx#(OkfO2PP`}Kv`fQ@mMwLBXIfwTkl*ijpM%?Bz#{QzDZ}#k@#|-<-TW-IT zx>gYYHM!ZB5KrzgfIrB5$T7Gon4{JFpr8K&@!RZ1HngB$n z9pw9-T(>+73wpevlphScSx^_Sn#4)l!444PDEbJZW~aY<6lv&lU}Q;ZU_m<6 zU)@-NN47r_f9}*EPRbv}G{nY{Y&*JQzTb?w11+K}THH%>^N;qd?XC-Xyx-|I=h*nf z=yDhhZeL#BQOHh85_Rof8Z!zq=H0?u~vU>Z!&FfbM zj!ejBPionZWvHG(o)*`+NBrh6H*Fz-9c&*ujQZTT7&o##KV&S)VMG2moip$?x)RdQ z>6{-!esuPq6jn~>AWnz6n&!Y)JzQ;In&pg5l{cAlaRjocj3sKE{vT+Pr zZKVY44Q`Dip?!9{=lEp8>RMKK@38gOAPv8Kh-r$tA<6$BsdB_C2EyPz(%$6cmv1Y^A=hAfBjjBM{E%)57$i z7N-3}yror_+wL@5qvr1)R_EX3uU!pO8DOUmsc~3~fpAr7bbJvaII6;pG9W$4zdQ(r_O!K0u`|V})JAX^o!^(PNAucf zS)519fE+-bN0Sh-bphfAeLb9-)Kb(9(N!%uV6tq!8gkS~}qzkbHSR%xG5 zi1OuZ7(}gDG-{bYZ^|%ndAS;LFEg8_B4d`No2CwlQjP8zbbufS#J1Yqbxr+ACkQfc zVA)})qU2OI8O7R3-?CL(JG0C}XtQf~GN5sIu34zDccx%(@s%2TRz<$d1YKrKuFMFE z98VGGmeI1f`&Y~8!B3dGs5>*KuO%~QpPr9nWH4ov-tRdE&?$WW+GNW#$c_sMyN)MbvjbruV(#9C;mA zc_Z_&kWe2uwG^u=tXE3db`2~5-0jBoZr8dK%n^Egk?b;(yW8kVxnK7@bo z!@*(=0yr0;wTs@6gS%+SksOT)!~!Q(V&JhxMh!3cLtXG_8w>3%s{~!lI8dHzGL zZLD{(vF3w5ZDWl^=8MV+9egu%@bx7FVl3t&k!7v>g^xMYZ|BlVs9In?s zvys_Vz96dWZQcRhgKftkM`;z_ayU5xiSLJK@F5E+9hH~c) zYtSb)z23?(iP|WGXlQPzXoW6%p%|B`B4byU{Gd%saFfxWO_gk2fdC`f)pg)jd7!TB zA&a03e43nr{`Otd+gBj=N5@&U-C}lNSV!H?J(+qR@m4SkD;$?z;ka4h=moovV)Ijf z&Wqlb7QIPSdnjDD#?^5-jE|=-0_mD>sga_=QdshdBUD;M2nBrhPc5R&`7RhcaoskGX;duKocbcOI_<;#IO4sMp=8PEnNL+!5UZF;z7ZojgOLQB!MS+ zTr`qdaMm8n{z8o!hf_+gY|TC)_3w(eOT0Ut_=>NK zx$Rbe$Z1a=@5PQJk||NV$R-oI^cWX85R)OFahjvoAr|1)8-Z$}L-zx6Oc|L7T*X|DNn%I5v&Kz`NU`1SAFUc~E*_!7|k1;NH5k+Bh z5eBX>rbpEwKWUn0zr3EI{dT&H%p^ACS^3$`eA7d>^XrONmPFOLvSu+$w>UQ!*y-8g znK;qjX|XDb zQAH#aWN*C|TNwU+k&*GuS}%V@_n^~SmK;|foMUf`-0V)3 z&Q6V@^4;+V$hCTm^>;hA$`UK)4TbKGw8Wp2`svW!>Q3FPAIGH*7uf{==Vc|fJ1(m$ zTn>04Q}0O^XwZSylli8XV6zM5&l)X6pMh+{ZBD4nhjHRo4OrHo4J=I5@11agY;Q>3 zdHBTbEgc&d3{jHD>czb<@-gwLnmTpz3t*F$?5=Y~Zv)w~p2BG~DVJ2Hg(d^{C7ijW zt8h?LGasf29KKx6t-8LzcZl+?TC3W1rKON^G|#p=}?0pJiEt4q}Bg&u4i_ zn4`(klg4+y%MAHzvvt#w$Ftxhj3%c_&Lp`J_Q|-m8f@DCjfUbL5EoVNLUa+@_%;-W z($=!e?6nRL2~*1wvINp&q*g^v7~BGb=W(4w$~wl5lQ`r{7BAU9mcis{z%8W7Zik=} zkq0|Y*9jMYXt$g^Q(6{JE4vrWc;bj`j=X@014Ph8sRVaCsH1WKZ48BV^_Rf|APnvf z*d%j^)ZCg*@Z3%-+QF?hG7d>ubU@mz(7@T7xw^Fx6ojjg$WvnKS3=r)7a=t=rfSB3 z#!PvMxn0M!foC-Vw8?(PGHD)QJGDR*um5yOE&*7XiQV;`fhokaYF@og1K?x1`x+D| z#l9o03`ErfNLKcb*g>YLb7hT=Bc&*HoRpb{`AQ?18w=XnyqlIX1`%|EksnhFYh=*1 zKWOS7WQ5v+r~_3|^_N_4k4kTJ@F&*Y8nG$e%wn3M$x+hTvkV8_(AH+N+r|Usu>6qp zVMx%UPuNBH>%$w3|+>8=BOW%M>mYn1Q^U8+w?RT4?Sq**oQ z3)RoJ5V*1ks>nVhEo8c+L_`@vCl5&r&cjgVC-#O4fpt0``MiT_?G(ya$hg?aBC6pb zfkwT-co0#Hu=bCTvwCy%pqOfDL7|`{LUU< zAS1GgB@|?jv@Nr?pzC#S*KM#vqBy=u^VH|;FEU2;IZhdSTPmWOac45AC#TNt?GhY` zf@D`;0Is#OvdJQ&e`M+d`QyoDmDyPny8mt$;o6cI*3nyuW+WkO(vtsdk z$grlg#&6Nk6ar2Bc3pNF?f_Y`>a8+@I(6iEEPjVB_mOQM?Ef)6MAZM#EbwNNqmBNF zNw=x1SnubCZG%UQ6M9{OXkY}Sb|j;R7KzUVqikq`Cc7XpF*48>@E+o@wW$MxeQa)& z$B}Mol4*-Y;rH>X7dm8zS;H06aQUs#>P!gP)J;M5`5?6;qXwrJKIXsL$b`3v(u!bhT%s)C}S ziOG!jFkW@`V~Ay~X_BMkvu6ngu6$2(=+1V%jM2h|XaaH9hyqU>S>)X$*sk4;+iTuH zoj2*p1l{!_p%3$(D2UGj13AIijN6v}VTfHSD-j8!HPn>|iey+o{x)C{3r?Bbu3$Me zYypbM^u#aL*!&ZG18q{j%pQ?k+a`w<6+-zEv1JPYq+=Xu)NQlBs~F~ini#vBoecRqc+fwnVxanY2e*ZXKJ^%!`l zULp|-Z_xK(NXL-YISa8TGHgQ;oVsK?1CZG~8PU~d2O;EAO7i7S2Bm&f%Q(@R$x%o? zwHDIQ?j!u+As;54Z5~QP6SNGpO@$VglDhTBm~`EzSCqF;mR|j6V{3FSf2{63zXrqp zW9MJGhh#A(N6KR+oO2+Gp%zuwl9pZV%C061`|ld@yRH+-bZBaL=scZj ztbf4KF(I3JAQO5#5_jpWXFt&JSU%`m_SvH9WIBa(WZ#-#;cvPJ`A#H&K=l<(MxIFg zH?*Jz#>4Gw5UW=)HmNtbLWVrtY6aAqAHhhF1o`O^Rxz9(k3*(pxY#}Zk&p1m?|Riy(@swK}cU) zn`+G^!aV99s3*<;T7>%=17CF9sm{sQk_C-=h9Vq#%10r^AE2qIT8EUcE>pFRi#n@F3hIF{9Ee3&y%oe=_KO*xh2egP_9u zFtWW1joQ^sW?RaV7|se@-P^^}-D92D((;LSk?YOQRbg>GxGML5oym!+s}Kcpv?7@I zjtG6n?C6(9tZ{%O2G)m25pdtrw2n2DJc0G|aZJ$VAVPBP6_E?f2j7w5mqpbxf2HBv zN=LSvF%_}zVH^E$#=AKzrC?~cdJD;+>Z0js+AWb|rP&EHB)0Zf+fm*C_ zXN!g`JAv_m9NzAM1VNH+O<P1u0%k2EO}@MTSw*Fz+WgiDpm+e>8X zx6$#t^>`l<5e9U@ybsbIodKe38~S?vWT&1*S8%Xf^$S;`4=e343En07Z;+*3Yvy46 zk2If3?wQmcR!L?~Pndc@SBvV^utYl*OJ%+v)K$@`RK)clKx!1iAgtot>RbDA;LL!t z>@}m#$Hk7R8t;R31jzHclGsFJG`84HEHcJ9-2 zl&3m2O9vu$;Zo@nyTy(IQum_b=y*V{#Gip2G!(b#mFd($s?btNnoZip-RjwV(W`V( zdv%pN4;uHc6hReJVM-^8nEpV#wL*?Rl@9rZo8IS0W<#K0$Mc zgr%MsiI;0gqv~$tN+e#x8_O)M)ygeYyqCsuOCy|nS^|oYQT<46TNVbW2(=;$uZ3Q9 z%;$!YQsh`hM$jIFyp)cS#H^J2JH#J$NvpZH`t6zNCB$8I)L{|b0N}!w$;k*m8EDjrr{)sBSoe{+q~F|bjgC*%&>jSowqeddmMz4dw5m(qhAR%4{o!b< zdRY${N2n_`35nCHex?mQPv^9gi4~fkZE01XiJ0O*t#vlpzeMENmhJT}=V7ZTaM(Nl zFu_@zvWW=73TPfWMRpSWG_)KQmInS1wIv2x)b;Z+Jp}fs#mN8K-Dtl%dhl95%$pST z03_kJ6hfNV?5(E@X_%!0k&3?*bDW~@PzaQ3iCKD7V_(ZPHUktz)n&S|=|hcaC4)Ug zd-1D)9_6$^1CJ}r+x0C}E-Akib=PJnM{-qDvYFkKb7Rq_kL0bag+vmO6h}!5{CeXTZWax`Ypl z%Ng&~&Ha|RYEt7>-H=vW zTHe^S3mP{JHRa1S#pyY_p{P3X&1|+wMtdSQt3IaCmA$@9l#~lj5EXJMC1i=f;k4sq zS`ILSINb4*MAZyCFVk2s$jHJnQFS$QQv15uKNDha)Z!dkBpQX+h1Ais?~VHrnxuSg z6Dv<&We^fQ9nn?%<{(SU2!DW?!h%UHu-EdNgxt|dxg(rXVgYQWrP=)H!(CTybwIcp zcH=-UQ7Q6*a>?lyy<%{yb7@YH?2{r@ykJu&j>t=)t(|kKM#hIL93WSLG0K~JW9sL7 zHI9%lqvMb3d7TfzL0(@JB#+BZ{eetU{0*HErD-M{3|d!Fn|7{*RQNi8OqT>TaDf0X zrWLVd1<6q$J*H+*$m)%$>t2U2_Gac5KuuG1uUIqm+OHdbO}mAVrmB?kLRW(r)JxOH zbQC&>J^~a^O$PlEJwAz^<29lji7pr2ToSJ#E2{jIichmAX#moVuLt+>Ij#Jer3*ZE zM0#n8KR%nJ8JF98(MECEIl3mKXcYC4v_5{27alK3JXCFj32Z_Hwb zZ=!>!x(_;uUrr_vZqRwQ>yicJzpjBibtwsf>9*>LixrJ6Z0%tSIN`L8W2UXsWkDq1 zm}lhHNFAa$iL6m4{3V+`yti#)x3TYt(e2DFeatW5hTKjZGalcA@g%ZomsY`e?}6MN z;Oaly1H3|u>nbLLO&p(5;jn6D+)vWHruH&X3>ip|IhMyVO;vu&8$X~8vlHM*G0bc+ zcGL*<)Nv36M0}y`5n-iL-lhRXTigClB%1a(v?gWM|9!nG@>ncR{|p}?7T7Q|MsF-0*I8e+d3Pyz({auV zJ|9AySY=`q2sex67t5?DP`^g1LZ3Ins)Q)V)J1|xwD=_5c8x?F8~X}#;zWnN*r98} zi9J-ZdGcwS^tJH=LD&Z1KS+$J+n_M99zwjy#(Q``RGWB74qaa(JypM!o05sr|Ib9R zvo*6y2o+#2d__pQET)cOV%r9^4*wViMYScIc7(2N z*~sL@)SsArBC})a1|Z=5n_6V!{`PLVnJFZt9N4iMwx>>uQfv~S2*52gPUb2|Np~Gn z^;+&$TC+*sFK~)*tITCSB(PBJG@55*pkgr{YXmUKw;(8_Z`-=y>1VpW41GO3I{X}Z ze#`?9#Ne2BFg|@Zxk*2yVay01a&G5FPU;{N9wdM(Qd% z#S0!)Pw|D`TfprSLUXu&p4a;%{&b_ew`3HUYR^aR*}EUO&8YTfiL}KfEi4Ojk9!D~ zwA;LfFTF!sLB#B6z??v=!1j9cFXo;Ft>QhyoE9;IYgsyB0qe*Qn~~|g0x0%%;4q8z z*2abtFh19hX-XD1B}X--T6&AIR~**t-eNWb9acR)@MNvg{ZCZfnyFO19dz;aH*y)& zYPn?Qn3(z}Av*bgNOH-{Ar5p)}~?J{pwo@9XV7!V0uH(fb`l!Dz&~gdqt?7JRg*5kqNDNm@wa^axDrKFGHY z)|k=A%XWFIdaRW!5__!W`Q^MU=5HjeeV6sl zQEBaxrdObzN|nGv|HeG1;f=5NJufE%%OjEg+U za&4t!{=LF9nBhuoGeSi|5xJdwdmSRSQLB0htf6=b7Ax^?`X$A@x9Y2EE^>Nry-8|a z-yFy z-4{6=XS=>-UBNUox^rvxR(zI#N7xX$rB%JF_nphF3u%bNXoU0JI?2ZwjTE-BaS^wk z5DrQ4!dA6e=lKbsmj+tZ1B?$2POpLBJphy^c>I5|dN0wekj%e9tTr?6wTjS0k z&stSRH%ADrzpl~e-Sk6R$gc4j^9HiUy#8#fD&$Cy2=0_OyT;$DKBWsS$LC2i#l}1j z9;H_GF)W3Y@R&2pw7~(O>6zu~4YM=*%<>qzcV#xw0HB(ok$irM0y^ zD#T^FPk!d`YZG_Z_qp=P7xF=NJ^LqMm9`{~V_hubPr0M<13gsBjVBw>RjUQG_NSH|Xp0C5Q^0#hBv41nkuW6W?SirP9VjTx@w zF?FQ`;$ZJlP1}1cbO$gs? zD^2;4yWzatWxa{8v&sphTi9>RD1^3iXL%(0HKsg=G!8Y;!sw%gzpXB@+=rn39~s#U z*qj)hAe`>R2o;CL z$uO1HSlWu2uG`4Do(Pb_wt;edfqFmZpsxYtR*~}4L{1ioxQUd>Q6lz2J?UBB8Wmhf)_4}`=?sj%DPP#Tdt>|Ht$VUbLjGZN}D zO%|NtkdO>m&S>^$^P?L=(;7;F;&yg4&sS(cEdi)uW0}5nkxA zCIosmJeiqUz=_sJSTkfAvK*7#-qD?@&u+wVzX5B}?cCP0Z^EdZ=~;gmuzqSAmT*bP zBTOK|MeM{r;==6!SIo^oQ<>nx6H(kJ0(mdl-lC}+R!{D!ATdFRQ`QC8m!Ax#mRa12 zsZ$HAv4i<4|CBgZB=Apk_cr1D@hd^cU-=y1bBK=vG>_!d%5w$Zem-aLIgd|}&m2Bu z$eYeb6NqR_Z1ucV*c(zzvJMIF6DQVrP07lp)a5z$c#dhxgcPLYO8})?-K;k*%B+iW z1+E1&IKPHh710}~!?`O%b)dFSTOs9n%eDe56dQD9033n%o& zp_ehcNDQ5aHH;MuyW&kaoj&*~N8p=lD#fdCAIuLtF||Z*!>z)kA4LJ`>(o#FDGg0e zuX6XweJg`THq6=@Z4!X)N}SVn_%p1|+TEE&1sDS;X9Dgvet~RoZ1A;4Vu_OoPtn8m zTjkscrV}0|!3>K@09lRIxfSB5PKPUV28M`jW5(#zBoXq>tT3g2+p=_R&T~FG%R1GV zQEHdsJSVCETL0E5LdIC|jsi>> zP?~Cj=cBNQAWxs*o*-LiCBiSR-{~I2Wd0o~G(@+Jl}E3BRHN2Zd6SxEJZ6a;fw@wq zvWZh7$q7xVCJ-G}Z-Ih{m~3m@i4j({PP_^x49931j?Dsuyc;uyo4 z@>%R4m+ADgC?$P=YJ8DR^)9w2jT|jnwv(yPW{&4j@jDlFel}4DjMXzw8$T08z?t4{ zJ!%4LjWMz=cxK}}vR0^lCSA%A>}?zWD$~ovJ(PtsraF4mm{Hr(#lV~EMn+n$MY@LN zCIHqCCu^Ym_8{N33xhq|NhVvl9Gyq|vN>q>%}l5IV6ndEWgP5kZa9Xwwa>|h3;v`f z2iaz{LWF*B8Ij!R5mTdImO+t{C=T*^=AU?-H_&SIIJp{X`Cr6M%8K!^x4%J%+YIb} zGljc!ayoP{rbOjqMD*>XN@zf8@92dvxE15cU3?#5hx}PVI_C00@hu-CIM+9{yj0$ z?@=yGNs-2x5Y{{$MWfZFau+Y6FKJ3|Mr3?ABEm@M8otLWfL9$oh1@}MLTJbVxU zenJ~D)j_wsSWj!DKrpp`#FGg}shj@>fcZ#KF<6JjA>5^odE5!zQo^`>Wt)|{Tt)Fb#4 zndq84y+hLPHvL#V3K=7;?{Ghi7bW8C3mpD zxLxk@y1pk5`*_}702JmB68QaYoo@rq(6vs^C3s1)qWBu)pWGpl9 z&X&x~k&yPax`cRaopY`85=HS!-9~Ay4ZF~6yBTZjOlj9nJ9n_+2gcR6JN7@qnAzl- ztgFAkL`l(j4X-hEidTZyiNTwya~t(ER_gG)F@)#Er|IMxolJDsJ#vE?a`x`{be%DQ z3~QDdx^IhP`qy#iM|5fdsp$Q)rq4`HoSsZh$fh3DeO#q;|F+FWD|@){%XLb?ZXT+7 zDx0ki-2r`t9@m)5A?r$@gotF)_n>09(CEp0!E(ExCIh+Kh8ql*>>H#69m*==_ih1c zFbc}vF*qKPWj{@&IZeGOy(O!FaBxdjm`nEBywHVqh>eo~*rA}VXAyP4CPi)%_GgK= z9ad%xEt;-yujB)V^xn)#PaFja8-D`dX)jP!KV7IBk+k~|0($2a#7el7qsFi|FI77S=#on%VQd;8`bP-b{D8}*#F(!gg$I4WK zN}ALw&JmiQChIxn?o7-ZyMxj3Al0*}JNC;OG=UE^rXGBhrdnIoci)jxqBiGbQ}@Rs zx}OxI(*(6hzjFUEwI7UMrQcq@M&7o@SLnB=_1k^%PW?8Wl2~_%<*n*M8rIXP$$YcU zzLUZ-m8n*B!m#Y$#P87A*XOdYZB3P;F5&;@x}JwqOAt{J++zJBXuN^@qI%~|!| zqiBmDPdWKop(k`2vrH`ut`)5}4e1EI%1-u30xwAwh8u);=Ntbgc!%hABLo25%1IbH z4GU5JNb?;I>~{LM3C4z(y!knJxi9DrW2cjp*_7t1A9ly(&h`tem>Wz?MahhafNfPj zpfSc&qk%MbOdA68#33+k;ZWg_xz_w<2mbw?UNJ`DMp+yoH|%9cUQqN&HbG~iLL#6~ zB*I6Kvu0&&ztxFl%uQ}lqQ^_rbn@`>Y_f%*7#y(Q_k=lfVJ%MvZG8yRkh5``|zP1A#mv<+L8MJl_bNUBkFZf4=XEg zx<@%fsx84|X-7ESjo&TAu}&@tuThVof{^bc@NI~@V^YrBC({_$FsWu**RnUJ9-@O- zOikDQ;q~EFBt*AXja3pmFkLKevzX|iTVz1$zbR^T_(A$QoRT=8-bBW?aX+=1PmTn3 zv5w0;BmjGIn>}$KHTBJgGA+lvXvgAs?4X)x;!09$52@L33@4!MTOSVL-x2x3~YV(}ZuYK7|SbDQ+$h%vxxyFXKF4ChTJ2!~Tm~ ztQ(Jxfd}?#&hDY#JERt~$0iExC8=C7oo>^Uu^Ub*%s$ z+=iYDi)OVf?t%tDscA}8K_|5@K`DoO#BNQF!s&8^OU|0AtYW=4u*-Y*Hco>di-y7V z8)BCwI9GOAKC+4h|H=1rk^Hi%my3+rJ2!AG)*>+7%BlkD{T{wVZrp+%#B0{#PcQV; zwbgV~W~Y81UOMf5au>2*ucyGwv|Hbr;R`IPp{sRo;GM*KwRrG4CTk?^k|;L@*cXl& z{XRRS@?8uon1PFQhWSA?o5<=8$I`B@-Pg$FmAmUGP!A@Da!NalmCb_CWU-4Afv}8MNY9XT!E8tANP~_wiJdG~4 z+UVdo_6L8=6fn71X9MbRxzIITr%ny5@^cm4OgCE!T%T6Vl~Ej+T*sce&(Tq`bsYwn zhAh8865!^!_0@T0#_aN}F}ot$TkpGta=rDQJEZtIW&`7@;K+Wu9w>R9j@yriNADrq zxW1f$2h;TxTkqu68`_4mHZeW>4PU0dGNN8;(^6cIM)Q5xrEe`LwQq)ErxDKIz>PWw zbn=2+@*Lwsd%gTPlmn)l^$m7DF0nIR8#$Z$&PB#&<;ju8XUB7>?XyWK@#yA#CU%$v zp;p4`!=?dJ3EG|KuBDPI`w-?iYHP{4Zn>^RuC7XGZ==tgrit$|a|1Ul!A6sTCz5F{ zU>;b$ZkVX?o_30<`ZM8EW`kt$`E!8QgChaPVA?(
t@@HVc(0Gm@Y1VqZuM# z>kT2n)n8j5pjadw#$8((=?#1C<5eBecQ$#9Wo0={z+iH`emsP2vpWz4-Nw3IZWrvO zVOz3YgP=))b-B2uML2fv0a^1{TE?r^NTjO0DU!P@)<5K+wl@e@Oy=igv{~noV6DS@ zb#wHZCyI6a>dOfo5XDI;&nXT^C*gJ6A~`tAXf>wS#r;^^I+k)Gx#Hz?3nXs!HL%O1 zp=8xkrT)LDoVsh9BoWN~kj3 z$JAI&B^B*4^-JMlu?yuY(qF$sSGHe6IIi|3yo=BpBMUScWdvmSYNdqlAgh>Ct>$Yqr1Hm9}3765}BBgEP#a6;|)lgxoIBllTy+efNs(ptAQB0Vj4eb~R~oE3P$ zZ1Rw$#LZOJwu$`E!v{2qC=9U~Y@3BV;-b4|#pSs1-LmcBTv5fz47GI?gNE<^x_QDK^1rtk@Ej6QlYkVMDwR#-)j?B)ro@Y+V0IQ(`>sB!jzUylq*tRH@#*< zUwYPSeP+W!zPMScpRaxzkUNC(=rImdf){aOMka4rJ2$g3n@%0<%zo~@`2rYy%c42V z^lb0L&+@Z9&RyPzdjq??w^fiM{<;x(^1O2Ygs5Hb!!x-S<&CcOQtE)&pANli zynj(%U!_CJ$e!xWh}GwPxOHxNVP9aIHzQ`B_u=nJxQMwey>Oqg&sn|GKF{0a-PB2L zleg=2zRb|OX5W^ZrC`s8rIj@s33V)ROt0{EJwa|JTHt(f?c6}?{JRJ4@bH}h`S zOC4vMGd-D0amT)!DZkTn3gjoHe21OlMow`~*m4D)S@%8|NY8uMoCkj2y}uxN&7is( z)^TMGT~Dt`H3wEzt$)jE)LHx<(|>SF=co?0D~OGDY4^=Kzn9E%Dy=ZC;Zb=Fd+y%uiO$_ioz3`~2j!GEC+% zb+DeP3ov!JgpeGj>yiB>(z?zCgbi=jrFm2@ZLOy)NJJwb2kQ| zk^=;XcyF0OJ-~Es)2-Ar&F4TZoX4c$^84OQ|d)8N!aMpI#&A zXM_E@o$omBk8}hbckx}s6l2U82RyR$>DGeO`|!%y=@imNN8|+F;&|8d)z3O#OE2+Z zo9HXjxW;&P>XSnab6nm{5}+aX7=Kz#9pd%|uT8#Z0$PZrY)qYSmo_qJk+W5YI>PJ- zUGs8Zny`m8Zs#ZUE%+Sg9aR7OeHto7W)lx#Z1c$lhAw@9pzycguCbD^>tR=90=>D1OQEY@=BgCnIK|i|7qH3A zS>&J$xH@YnK{1ne6G6gZP1qmqq~Oe0<8{NcP6sc z9pd&<6@`o@EVhwcIjHZmO`T#LkGeGH_=&$rH~6{u?ifyraUsE{B3^cQ4uz)`rrtEe z70&$0D%74{{$M1w#H$JN=yTRr0G-8(%02kbJ#u4Fln2a~Dy#ZmG71Mv6`BpM!IuV? zv8^o7;$OXo2(-b}`kZ?~Z{c~e=c~_lG72p5y=`XHXU>@wn8S_OQT6mVrk3D}Orz@$ zvuogrIP=y7YzKgQoB%kd6to1U5VO)<#+JGtb6G6&L#(^xc6(`N>n=K6mJLr` zRu`VS(wl0EKc}mHs`q&Hx|xM_kjZ7fSUBrLl&}6ydLsU7tNdi~vV!o`JQr}Vg$Q0|VLXWsGp(@(C-3ac?!QgO7A}8E z2(mithKDt^+HtKz0Qvp}AWfS1L-1e}$Q*IIf3H9UB=tV&HaXfpiu+$wq&KKLs>N|O z;>-ErWFJC)7J&0K`d0|{0vDTD{CIF~xx^*k%#(3-pF#lD;R zKqPJ$LDW$ZEr{d4Uv4byi>da2&|;;zbMPiazoBU0B-u(oefDVUQ zzakxCOZ35Lx_(6_v{inE_szqkr|y@CYoLN023a1FdbeY#*uvo%1zrTWpk*e zFL8*P?Nw|IT=?^!|9s?{ao6;gMXm|gl*vxdQx^z$J~>Bzh)2+p?jQ=gg8g3I7ADOeeR*S47k8lhYmLY6Qe63(a9GRb1F2L5%*wbs=tW@0gr$Ee>o?ma=KV z)g=?+_aUG8#jP>%+j)qoeXC?Ro^)}#-Oa=j$W*k3@9LX_^;w??897BjKqMt;8I{na)`q&aKo?eig zxJeFF>N|^_NBuDia5i<#>E)%8_TR8#xmBkjRJ@A?t37S$%~);4raN@qtg z7g+l_@fE_B!exR`dmJ`gEf&KRxu5O8D_V`^YdiSa z*VaVJt-f-;Jm=#d_u5)_Zm0i2K2@c+FNnw=OW3XLZJBUN)*Pt`86WC<{4g-b{#d+7ZR(o#0n{uS8)p`F1eA*;u8IILeK45~o$S#NBbV z`K(^qqs86M9%8WE1S_ilr3oP@xQz(DYIbx+H>IXTvuCRhp?m8jB8m$LV2UQ(5s4>0 zs9Nkk*#f2v#~^j>0lb1-Fg7=Juh}>?s=L06>c)CEv({0@iAxUuRAi^#sVt#4wrK)O z-0RekLLKIItGm&I1F07q6s;bvVGhjIetsLZi3Xvq)}DRxyHU>w7p2%C@gH4&8s#a> zMS1irzd$w9d;uYfghL~@W~UY??p{JTlriMiY_hOe(LGL zF2v9XSAsga*eWBj2Z$Ofr|qrkQPLVYPPyK_?m%7^>ZO^Gtl3z>ni)^@jqG``Ek_$$ zDBqf0GhJO{KLpiRc;FmD6PcqJB9F6qbl8t`c>E`PN9tT2KM?t{X0o|~nOCP%(by4` z6KjP8sp?pI5Jb+~6yxi=w1HzX4;F+(SbG9#xqX5y9wFQ&DGe=*}RHRZkWOhVTrIn@EKPhaNv=`a>N(LzJSne zB)#yB^t^rQl^;s*#~TDC^d$%@`z!iiUgbFbuVv2VBF1w33KA36`Z!Z%t4oo4Mn@cb z+lL*P{HhQXp8R;-QuoS5IO)T2Ry0LeKYrsK=ixoE=%!ul&98?RtSOue%EWxKc=U07 zdn1;RRzLq@7^Y)fvWkaY_p*>U^VB^{bD|n@!LKcG$(e7&W9Id~y75V`SzNcsz0#eW zWA@egQXeNO7A$B{qiTV84+3gugKOlL`Tov#oasy5o97hC{or#cDj62hAV0N8u!7l$t}m2;U9QET3~a0!jin-Yfa`=R64+hl zkfaLfgqmCl7AI3lxTbL$_x}c4DZ({eC+Q-UxgzCJHHL)N#Kp+i5}vr6G*5^ar_szx zUmd7oq3Sn_Wg1FNMqkfaB+w?i9WS4);kJoN(A_O7-EU088E}%>QfBTF&%%07-DP83 z8^#1K8{=N@oS*at;o!i|b^EmkbIJB=y4g+SM^$`QALx?#GPg%3a(G;y))cguMHR-?ffp~3hJ4Gs+0Md|h`b7#8Umu{bmEia~5 zdri8%GTq*kZYSTLZl9KJFH5&iPqzot?G?F(qlt<1kV<{aYG~pySH^k-Ij} zFT!x^+U?yWlI>7^vCG@l$rGE$+28EoAyL?(rhQk2gW)Xev8zQQtPWxVw{E$#Zf8Rs z0q?V%yVFfUeh5W^hzXIWL)M-_eL1>3==;(6af!A3Bukmz^J;Kcu)LPNwFT+`*CEj; zJ<#apOY71rfP6iHI5AL; z3;cnXQ?GHmu1sfmGX4;BK>K9*9fvl4?K`SA>2h4d3qlEB*Tt$Zz9?0tmh#H__hDK8 zMr%-5T#)fnvcep@CG4+sF4!^$1ph zt((mS9jXFK!JyAvS)QyyXH=R;SR0s^dW?^J&bOQSfnp_#BE75SGOlzxxcB;gFtF&BM$B83rgnpD;)W!Ls zUtYNi#qcp?%TMigmqjGaIz#7tn_||h`tih@DD)iC2qLZbWwGZPJVy7J2y*`9Gd1?? zeMO83eko$TedT;Xs<`v=q5qqW-6RVv?1BLzMvWR24Qe#0CSAZl$R^yvhDer(5bJeaZHurA zs38QG*lZrx+E!cJ+MBKZdau1`u~kb%6BI9qR#9xF)#^!?ws@&Tq3rved7e#z_5E8u zAF|JLo^zRV&YYP!bLPx!D9?#IaiSn&EcNJrz#Gu3FIb2bp{WJ+)Guarho@tTl^^{N zj=o7RBb|j+AL#J34`hc~ch)Q^BE{vdFR6G!kdmsnS_`fw8%x=ZCL3p@^UWv$lU(j_ z{9W@WVZGRV{eyL(G3Mg$TET{j-Oz`$Z5dgc^t5GZZMRFqxTCoE+6Nhh>v& zD(;?}*Hh>3o@<9l6E31c7E7D9MCg8BW&Ugnrj_2P9$QM}-nbl60&LVoFN%p>j_C&+$yU4d0wPK|7l3X`qr8(Basn~j$W1e z+MTRAYZM*wIm8kp7$y{&@~FEp{x@gXWYo5`2pfSE%BFM%EMtheHfjM<5*tEp?#tIS z41KDwKg|VGBl_u*p-)Bnsl~`iSu3cPJ7QB<@qYFC(wHx7P)_H#4kR{}=CsD&9zL8F zo&0sBz6XMCMi#n|=iBm4ayG<1$G#`f5Gz2Tb-(m1IIpECIj7XL3egTuZ*9G%vC5Tl zZ*~PMUBPl!aE2>bie$BjyaQXbz_nzz=Fr7!TawpJ@4Pba5TP8kba`T19!#Wh2Fe)m z{nDFW1|_!Zznlq@jWZw>aK$z&wN*M&8)pK+B0J|iUE&P5Tk7sUkqZu*8G*6z?0gQpAK<0Bu{>51Hdo1^nbp`3y?i-`dpC=R+eGhO z$;O5OIoIuOoRn-_G;lMI)~b0?uC-HSljK3n2$qQdYc2u4B0F&~hWLi_%C_eQ8@9@I z5oV*Y$ikJuJcwLj_tMmyT%5cVnHs4~d=Yp9TnTqm4g)*8Ty)Ix^oiAtrLobRgRtAd zgKXG*2MC3pi4qYtWR^Gz<*?}n;wgp&g^2?$Bp~EPpwUvtRv8-U|t%+oGb{eEyk`bA1@s(OYLxctyyYum-Cij`vi2%qDYV>GBbu6`l_?+>O>ckMdZv0!X$RVvVhrE1HO&bu zC~eMt(5`vmf6GqhXoUp>y`oIm9?XOj@sp>e^dd8YtWVv&*RRm@4u%>F5 z&Dq+OL$AOo8O81MV-kZGb&St`9`|?oJcN?U&DcPRS@s@~G0Yu4mN-4n zQuC$emk37mNm6}VgxbGflkf3$-O5wuSE2=txad?uo9$#agfb;GQO0GWSZV8f#g54(>f}@4>y|rj&3zY>PexNA1};>WR{~#yq+0 zFD`ZyEC5kV%@E>o-3+AW@njp1ghW>at-JQd&$C*J`EWM8(E3mW&maeq2Nu|vGck)f zqmf-+vvrNT^wZ%Q4$u9;9y3Gq)7DH$mPQIZkzIJNCrDc&=%8F_KW=R z25}}Rm7F~yS>A8;iekVn(gJ@O;onjjJ^t!GxLweITc5ZCib&)>97Y04*gObiC=$|* z>{_WJY$ku5WUfR^8k3V_M~A?2um}YWinL@tV9JdMZoSbCG*uoaCaD|2elKj2B&IbR zCW)I2dY7t}%^i1H1lIr9<+Mw7Z+0R1+u!~&@>{O`)iYf5QZ`l8KceUzLq*w^4}G)u zhGDawuQ-bT4KsnG_`9x`;R~CG@6;J^iHEy;zeDdU@Bbk=K&@{1Fo6dzy;5CJ2IJo+cHM27wsZy={tvmJZ?cW-J)7X zMyW*NHB5|-jTBKFH)xCv(}R~Zc`b88Md(>jn;oL{p|MNTkyZ+fj^?z+d{TIIVU-kcj&;M1w>Vj7B zr4;}J@5l+tDgjBIh?sV}$Z$Mzdq@LYnEwa^64hDg;uX~fqh-I({i4iOW*3`Mroz@3 zn}aNPwI}eHN!|2fRAlv!YjlK9FJL%b_&0QoNsxmV z^CGCmbt#1>OA7y~UFzD_v8tIO(67fRkUicz4NC_$QFMk-r-DKo%?EivwDE zeApbz6iey9YnMg6?$FiQU=?$-VmRf?bc+<>cM8eS75%VzF+tm$prw{)V4&v1mzlj* z*r8Zlybc_q4kdIV7yZJG6ok4&WwSB=}3q{zt0T3~p2gHz7ZuZ@0ljVUf zkrt%tuTU9{(k4_Oue0im*hIY=%BL&71YC8f_PxGGYH}0iGX&o>V`CK!4e^*<9Ekjs zE)drW`&aT{*r*<%>Ik0BsgxTR_@@C!n!C8^skAH%fg@Xqeua+^UhLIMpNV>S81-bA z8h2`J8`JX}G6T@@c86|(8>PLG$7m0@o@SdvgXyVkz9)zSf)22bTW(70yk(5CxIE4- zRGX-O`X&G`w@Wb>(>l(Llppy6b%*1>L(7)_Eh|uT zBi`^3(X3*$?1lOdT_+r*(!nX2%7<@(-9>Iua0tYp%fLIdfo>T=EZ6W2JBjnm&rr-E z=H(#fovJ=tJf=Z`brzpMzpjh+rbZSy^`eUaY&nFrSNw=-kIPou{}*9$P$&5KNiV&bTPjIFJa!%|f#{ScC%4j|83_0uPY>u7j5m6FI~;iixsbj*ZU)k09I;gKB5)u)`Gb7vq2 z*a74M@A~TBZ3&0W`@hD2kA4Q2vOO_(pf9#7oBvqyL&0N{SeF{B&YaIMDeWCqA;+Gw znU@PStG)wyw+}rx@An@CsvJq#^{FltdjCsSMYe@IK}D|fsA$<@zV)=Mg`)KNSc`oZ zY?8~3v4YO_b24LM;au%ywqV;)1uxfv(5U$1*$gWc$-|~=6U^WVR+z+GJ-7#;ZIFM0 zzD7W{wDb5xv*WjAi=RMooyuo~QjH)6pg!CR;v``PC?ga08IoD*`8E=YqP%7% zLE4eqo`_BAS>|uik3hhw*fwOc(}ZTXizob20jxqdp$w`GMsoayT#$QhPApD-G; zvalsSq=-0n_{1V(hKsC-pVDIbWf#WDtwK#aZ+{b{`eIw$)Lk)727O!3WEm;MrTRc; zj^p7I8~YZP>tY)!V;=v*z+33axpv3j)`7kJwek~Yrf*9f%pR@d8decM-TWlX@xCo5 z>BA|$E%(tcSlm62Qklp|dlNLxePo>M}YtGUWGqi z5It+)bBrFa5RybOXsck#z}K|CDwGnVi z_G6#Cu-^Hac4}#IvN_Lbv^O!kE%p$Yky-7|lyfJH8ufy0&z$@%gwygpuq`h~03YCw zSpRvp;u|<;;1zlOBtUPn+Y{876)T8#46G1@_@q*c>{46BvS?s6e~2U{{lhcm6xlXF zMwv@!)mCC+<_v`RBU{yi=!$_u%u7U4BizZNoRlWxQBAIww_%F2_Nb2^M~)ui6x~5^ z>Ba1UJ80z-OWkaj+DZhyoB=jP957dz>+HnntmVncfnC0?+y56VpXt;j=P~KaKC$6( zM}s?a;7zojpIGW8_7Ze*tL$Qule4D>6?+MCQiL#L-eRet8PONv40p|!k@XX>q_ZKf zq_(>uKXGVM`*Sw^Pwc;Nn5&Ci4%t+5xoF6$uR(2z&CyRo#(x}hD-p$c(W|YQ^_AvR z#ZqyJbwKo#DeDa|Ch@_j^(cJ8=1VLjtA3`1F)KTUivU$T>_jesPD{}bsZWGmm%PmT zqq43nWBf8mdLI;8gR+prm{hQ&+QSi93ia`Ud-#|1w57Q%RYb$TQu9bI ze`gsh7PNYcC0Zi3AIV@9l3!nN-dS7}rz*MA8LScV-B{TlERi;_D9n^MnQ7w%!kP41 zY(-{r8>HlaK`A-QQIyWMkAWy~V6zC(0TWyR}$BB*d7f z^CB0^(66pEo2JUpc+&-z`~Jf9!JMN86sr4@?h`)|3!AcV4e@Wz(huv2lv*7ri!A^k zWmvD!Rb^Pu>BEZHhQ#cZu}`BH8zYqdvj45kKYiI5i}d%9)uaq@I=-qLOy);ggMZ2J{t5v8zt=l&|KC#Db+15o*eLO{@D0dRW|ZyV?%e>!YonT6+bJb??l;W6k zUMPwZWg)c-Uqc!wHy0{?76TH|DB&QpwM#aZ3O$fEEB?1Oe|8(7iM$iwe#sW`u-qJ0 zyzI>4rE(n+{jSoKi(}PN{R_PO4_#`^RzFZnfltJr6SLV9A6~hsJ~o>O+`cWNrClW8 zFO4@uJp*hha-szTeewf>T|iDQV7^0i;o0jx(X$hbC+ zU24v@D~tag*RQldd{kxOer4f5gH(c*nAOVpGrFd;qhpE4IXaym9hV-(0dZ*HyP&xi zfgs5uh#Xe*T5Fd?XsO1%S}U2_R>(vw)OtP<%*Q!_W1gX|$;mADgc%jGa%C^Sn|jq& z(VHA;5@D^M3E{8d40~O}<_-_dZCZm-Q=}4LxRf=h-N16CIaNHG3m{?xHh~B%=Lxe~ zg{%q@Od{~N1ngaQnvv@_`n z8Gr?!SVp&DkJ)8po)F*qxcE@kA9&t3kKM7e%`?B-3$nyLJ&p6a7t5(;ftBNXgeWD^ zvz8;NdAhOgFx%fAu@0Y_EFBehIwi7Dqh~Rm5~VRB!g1slowwgZ?_slcXKfusGu=dc zXRiHAP7c2!fP8n2tqygL2JC!gPQI7l7|OTd`kO=b?YZt_`f-_)Z})J!_kCQxfRpcK=D~rl`{VK*dHL0$d{rOGr{|m7?R@s2Kj0L4>`$^jo$7NxQcO=7Z*g*7G~E4{ z@pm=igvYM&awp&1-w%QD%a7z!FlLSf<7}tUIObpoj2n(FhF(8aH|WGqJ#wSn*@p(i zF*h-J@x1b4r?^ z)lZdg3TAW7__$oZm0aUSS}nICeUY>>PV45ZIU^xywQf>-*J++?wbK2eiqE3rSM#cDq)dAm*2RN5 zK3*!f?8+G(`}hVer@h$j0ossR`bXhD%-Ya?EfPY=h}@-$|5MQdQ(aySb=yn6#eA9y zBPAq=QvNv*TQVGYyk5AqcC^Jzku0T7mYLd70sSxt%ZE#gK{1wq9iQ zaaiP6+ab#r*wKDW;F)%=>*0Z&Xax7me5A)l-Koma%L4~|iw;EpQ?VNdY0D4B3j;gW zpJ4U*eobb2Rwcg0Yak!_^xj)ad z98ONmM@B%kT*|qZt0r9*HoIkAa>BQV@p-VS6Hj+Xa<|@&pJ!iPFPikf@ma`#ca`avK^s5iZPnwp@ajMeJ&(>iI&B;xKZ^SbhBtEvt)P`?$eA#FDy@;o@{~ zPvkxtM~FIeL`^61zK4_J5NbYxuG!Lx-aQCub#23;zf~D**LRcDHyAxDY|iS*mWebI zOP5;iWzm6k+8DV(jQ125yvRP{o;G{3T}j?m(lQYI35ZlqiV<#aRBSX!&jCzpYjZfV z6`)(pn-Qs%^W_@uTx2l{%MtC~yD2!I0k+EoPowWhX@^#5}BpjKp4gQTGW!MJFL^8aPBrJNB2Dw?*?tMa(rT=y*dUk9`(ys=ZP&m@5<=1GF#mvKJ zC@Qv5+%dVtF4(FCizO<~G^yxehD!CpW2AGhk&77NRM-zWz#Jf*j7XOV4SSW-|F2xb z19!5hS;B@gX2&>HAJGvhXGFX@Qdg6cp6l~hFSVQhh2|;Laf%;D;aMymfx}CEnzKSn zaqH09Zj0w5@%^tzqcS@l25-dd5O>2=(t<<_p5e5#+J;8Pb)IzSed@)GA|fG%amv9w z%aYp`D-1^#G0lXx)FiJO1q(zsNRO%TZhIX^b}+V*zVkCS#6G(#)J-j;%~z>kObASh z2L|Yeiu!&jQBFx?-bj8sCh7wqLh+Ai>8G|F=El`D2F)=-DVpDAwM|pY`QdOiK?>~0)oitB7 z^!d*+7Wm~KG+$%<)PfLN%fQfeq3U^85FgN_3zmMWRDbMavV5mUT1Ai;#5pT>=DGSLRr=-Iwl=T1ajijAl%-} zq($VE=*G4e`NN`^Nd~$yZ5omzeLZH6%nxR48`AtEZ~gjON@avsGV(Y zNt~8r{ssO?hKXeONFxG>e7cO`WL4Zq4$KSGU|h+zOpo;7HJeSq+CT<-2%mD2BC7#c z@;qnLWi4kfZ-s&TaNr#%bmx;bEXcX68^Jy%(&NZ# zJ3|v0DsX`fxm`i3@HdJDq-C2pD@0g_P2lYXW}7`0OIiR45QKkcX4R9!XmAr(5Z<05y_A1F37AI{7W zT&zP0?RKck*JPyRs(Fz>klr{1frzN)vLx?ukLZ$#pT_;?9Y9PE$|A1{uI=Lw`00f= z^EL8@{rVpolcu&Z^w3+VKZ7I7inMV$d(OGb=KT_$bNHxA``Pv05%|y_@QIYXf-N+WiuV33EP z1Z8oC{+;&Wj>}8=D2fz-#M{i41atM^f2qF0KP=vLJc7C30s%X6FXbQEtiX;jU0( zU_#*K$X%p|ef2M!?~ofYvvX!ePOJ#?Tk~!5YJFH4>Nnh&ll%p(HS$9#XKv7!tXwK) zBRhIOa3%h7Vr+C`AJ-N<^Pk_RG)(W8r`>a&8>r!)#oB|34?Jse>bP=y%Kf;99wvGR z#}0*kO+1nLog77ruO&~f@dO@S`@Zji&Tfc<6B{;F=NIxRokv)f!O~U|`zwd%SS)v# zXLXpPa$>l@ZWp!Ym~BFsrZt;u8A#m{Jtc++DZZr|592T))i{`3{3`zV%C<|~nQzR{ z7Lxf#c|eK!R|WdS$vF{}eytO{@~K>+6fg1gH0SyrL9sr~(a)RNzAb;HUG(bTm7j*# zRS6@pvyCfO*jM+$v{c;--OK*iSNCF0Xn;Xy9qilM&G;tjUUet>+{ySpjCFi<`RRRp zN#Xc(C_Qst_s?U z6+bT7F3i*}0Dt;Q+8Aztoq4``4zvqNdAs0{1DH`-QXIf=wQgk5zPguldh)yL_N40e z44lHcjohv{GZz3^4)I-t5{gbs^bz|wzRT=mStRS8l7;5$+RPfm`Fea8qkjz9E%7Vm zRVQ7kU@o0J{ZbNzmK0vocc9|wTwE5a7bC0Y0Ef=z*6aCflQ+5T!i}gop1@p>M3Pv! zof}bQeX-LuuDOfQ=2mXmD7EBI7coS+1l=B`LAPBPdRl#H*U(13sYMiAf^!pIrhd$J z>kgxTJRwl$iBog>svb}02OF@${a|@~Mil*{o%zo1EBY{!+hinr&4~mScR${_7Gu4- z!;ENtPYynk)H;%jl-uy8PoORYOWy-b?8fgfS`fuF-L2SGl}lh&4FK*Kpo&`?5Y?P> zgPhunzn=Z4i?umueH6A1O;__T1y^Od9EJlGbA*tE(65l$hK5-?`HhQgWm~cBS~#0) zvqSGfCibRtxuHh$e_Hp@_CnMPEHYBfFF(`zioxZ8g_263H&{n3~eR$vaAT?!4fnkYE~`+xm2#X+muR?#ISjTwPP=wU$L;G;ydsEma^a~A|AfuQIqwaS?iAc5OtM( zUY%L%iFozfOZv?lDb%+oc*CbFA<>HT{Xu=lu~tLgwW6?ylxXtZnp{M->H7X9%$YEA4Em$aM4G3Osb{aXiq2asWVlX|U)L}c8;ARx@!v(&|e@}iTJtlMMa^6s^# z3G!{O;Jw+NC*16s7G4pe@%hv2Ox8Tf&|psyPJm1$?>gRR%G_aXTeshogBM83adt6W z#!#{JPa??1ECwDo!mGWGG_ui?6u9>amu5ZOK?5+KP}`;Pcl!xB@4#U zjUFP=mM@XMHKf>>*w84ogqxZOk*Znk^N};y37NI8(?L=9S%cO)rWL8lTtZSycquV2 zps?&xouh+q*|n>-R~fy~7l&%y!PY5qUl%fBIaDD_X)dL<7Q*A)$p***Oy#3b?C#n7onAyzD}uP{j20ecAIz_U84MNI3;Tu(e( zZ)Bg6seI6wp-KN0k{#J4fUdq-ln5U4ZJHf%oPiP=s--sBH=8?>qP=N0OK#SBR;5pg=Cs!P8a@b< z`{5q#ELwhCO4nydxsvQ`%IwWK5~3}ZtuD`5buPLKIUGK>Y`gEbTY^7jeO$B4%f0c2 zB5sK%=Z$eMjPG+!pcDIRKC;jR>=cgZs`8U{+uHLIRikljTbvXMcg$XCGCBiu$rEhK z7`r%C_f&G@V^BudBpKtUi0~aBPon5;qmXhzUdI_BGwAzmO=a1Rzy}=@BHWbap2WIx z^C63fTzGZCs7-2ueCxzkM`yK|JV>4hU}zMPXQ=`XY$5u*5emE(o7+q)DTEplIa}dU zb9vdJ+zq8-ayF+lvG~1ALt^nDVX^m1flPc`f885ihNH**?C!+p{f0w8CFq&m+mH`Jni>a2Wz zI>yf`jfGKs_H3AG-l87* zF82sxrK~!4WJ|Zq)M}IxbdORGvUL-Z_!8v+jr&kk6Fr^#NfhOvh`&yyjqY$n>|P=Q zY71VCj8C)tgTKuwZJ7Gb=6pmY#PW)#QCpcCKR+}$h=g)4_4@LPz1I3dPC_Pkc8tm4 zqSXVz(bjom_i%jiPyUaK%#*1%j{_qMU~dd3VT2oj@PwkL5dV0S0!l#h1dK+aL=pd8 z)5H^YcGDv^DRcaIK58;wRJDF~+2LzK!kk;H%dJ(V7}+n710o#PjE>_R(jTbz#Pe|^ z+v=Z-tH`0Z!nxLHjPLhgPJe8ZgzQ~ciWAN+ci7zIaF;+dm7R7Vjd!S=%SF2mEFfG z9nx*sR9!>17c?r7&mV&D^84hAQqBpLO#-al zFQ>}bZg;^|8A|)Zq3hC zyy0@I>~emq7RprV`@_O=DY$IevPtWwT+{2tm2H)mt`&vNJFgFQ?Q|*Rd!Vs_YyZAK z;0Y&K_212v+RYV*&DP=%@3Fu5!Ahj`F?_B6=wQ#L5Cdtw(6WF_7@M67hCv$ zLzR=aP5~*{{SM|vU9H$ZBKjybgcN?0C1CFdaZsl6|20F){b;2C6{>+U@rA55JYhan zu`Lt|MaMLog*O81lNmb_IAKTddf0+Bt_QzU#V?>H zFmpDt{BFb(s#31f#3uhn-xe`pOKf~0Cw3N_d$?dRnG#pLK-%EBa2`Cm?vI{J$Yrd1 zo6WN~=|qW3WlFDzoxXXK4lG{nC>%BxHyP3=ujGEy?}r=mATH-^othh)GwQ}si8mRof%BnHRD&P)uB$Ke>4 z?ros2Ldz%j>L7t7f zdiPDSypIsv3iE}ry-D{YlHq`4c$o|uU;Pnj@bEMV@^Oy)5u0}Z;p(Bul)e074^FBQ z3*4FRxnp|foZCIu(>HfC?!pRjQPpIJ6eQ2@WK&?xsk|~%ga7(akqw&PZzALuGBP0)xywmf zrb+l)6hdR3-KA~XV%gCsE+i2|CS%zASlCwF#_RRv=5IF);dP&kh51cnXegB5>J`C? zOiR64Erbt|r%U7;F6NyqRG6i4H&tjZZp7GG>dj{~j~L0mVhgQ^rboFIb(S584;{6)lzGqryQzucr` zzu6GDphaP`nhLCyumJqc$0kLNWO7wIT!Q_`dV9XGI&Y%tR&~evo|2ibvX0R)hs`1? zQdRW#p)O%_4}Icz88Hf`#!XljhisunR(v9{$DE3fp`;Fb^duq!uamr7<@bw|f~)AJ z2o$oJdV$$t*9CE`5lK|+%fQUxbGD;qMU)Ko?TVbJFVc>5V%0;MUJY9{rLrYF`B^Ou z8eM}7Nd6M6J%Y=aEeiv3p(~LsK|1&Y0VMX93^U!KWl3|>F?D9(L9f_6UT=(*)4Fha?8&yN z41W(`Vo&xI$w%F%w@TcO zH*(U)+C_>O1z|Fx)wTzb6|^rpU3LY<^nztAF|yX;jC>px@bE6NR)A+pfuAt?u#)=H zvc3il!x|lhNp}?C=CZZ2toJ1rB77_?%!K=0;{ZKb>oLzz2K_PwKUrTYv`a4s&$(;} z7mpGpl>WAu{X3ONRS3JdpG}(#zQVu}6dgd<+TlMd_D}wPIu2ibLo(KzcRrL)ObuI5 zIr6z15nFbn0=!V|0}ukf%3~E!>Dnk&4)Ci>hroeqBnDS)niC5o%(U3KINuYs_Q@PD z9LxU+2{SI{!P~X&)iSe;BBo6LL zT19Lk+VO^%JL*eMjTWRQN5`ZmM!iJj^0G>{Hn2`(Cpn3|MTnHq)Ix;ZM;F27_S$kqH@F-i*_~~p_hYO9d3q1t^#q{V& z`Fs4^GW8|t@y&LkmoM#apl0%5YDT&O2m$;gYIDDAEFmtmFZ9GvF$sSuu!W=HskcZfAdx+(g z7@Ud0WHCq`of$UIK{o(M#O0cRtK$Sce2TCZH(_y?SywFko;O}nsF11=uLuj>n{G~& zcIV8&0byWl`+Fj)BkCjzyYU0-iXDUdKzD51QbbX|Y5a*(y=3wCjOw1Zq(bud%=PeC z+0)p7Xx|-dz{}k*A5mnr@acxzZuYVD9u-PH4c+V9G#ZgnHn;WJ%`LY@I24CG(1ml6 z__?{)V&*d`BX37Lj0j%wKH|Yv=^mY~4wj%ZS;R;{2Fpz!JX7W-Sds~5Jr(}k+HO#X z?&m_U(Ja;fn6jc-DiF<5Jtys=TVhF7JZ&y-8(K|AOC~a4;>i0DT>0t2hJU-C5wNu%PMY~8 z{kD}ReKiY4Wm5y1QA?TV0hyn2y*Ldt)m`0G7W1J#x4-XwPkNGl!#7FBu1{oiM(@;N z;DOYyp*(bTA%_X>)ZnhKx5F3lf)6Mvwo;p}RQBCsiPnTa!#Pbb^pa?55 zhzBTaGx$qZ3@t%MPT0dm9(-I`M$sBV8xFt&LHG0-5l?k{DCplG}PoUC-*AA*6j$Yt8ZJvy^A)qnv? zw6xj0%QjlCocSgE!?1z`(kKcNtDtj3L(aNSrPf38gSiz?bk5F){)t9)x!eeCHeFId zL24brC?%2yAvlGRHDGi8%<#O=940b?wMxp9@yDF$MlWL3vZ2@!pQdEJI#|paR+`>y z-cyR<2wCr?LD(_&xc`!g&)`hGH99M74)Q@K!JK{#@EowoblB_!X++FvHWNr^93CSC z75XAPAfrgQS|$=ZT+~}5hWJWQ>D{$Qse-8?uguph zb~)2iM7BWn#);RwgoeK;Q925TGE2Qo)7w%BT=tqB@*!E6tGw2kBc=#;%N)J-eVrrU zZ^^r(22xd9zRLH9-l6jhQQ@XfIMR5o5WJ5+$6yA6=Mtx6)63cIKL$3Un}_R#&F5u= zI4LmfoWrIFM#j#l?meEpJhN5pmA~6q*CN^YMY0j?Jf>D8A@IXc7&1ss=f?l!#HWPK zI7Dr6WhKJnWvm;Pe41;rqXwh~P2ET;=J)pcVpTM^hlzqFm4cQ5=ZhwAiUUjgOTmwr~$1KH5COghK)w z%n=@tKhTV&16hfG?P{sgE5-2wN`7RQA=N`--=7jYjz{evr4r9%(r(<`_O&o(6QK#p zU%e_VZwU`!kBs<(YE2UK2YNTHiCvK}H^!RIyoIF|y8wyzM6STbKa0k`UyP4(5)Et+ z8i?k8i6=+z%o3+fyNmfGGtpxH`KCZvIPegewG?}*ob(R;wuJ@II`F;h7x+2>W9J?!b zW5l%eAS{3ZChS?XU^#c-N5lq$cl)glcis`%L=PV0`@#Ub7zX3}+&CxMrfP-Q& z>PgMdISxpFe6S|xDgOVN|F`r1E&l(J|31EbkKZ%+&B-0jzfmst=sbS&3;5&7S{AAM zm+E>syy7MgSP62i=UCy;Lp%=oulsl*0*|kP#+&@7w8l>1=gio#FqiT4jMGH?=S_s@ zTrE13*7PYA?{kfK)T*$#n~I~HM;o{zMhYT1d?PAzr8_1sV`Uvc55eWl#1UVt(f5Em z-T>VX)cM!V37g9)Z5KKoyCC^EpO3ns4x46b6fX1zEN7pD(KNy_pNwoX-TrrZkuV6EVAav1^|}*tzJ_bx`<|P*27_|&2LgHwdg2u5A({2_EWOH zp}#()BK&%P*ld^N5?sw+(jSaSs@Vrg4x7~3@^{jMbS4`av5&eEkIfX9HdgJtHh^4} ztgW$`IwOK%{xuR3GP7>FboYll=umJ*G{-*3E0MI@NgEmyDZ&^K4ET0BCZB)9AVP^Z z=u70NysE3-z(;^;l$sAjK>~9nqelO^8pcR4Q{U#ts&7EYOgI{=evyQds1GHBp`QAx z?qHR*1Sc^wt$Gfr=Mhdb+2~w16xotcCf4fJ3Q#2g$$q(q+Jy=dmyknAjb+H$7N?z@ ze?`9XVGSqg!Af)OEs_gwj`3G;sR$xunFCb2MW;x>Zq6(N>8Xuq;#-7@Cr-`5`^|>F zpxYDft1mo>__=wXF}Np!FM5bao2uWyZjB&6U+5++X~*JPzdG4glWg-O+uX?$A-vJi zt<^A>Z1a*>a zi#CfZubfEEp_-g`e}YRDOZrmSW@~Jl*x0sO?+!(VVrg;&FF`K3S(j`oNjA+$Hmy!J z&Fi0Y9<}5qn`S1PYLZQnWK;Q-$vKtXP5BbgwP|{?sVLbLOE!fR{aeD3ZDg0r3alBk zsgLQ5dm_6)FHO-99ff&6Bu{^^TsR{76tkRcj6UdtOQQbZj5A43b+4vSvauN6CULi! zoDH5tkf6GpEH`@EBm{V&6UWh$S0i1X$SPo!N7-WPGSq!_sxOj_3mk7gud-R%1xoeI*8pzBF2Z%c1aVs|aoq}%3y{(M zZB~s$$r{DuDjt)Ba&wN1)ykXQW#@)T1dG3q@f1O$#fiUq!TPB(4x_ZauzC40+LOkT z%MPcJb;k-g67={m(KC{998z_RwRYmr_rrI8qN+m9eybZj9kZ>ChdKIlcN3zSVL2tW z7Qr1=f_2O(&sgp+CSA$mFIunQdOvM(s)-D>c#!?qV4%^{zJq$0QlTIi9-J38%LE4T z&P(7i4^E9&v^kQt5LE4rXrc%Iu@1w8YslN03QlmqpUSHoC1LyTO|JE(Y6$>em_%K% z#2>PUudmka?p~PJSDT+>)?Xp3&<_{wn}^DXOcIvzlmk|-<-RZJ0;7uW%YbilJk@jD z>n6@((OEJ4Fz$U*?v1Z3|Sj=11|5blhd) z`>~2w{S3)qZs%L-uCpPv?6B>XH1On_W3$dR?*>debE}=L$6eK(Yd_lQbCOydI{L8f z-L#>}acLxo35+1U4f<+A10VC z7Ovt{6$hy`w1+e_J7nT}7;YO%pw;g+LECL;K&x1@-3GSFye6eY$LByf>wkyxGcuCH z1DiPie}OVI4CN*GH~1f+to{g;Uz`8`JC-V@95w&ICHC2S%vzpC0KEG=L8jvL6x(I2 zJ&sEV4ch!E#WG?({Vh98TQ3+ICfn$B(l7+sg9f<zJ=FVW`(0YSyI|Dn>b3SH6@uy1d8JgnwYw;zC>gYIvE%32L2f3GgNybc-&a3knveF@E2})3FY9}>%rCzuosA~SUJDqx z0gCeyxo^rGSkFVJ2ykNaf`@B?*MupE%3IfOlX-DgwsBpppBir3zPLG4O>dM5iTLa> zWy1*yj2bmAW~g9wkiJ@Z;!CPODQ*_$-%_K+R^w#ONXOc8}r?AIo!^!vaF9Lr3T=da}E`2 z4Nc}}RF}D5Kx;OaoJMZvrXx3%SzqSxyb(BC$=Sz<8PpRK=I{{fX>X{O4e;?YW(OEG zk{GqFJ^ZG+#AXYwe2xxq;efzq9;o*$II=cKv}s}g|8CFEaon}fn?Eq&=3FBcpn zIXECYS2T{b;hFQ6gK6DA-(m>9g;_r;r;iP&cogLSde#9+Os=a)McZG)F_N<4UZ zjQD8Z#9f8&3!kGR0KN;Ds1I{}Mn~WXnMWIRLI$>oEQaEp9A@FKIkF<(P^NEbro!~4 zl4yPj$0q9%$y!3z^!W+%`PlK|knRArjBrdhn!9tpuD40lf*b6DlcnJCA8n&r+bGJy zRZu4|*g$4SX#+$;kmr6GXx~luf;y>`j8v2PBTQFz?aZfrT5xWyJ(H=fr^q1p13wQM zu;_RL7#y9M!k~0rjt5Tw*Ga#8%_QZF4A>=uwWoUH;dSdF zNZ!twCr*p|>yZdf44Lmlgu85u7pHFaGKAInYY@&K6CQ#yI`@gJiv0Aad6-!qpCtDK z^wuRi)CODNN}qs{lkbsj2%`yejxrWGok0zz-wW#l8xOC!byZuIz1Z50s0)*g7V{3k zVIYou#GtDk%R_g^)#8a_A$~XFCx|duJPlb*EN0A4JnZYmiNnrf>2HZ#zZeS#(?hR1 zxv>&}jFxQ_-}69sVyWaROv?rgEnu_xjg+UxXHS$5Md?4t2ZZNl)6YyZ@##-Vl`oVy zRh}SkOM#Cjua%#Oo8iVxh=~?$6q}>(oJroS0yN7ee>zuI5{kT^l0CH~<9ep4=U5X2 z8(Leti)VA6H&={hm0O4zf_*~}NMIY_1GA%Zm3l-cW427j`NEB?{Jxx%nTiz+Y*Bh# zcx$eiIVR?vIWBhg%#&m8nHR?=R%>oydYb#%eL{%m>C1E^`*Z(`hUQp}*iI(A5n%&T1EI_MRW3D@l0&aAQ5iCqwCpZm*F>(-oqNT zSz7X8A1qB(7NkoFS3uH78QMZ%a2Q&=7=cHMt$RTt$T&pi>De}svHHup zr1`*+I36jRP5w#Qc8v=x=k(uUk=>}fL~4E5-fVNLfe40mp7b)=bq5ZvQ-QIux$H>n z0yH1V#f5AM!+3DSNc*c#(H8DXyqGec%yzrBWlmGGIsR8lauAFrS&IwHb|zOAx<0UW z#{2QJp-Sf3O#1!o&OWjYJG4lQYfYHUB5OoeBTC64>OGE^h`bzWL#1^+Fw5Zt&T*#7 z7(4EJKF7l2YZ;y6MjKD}OW(L)BNTw^Jkn{2^TG4^_LqfxK~&KlpR=4qlL~T*CZuAw z?~#JCM}1o^hNvY>sc%bw0dGJP4cFKrS4oJ3c`iS4mj_FV6by_051dW*o+Y_Gy)!Sx z(Lc;(=9Mw*7n!DxKg@=LIVee{T~cUF$LV zLMmFhIQ1~Xfwc>09KS98JW)B&dGprcpWQS~j_I;u9hfr^VY@)40Rh-BUlR3Wuow6qp~`#>)0R zQje_@3a^?pbbEYYu|J;Q5FZ^(^LQ8x3kKG@*Su-HBOTgGsYJhf;FqAE+`jZ}5%Vm% z@jeWVPOu+e>t1`j?}725aFe=IR3(ccY+#&*5E+BtQ5H+G?hI**Swp}DUxCj##bmxO{opJ1ILrf%ym|s zPSmm^srpeehY@5dq2!;7X43y+cXTB;sQsbz@d70Dfy>k~itnqGC_1fI8QCD^j)s?pXTr5jXtL zw$#bN-~&6r!NlO&O&7)*5=3*V<^Zv*0k>fW=i_p8xkwnv$)5qlHS+Y2OL&UN(^h#} zB~J-?x=@}L$?WqbfMyusq(1EG0}by2;}u8Zo<=@gKcJqFbZ#J~T~A!^p5t1K`iUr5^KB;< zwxnaNC5%)&zdG)P@kQEMoh?ET>U5ju)or=N)PZlYtk~!aw$;m?Tu#@9&LUKC8tCx0 z&%jO6lwmPUFsc&z3S{XEhj8ppD(6(Kw%NM)NMYiTpxjcEm3UuaQ8dS+5?7e#DR!{yo+Ah$|Gi5Xr%u9bm4-=qz4axtG}7RPCOz6XeEdLD*ik~fzr=Jm9p*UrFM@e}RDi9?B+$944Udn@f4 zuzPo^|2b@yBS|LeU1)yU0URdw6-;Xi)rZaZkwXHJWqIrFSo z>C6=p^y1O7I*)pPAe2s>zagX{u@>4y!3ogEDYJb7w;BTOY{OZSV!nYwyQCeEoDn`w zL1IdgMwqDNyd#AWs;1carT44_fb_6c^hVD ziyZ7cH4)X(hcH59S%S@e>s8V+_e*woE6Sr4wn!H;7t;*_5H_;Sm9N)hB$3VwyzQ%h zTP_*n-44U8?I%c|5R=Z1?4TFT=KRN6~?~l32et4Ve2?eRcA_iNPOa_ z;7zs;*Iv$$7gMUz3{*iUxv2?hjCc~R_-UANV9GIJ@!+582KTP?;1{IMBaT6EJkqO< zM^tcg;e~ssPLC5C)}3m|YzBuiuJw}}LgtyRGQW$xL{ZPwAmI%kEsGRH@L*2csV4n- zdbWX5upJK13AgA8)D~}YHKdnu_G})8bFz*OaP9+#eByTwKAf58C>k3dr=hC}Y3~md z#HL#7iXmt(Mh2ZF`1eaBGu(D;*;;?^kv!L02{5+8_QENbBK&n;x|P>?Z7@f>Mt}=B zbs|DmILe(TFUkOX47{*N1R4(}JavOXmxe4vcv({L{F{x5XL1j|-}~1*SD3?}N3-{p zg&|@uz6UU3#S(XTTg~(P9IsD(Lo_RvmEEm*oKx*4 zeCvwY9yiKOPV8#&%`>4PJ}Yy+)F!ebByu?&Dq!kxa4S*b1SV%q^%Z@P{i^-71F3oT2vv*v7;5?$Bu zmV^wtzQ`5#@jl0&McEtu)_eetzoESmw58!e=}R4_taE!B3ZDNfc)H+}CXdWz$S*OU z<9XE!{IT)YQi4s#|EVP@XCLSh0l5V80S(Al_9HO&Kr~P1OfFk8#+;eS>y`b%{7g=A z74OCULG80S150sMq2}JqGYZuqZmMU3vNn*pn40J9b&O9qBED}t&RXcT`Z(do>6EJ% z_XqLnAlF#=)c#J-!h(w3WowJ)KL6LiI~%833;mKa564P&&N<{P3B1$(t|J*h?)vLJ z%)Gcpw%z``>1HqsTx+>Jn5^ee*PX1#I+@dPB5c;ApXhxv&zg6RKUu)vLX4F8DZ)g_ zYW7=R44M7bqI2D1&%Eed50=j9or)+>7iX$t`8^8?SOkbq{-x5lzpA8b-^XUT$cdsC zYBv#<&{v-iM$Qt%i_c1QKqU$yB`LH5$MWKeAJ5e0Gqc+Sqi^&@D)}Te{^w#rT{|A< zC>8_^*5P1@{j50=W6WK`ah$48ljY%0)g2Io3ia0w%5acLaO!|mD2BvT;!rs%5jdGQ zuX%JtHO17xnJgXiQU|n3f!S=A%@PD>VrxAOv8IOD*$uJ6hWP0qc@3lT{61@J!zptx znxkVb2eYO9<+fOtuCQs3EBG-)ht*U_Jxr0-)NqQ6X?M*XvZ2R{UGu zC~<=}9}@Z$zl4+R>c+x#g^6I{KF;#lI3+d|<;2DXde`rlG|UfF_d>Waf$COzW1_*!r;b-Fp(373U-_^QecbjsEm^K()>L zZv(7hwTY1NWpeE@*@c;ACbjW4gr|!4lU>4Q2;6)eow}o40OkaJZgH)(GkJfESL;Y} zac8n_3kFut1r~4VD6p1o3B0mtG5;$qb!uqsgc_ASZ`E*xe9+Z9*Y!$rzS9@ZUQj2e++NO~C${_MhU3aGL7KTVcLEL_xT2ghlg);X`$IR!+ zgJI;0u8rN6=1WP#HLeem%eG))(U(j~wq#u=?ot9$b2TweT<2KZ)$@1YKu3OZ<(6c; zGjK5GckNB|y0c&U+h4G>tX0>^kd*CcTAI0E#xHZfjITQT))NK4eQV@rGlnD%){RQ& zLe{V|F0fV=C+iNtusLVMN$xlsm&9V2wT57#m=LjF1BZO6E+*FMPj(bT`NRg|J=7Cc zkW)%;qTTPwiJh%V62Z)gkV{ZSmMk2^I{}Lneiuz}CkYg(Po89hr#|th*i{4$wLb_> z)XV~W-}))s1MhuZZa*n|T&_0N5(R~7Q@r3lfl3dsKYacz6sO6>Tua|jW33z%hYfj& z?H<=2oW^2tQR|T#qaJfAYu8%H{dMBhSql#f1Y>C&_0@bPSIb}jG~&c8l@TT)1iY0O zhK6{|t9Pqpz~0ayJR(fU;L%ST*yF8!vGYQ9u+6;`r-g*&FEiH7w(x8ho)j> z?{|ppZDCSdk%)E%pMMi>B9IlVpQ2nx%|N6p6Gp-m6qu+VgC>r9j}K$rbOhr=*Q1F? zOA?o`qAw}pzb9u&e4q_xw3y@=e)H{9`fAeTETg0z*U&X`gnHPk+QE0n z@*}x_J5EZ;mmj!!Li_l|V`Qqu`RZgjB$zzGKJ-4+vCU!A!g+_3m=&35UOiO&*&?S^ z^EgkK``Yb1F8X*__XVhb-;BEpyG+w!cK@q;HF~U=g=9=|>}#N%!uXsF*C_ul2;@ZTgoDdy$Q7 zk*Bpu6{D#_H+m1j2J7rd3T`4P%%KUE5krw>rUHeaSKkzAT%K|$tBdDmW<)$zWqkrV zIqxE8*| z#U8Qr@u~H>+WxF;`_J&8ja%`<=y6_aHoF)(>zM$yoSnj&`K|(a^}c4(#Je4kX0K ztDx0=#lRvQtOEq(^S2U;k$i&Iii1|=Gf>bbf3%P@ zWLVGO_&|+>h)w_#up=wmNw)1v$h0Yn72$2YeRq7qaQ+1O8P2Sggn)OHiN?57hxpTo zbsc941+-r>*LK&Y3*euUWIDOeCmp~Rqhp+|dQ-;-T9I!TvAA`WU!*PLoZTU<5Gc&w zJW}lgjJ2!_Z=MvB<5!~lsC)UFshNI%R~zu--0)U8&ikxZ?-zk{7fZ)6b;WLntBfKp z3wJyVwY%K|8dydH8Zt$eWAuo$)vRjyQdC>x5G@ z=4_sBw|$1Tjkg)BA+P2GIb?{wqHS=S2;{(wPA%a!ktWYt$!EyRix?f(9j+I+@B(Nq z9%}V6ZA9+bpD)5iB%~vGieQ*Jef||_uEossp)zM{86b`0f^CrX$z?aY?`{Q0#!CDd zuxP2bRtzUMB0p@**8aGns|>@1)V@Rq>q$DmU{UfpMF_2(c}rrUsci)M+x_BEL*!O) zNP{0o8KntSk?naGm1u44@*t@#<`VLYlbM|yy*x6RJR4A-9Rknw)Zqv(|KvruA3Ci69ARj-O8k@;aS|O=-x3b26^!a_r;J{ARPcEcW`FkI40`_FY-+z5_fX z%6)M2IClUTWyTAexG%~w{{?%-*6iYr=BDawR`#d}rjOJyFe7%L*^_+Pt6F<1-ft!B ztt#An>kD;f($vzDIp5rQrwpE-(~tAbGdxSkZA{4Ib9Bv4YF#5T)1dKNbfxl~RBYcfa>c`eQu;$T}U zyAH9LW$8`y`mt7C=2#m5v;HzI79)YKL&W?cJdM3HN5(DZA z>AOE!**zz)v;E)Mo_7>yR+nHgL*r0J_6&@T`WK^j^gg<5}aCX3tq>6`CKR&#$3i=G^D%2{02;K6shNEadF4U-E~6M|yYc>Lx=@^o!$ z@y%lu)R&kCdF6J`Oe1Q29%@V{d&idU`UbP!ewc4QXFn`3SJ)2=O+p$%mtQiBa+77z zjcSr>0)TeX9oZ>5&xCmf>9_-8ZBH)o>RjWwmTe(K|7@z{=hr&zx$~t|$qM+-xQNlt z-bG6SJ<3?9MUCeG10vVH)PXA!IYyDTZH~u$Q#P8alJ44e{lv&aloxU?V#xo;+q(cp zRa}q%yV*^$z`_PdAPPv7plGB<5laZvB(MQgFc3*Vd04BMrdAPlK`bGO8(Nm@Dy_BE zueNHnt+xHPv{j3Ung=EUQG@Wsgog%A?W`LiJcIy|{eRBf-8^ilef;`IH}~F|Gp{o< zXU?2+<_y7PV#@tz3%Mpuv7Nbirzu$Sm2_CNWAHRcR0sAjP>T=G*K096W}(WGtY>!2 zm@On_{5w+1)J4%_0Mc2Sq&bT=NSW8D*k_>UHeT{?7SPDGnF5C~k7a|9Is-)n6re8G zKHFloivw`G!(!4~U+d-F#5Z#jg~=&c-XeNE`TdZPzUj0Cr60IEa62EA8*pr+a2i_gs!&9E?By2lyo}ZvFBg z_MFBB4B#75MJ>aPcB6Bp4GH27eo<9KUZ|d)&TVuvxh!!1ciSLW2a-dn5*^~|!1?u| zfk#~nst6_nOQ;|I-DN%C!@)h>&}<8)e%@s1POmrtlK~_qKqiyQ6+zL!i*u!Cozab0 zbiK721r83qWZV5Ya!vywhd;+b7-u(fIL$$Dq7F{<=OjmhQ*>~u4o>#xjEMx}ZWidP z3HIkqK){j4A*A*!rZn4xjC|=OP8@OnI0sdHm6fnVM`xQg(kuxD%Wx}XnMYGn;7D7O z(xY0G-&?gP|L?0Mb&8aqPWkL)UYQdqKV8bV`YIbJe;MT)vr<3`6>F1L#Q5zQMvmP- zHN$S?ICyXlS=TO#VTYQ1Qde5DF~fAQM)|~U!D!1C)|kAI3?=E2VT@AY&=OIjK?8J4 zbriN2;S*Ftq)lB)HMiMq+;&YIlYYZfQmC9AIT+!w(4})AP}b#H;)((1u1I~^XJ=$P zyzz(zz4}HaU6_`>6yoIcF02laIU4Wo`nY0IoOhwAFAKl8Nf;$a+Km)OHCQhw2ve?v z%p=@`zRh!W#r!z$6+NM6s|!tNKob#ifzHs+H6Jjna{egHSGt)o;FlTw*L}{wQW!U9 zU|s?O*w6^B8{+V{UfUg+^`6wscRjUfkbQLKF&W>m~G5*waUwUtq5i5qMgsYD7iG29A=rQ^4do1NU32 z{${s*kA0=T27wd6-z>amDqgsaSpyw#P1Yk8PnQ~=}!O54s>F}qySgN3Hr|jEYukXKO%s0A*eR6 z4_Q+{9PW)6W2Fwe2mO>v*I-abaC;S}`oF;G;=Hsf|J>HhhGm0`Zh(~Lt8Y&M1#?@4 zm+G)>_Z?2irBGS+HyI^3Ds6*tp6#o=n(8gV_akm|yjo;Bvja)*+*ag-Q(;p-WVDn=Bde7$QsukxFvxi7<9I?SckUb)Bb&UcTTwJz1DF`o^905K~$Qb z)~@cZL;f9)?n}5+(g-rdG$t#pma~VIS^k+$-YA?-v$`H$N*8u9DCR6c!kFtwt8-O| zWO};CGu?CVbWZ^+F}}=*D>5>?X{dJGP=2lLGWF9oq?AbI22#Szgfy#pEr{Fz65VY0 zTBmFC3Cmc=>^6k;G5$Cqi1_J)9_Bhi4^(|RKt~-9eQZ?21u!TrRca_;-j0*i z;#$>Jv5=V-%yAGA&Bw=qOW&snnR}Lh9-Kt1;cg|)n7ScR4vAVx&DudBtR1pciyX(Qb zg-}T2Ph`bn0D2prxO!ny&f_g_4m)YE!C(#k0XI7g>~mMU>d;V`Ro)KUp3dsek&fF= zJG&a!6=}HDoPaZ(9>0WM4(cP(bnk%Cf{i>?_Y-R7ZJVD*#;kyRiFeI2&4fl2Ey{1l z0bP6D)PWA&@@dJzOK?Zl*=q%KKRa;g|3uyY$r_yl1nKpn2s%>!iIjzfT-7Ibo43QP zWwiEf1vLLA`jUwgY=+gB!8g=Qc}mil9R|HD=`hn6@A^yHV0K;Z`Usw8()@FEFG?MH znEME1(HoMHmRp`FbwNz1qBt%%=)YX# zAStrH+1UBk-~eV+aeVMe8QcHGhIW=pLl?yaA7g+QJAxPgw@QVlv3bAYyPv-kXlfs` zxdUf5Q+mB-o!yP}volrQI$8KK3FV!44Rdf4P?CTTF!7oCOVy<3lpMy!p?9kZG87705u1u2j(LIA6yBlQd({Bzl zO-?^fKc*Rj;1hAQ#oa-2+k3v%4e)#e7)jWTtTI;l+w3}(Jhu>!zo-}M~N`2 z1+y4yUQgb1&(+1^bOAGsIfWFWl0WZcJ+6^Lylq9d%Z3W!Po_z~MiLypO8sRYbR=qc zqO1C#P{0^ybOiCy$nKX>X}~E|#t@3l+>Xp?HFQoIdg1*!P>MgoP;!D?7WV9^nb}#( zIGz3`+fh8jATQi+%vH8+hW#DC{VHSXJMidNVaM_gO!`%h;P)Y4WEyb(FgS(2mlaZa zTu4-X)g5hitGBI_-b5&k(2jpZWFlnNq%rgy9mnd#MPJ^bbD+4tG5LM?0kPi*a-e}5 zNrS(YvG$K>JWDFn*sKD?9`ZHE*aLYtwwvo#PXgKiR{k2#4ln=zai*2Wb)T|;Wx89? z99vm60?zXCD!Zi}2g^l|R69hh0|Ss0qBk*%xVL0`(& zh;ezJPBeiN1lsRzJbpH1NOmU8nA_sH(14@e-;>4YSsD18U_Wq85#0vObqr-ey_8vt zB@=T*2Js+si>t4upAM7AW_&y&1LU#0Unj+eE_+n-tP{wS`ZrkH8bf9z5 z*}Ub}*wNYaF|#?uBRGO2Ow?H^@})gk5uVRM+2J5p1V?yv^KmxfbG(!xf0$dQ_R-T~+_>E=&Xx(aj zk%t)>$|O#Rjwf->95t#d@=1xrwp@XEnAj(+4 zJtpmuc`I{N#@Qi;!XM4r1vfwzjNFs}o-wmijNDXzHb<()__NtxCtwU9CP&#=nkDawlMp+u!Y&})U@Z=fE7jv!vOh=;j9M;bKWeIW+ReILPDNaD_8sR-P!e61UfVW z4eJc9V_j>1BkAGF6$Haq_0D&yuY&EuIN05S72}!#!o>v}0huc&S;3y!s}C(FnK=U8 z`{kTi3vkyp0XG1!@XQwA#j%PuVVNc7E0ktmeZCPVtfFU-u!$ZgXH;C*aC~|Fl)PY$nFT(daWk+!J2CQ;W*HBY+-i><=)iQm?HF4xME8_JOZ8rmjm24HB%g!J=`Lpex{yK@Z}qwo`dhdFw@q zfXaKOGD`OQCn1?})J?TISO_fsmC4%MN0d+xpBp9AvdakG!h!|DsSGKZW`^auH0%yb z9CU_IWxb)ncjSyq<)e)zxA{(oy_sws;Vt#;k<3c2!pN*HMY)e{s7q`g%abHN7z_8H-L#-tE4FGLA6HWZ1MIqgZ~G(S~>E5|zVrLDobb zB5TTZ7$+&uKlMt7v7IX#q-Ty~u5>K>KZAnYN+?t{GXt`LkKr)0F4yUsmZ*X|kLdLs zD%KB}KyI4`lNyTuKa>YQyiTLd-41DE?tKd^hG#nQm^f1C(vL{qIWK}0@`X0JvIEeF z*>HHoglx+MepC3}!EX(}r}}<<+ceds0Jb%pZvYc#7 z1HWHS$+o=2@3yJg7LlaS_TAATzIe>Tc`}#1*Z?IM-zZeye~-54^Kk7DOAMU3I~7sZ zs1w<7R$Tl|NeFKgsaG*KcFg-2P!U~Vlb31Ch0Y|h@3G3Zhut3AUd%gh;ap+9nj3}G zk~&w>aN&^h)y}KT^`g`f$BM?7JekGFIt2%mS;sm@~+f1j0mjo z4W|)j%3IMemx8#}g1C{$-4?d1(YpnCV8lQ7xW?XCgdAY+Vz3v3FUDN3S1H)D&Q|At z1T2abRFB?U#HemAQXBCl)g@E*!*s5K@+~v0n(`H&m;&Xy9e0UaXK9?w=OhYF)YU9gCC~O|hvsVmv?iCERu%lE9b!eGWSV z!V;L-jsS~eTC3MfHDD+>h^|$f?W~2CsCXSwW~bgDwM)+t5k3l0w`s;`k}RoQQWBBs z7qxP#G!LsE(JAcDQz2LgG!y&Tga7hH1X*08nli4EwU=;<9ddXmk6zhHMmnH#;Bo7hh~&GR}3PoZyLlo9Ahll+>u%1aps9(%$pM< z!=~%Jn7G9kLrYFVATO6OLfT|~Qq41PhQGk>9}u|C>UufGKg+JKM(EV;BR}?CXz_j1 zj&CvKz_1T+1h5a0t8$S2!(n=pFe#!}Ig1$f+WT?%BODjzOk=Jc7x`vZQK4_<8pnz& zi&N%hw!3oLv9I*Cf3b3KA#(BXH6pN^y-MmHF$4WRAM+u@+>RF)nFLxJ&uxmHE1@6x z3IJWG=Tc)_=2=(nNlw$*QL3mPqg1t_!+#~rkOXR0Lp;z%o2`yUufJjlXxxX&})_@&yAbP&?{n+vLs_&G9lgrLJg1X%c(6cT7$6F$n6lfY9#SS&`D#`FFbX7W_kGfO>Xu>V>WGckx zsUh>orn4f}!fU2?KT|&rw=-Ac6tO>PLSGYKx>k`} zHe>v}0yCNU^J|Pd6n?sa7WPEoy@0BBuEcqryMrwxwx8-rsw(;s-{j_xrdeV^tL#jQk9+QNIQT4{w z(H5*vaM9A&sG9#{YGzNZ8EZ+P^_Eaj3M^sIGM0kS1&@DvFRe^k00Kca-cT*fQ24u~ zEJE^BEMJTz2BTR}~LEQiqEBXdl=Tu;j?Gg*cyCo;c?Y)^)Fbg)^d+BvO7q5 zRYi*@6`^d{;u$Qm*{~-j^fsrAq?EL(nn3p1&bKc*-#Q|1!Yb&45FF_BoF7%bPx_jS zBYIdLV;jNBf+pRWr#=T$uuhmZwPs9iEGW?aoZ*)xZG7WcotxgnZISmU{dA@S+Nk3E+#EIz7I2?Qk(^gcmx1ic&j;iAPFJdksi z#3cQ~e00b|J@}BOqj8(=s-9-4@l`stPFA%D5t`|k?RTgY3g+0Nl7Z?^2iIy|G)GAp0|4`@og4`u3VYIjX{N@qqc6P@+*)zDlkkH0*EM zf!DfOJRIj2j@uU~X(>i&>0j8)nU)M(`X(uZbaj>?l&`)^=ZpGtCQB%kxdnvW)(&z@ZjhKo*bRP#Q($jI*ML6-<+=ly5{RDxl(=A*^nc!>I)1&R>=f) zM#>nm9-Wr8;yurjbweS0k%B%A#rpZQYuaE~Bh*3f-c9;&kQ7%}obX(N?n$V_AbD!? zP{)NHnj9`MUuvhtSQghrJ{H%ki^N6|Ak%~DM8>l>%ZQn1^y%&j=Z4 z*)iS`1P+||O`A6nD#h5XXz{*hpxv(t*~n~IrLW#vV}wsuGg&R3VL^bPQ znv5Z9ttc|by*Cbg`c83gyLGHCn`nh{RA9g0_Meby;F!~bj+(&|t-uQNO+OAtd* z?5kB7G4qQMwpFSGBBSo;QlTxyL)HBk$3Sa6vBfd--@f$Owms7<{{F4H_@~3EnvDeJ93vrz57A z+L+@DSyx%5EN1Q zze?pzra9{qW>kSOURZW*9QZBFe!Rmn#$Qf=hr3__CI9P82$+*B&G$}}Ie7!}XD*8j zT{2r`%PWVl&NJf5CQX>k*$e>k)U9SyXrc=#p)mmalU?Rf2_r7cRu}rb*jZF2sQuxO zXbI9d&?%QUnCMJYuRt6kE7}v!3kwsP^oCrk!XE!gnf??h2E?8{H0oCpAUhk5Juom> zio0Q>pNWloD5l25!9j30ziVvCAd}HmNMBX}g^;lLkS)!i4nx)o23F`%T}+T$I)Y2E^AHlHSi)VG?dxw7BMEo#ujuL^E+! zh|#(k>iyKr#A8O1?8GgL%%w@MZE*OWr_2Q|F1o-C0mJ43x1R;>au&D?A`6_;T;RN0 z{X;{^2)o%6@h18-3gpd>UnE&Eu{I-LtYiG_BJ6q>K_gsiSAx!HtxbXiPPE1nA85@yK-195C8Mfq`Y7o?qGwPhYP&e?31E-P> z^#%2jKy`@>P`Mz%nI{8DwL~LEG75}#N-)n^0E>q&Jqwbf17CM_ovNTNH3opB=vh+D zGq<0Z#-&h+ix}APn}phi;xcPX;*Hv8B?g;WC6}l?dn+HB-M;{PME`h+Qw=>LVvEn8 zx2R^^d#@~o!}B|1d{rrdIrj2XZCj^VO5&f93_=QeQllQe1h#bYBw4@MT75>9!{a4d z{Rs)GZ?wEJ95+cmnzLjY=u9(@EW4=uYyGrTsfs;Zq+{>Yy;{l#J_pXIcak&G*k za#PqT+{LPH)Se6u9E}#5)UHOct{12ZP>u_3{=%OkdWaItoP7}27pkRf(|t>Z zGx?V|IXW!D8U|#u49JOa;6UpU*^RIyt6RGY&!KRmPTd4(4Hd{j`A9yYR<1N*ag9)v zYc)lYffpTS4u(~zj!YS2TqP%KI1k3X@VP$|On3NG4|8SHtKGl`7T&E1Jfp#vG(;Rw<`Cwe-1e^J~OZ9^c3ptc) zar&#=jj<3N6KyD$Ht2=8 zH9?%c!AmfkkTYU7iESe{9V|@54I-BddCyb7gHl9>)I-k*LLyoK!<)>yEY_~$-aF|{ z$ZTAZic_Jm2PK$OlogMDrZEr9mM)l;8|8fdAI_Z2bvmKH#)n9FXTj>fA`&icm1m?? zd<@zn@!0l??n*T1<6Z08iO6hR9?MaOg<>R_S>^fxQx$2&HTbLQP%l@KBkH-W376*D z(;BUDp>Jk3Eh|EHe9^e~%Fi$oN>>cnB#)Cv^dl5!)SNuR(ZuSIO9z-hD-k$ny4zRY zflYKrf%={h6{O--dQ6Hks+>X*`+ABg#NBbJ8%50V&?N;b@eLp)Vy@_2w|P=AUt1TG zb3Kj#W}w2QrP9QGuC*hm1R`baRZt@h%WRM)YtABcB@9?K9`aIWxrma270hzqEN4pE z8BVj;f$QKZGbAC@Dm#~m;k>h$BXkZe2tZe(M|_ry_}}^IH6cbBB?+D@GfEOYm+QXs zj@3nT7LH>qvf627j8nbJF||mC|LiI0aP)l!b8Qwi{=^MyR#DisdWSWTl$lYzaAB5?sUg&Jo9G}9U3ay|RH zakB3c)ms!%#C>Cf|>5~8{D?`UC+vFk(*6mxVJP?hiCNZ_gt#@U=E!HhB9 zI*|>v9I9fw8t}I#W%OxVWqK>~L(fslMkB)CGS4Ucjp4Q{?eK@=!f#4z;<%67+sC)Y zZt>RA%~!jNQvzjvzIkI?=BoF75uYY+Ce=L1rDw39%OF1)mvY3O+do%y4^05^0i+l-w&f z=BcDl30hbqS|(XwCA0m1BZzT2Ge4A0m^GJi-Az&8fqgO>Swa>?1iD!T>aza<0npbG zh>KUeg*sBc)nA|mQTeUh45xlYK$m9Ww|t>l2z{g;>Jk2k8E({sws&rycW_6dhrRO+ zsflc7@Z@FewAJFE-hn#ofg&3=xl#2){ITbPLkTE9RmSLdJ@AgK5Ow2fUts^tC9hpN zh`sjCR2G--*MOP(7uVWN@~*c|uC+hq-L?5|F{m}-d_j9)h20+?D76!0TR!lZYs~;( z&=FXH=ea$uK&j)TC}b?2Z@D&qVsrWa#QPH0J+qDnRvhQKD8Uy@=b4+rpS#loE7ErIymd{rCyeB14nqKmkzzS@uBN?{`RrZVZ~5w$w|sWy?Fbc++pLbjiVh?EmHFCxO6J#G*7hFilPXfMjPgqtJV%p`WID29Be9Lp{{_RDbUqY}HpsM}2ETj;7~ zaI7|4JtqE6NtPOLe4BWl4+`9H{!Hh07r#+ExwRmEeXe!gH0%1@81h{=4a*>#uTJN_ zqceAK-3hCqtl~knE=KC`t4^p5CJcLQsUa#YV%K%XlE-1n+|O?}za9Lx^4r4iMSj2I zXH0#l#Xt4o7M_Zy;@Qd*Gp2{yc((C8&ht3WcAgmcJk-InBVdP(wOG(Lv`5fkgr(65 z{D$*O=f_^y$p(Idw7C1b+4@Kbb?ku=&=G|Vc&_XStj_3&bV6!ZC-nc~*e^RM_s7MQ z5k>Xxjt@@8yqd5g^@lxWb+-Co^?G?zzW+f1$j@WLV5h=@w=5@5x=GU z?&D|3Z|XzC!S!&SoIZXynJ2zUAI1ypLkT<+cshAHd1Bx6u!AQyPY>IB_Kk_7=g9AO z{I>9G(0NDtCf5cAy7h(rrnTMrfwlo-hV|?_Q_Wat)cTrT@R@7hV@QaMytL}mc+VXX zg~tZwHje4;nI$x)KnT*Jbmn9{zZ+$877|ptn4o)>Wbxl|O5i5^*PCrS}k4 zx{|Qc(xriy%P1xAhWx$G`z&1RjV#8WigtKt;0XdPB&EDkZc|QR-e2cE>^tc4T!u?3 z^t%0xA^xFIR_N*sPWIP}KG-D#Fn8Jv?@2TLJO*u~?pp7#Ss1FchP3UojcPMRpY=o5 z4b>^QoUMDF)4j*$2T!v?!br$29-qX0)V1*Zndz(J)it~r>FQE>*f*LYhypc#IiHx( zB4bg=szVo}whF(>T58Eylj2r3_Whgm?xszG?y;d55AyK*UJNqy;UDM`t5R61T{9h z6rHFvD~%)YMy{97L+QenYsASaz1STB=>ySpHlXF)%Yq&Ih6Iat^HmWkk7_JG)5de< zCD3J7fgYOC?WAc!1VeW3xwh={sRUb%>*ZuVi+z>WiMza8j0U^}>x|aE>&+OoHY@ zg)#S>G)|dyZi*BWM={pWMXySWTrXcq8WzfwTGtZS@|GR#m(T;%gp7r-yk#Ww&3KE89eLT?< zt35*zv-a0QS3deyRC$Bwwi{$l7!f4Va zZuNW^`mILV`s(0x?91>7CKxp8_(@Gk!l5zKi&L;4_W}RQ}VJdgImgOb+ZubM1>WV_S9*nZz+%Gd_DM9OM61$yBftXk;&a zK8C}e^)deJrIce0_3d`tt#;AEYP*;+`dVd^6PVeiJ&)Q`@Yqy4LL9Q=1V#z>A? z=gcwYt8%OobylR8%vRHX4t1#JT&+&{svPh6yBzho(IniD%vt3-MUU*J6{r|Mg_80%8%+|Hm3+a+U_CgcG4|!}eeyBc? z+RJvoP}MyntVtm*%mnCmbfsA2BeGpXcsG%)=ui=MEo?!_HCt^C~uk1{k=h*t0a=n$Y@VK0r54+Y)p+x)$x+*5>msHoo zSMq|OX++tnbnGVjE)J!055|2S$E0~4d$A*QxHw^PsIM=a;7ta?i)xcd`=oi^KI>=6 z9*&~uY(&r}Idaa?2{}W8Wqg@bmDA|1m$yl6h0nBm`e;vADm`@mXqk_msnjlz{nVuM zT(4KHwE0wuuX0;v-SV-BbA8(#$c_hlV>8`tD>xF7;M!2<4ebjJ=-%Pu_LY?=Sy4Hu z^v$JT=!gVBWPV0OlQ_Ayz#N3gTSX+dTA>!9S~4z)&#I!Rrl>@?AGT! zA!V0ON-Y&D-Kz=1gwN7e3W%fpl)Skrka!}*Bg$2QKF;!j8G3@(^*5xt$_|#f*5)%{ znvSRs_tAN$tlpTWX5Zvi%xNsHwcjRTpnp5c(#<1C5`2#D&d3{jMyhX+!Oc=XoQCjv zVruFThu4-J8uvk!2(K)mHwhZx@V4GFd)j9SS!TA;(V_9PVeKqG;P1cpnc7M{s!iJ<4MR9EF^@h7LI4lU!;Gi2Xb zOG~b$;%yHIlFOPb!J{N@I#lsCY&B&xH3uj1IMCRnf>+9SmAU|^rsj%muC-6>Q8oW*H&> z^mufwO(a3cf@qMz%aO^XtRLpG=NiKvXiPi1lZLC=kyI{m-4wf2N1SGZv}^kd7Id=S z8zzSlUac-!;ySi{@>O<=rL-S{MQnZ#h>0_PJ5E)PZA>RIam_rbCoeJ>phk z*gO^J7eGhk>FVxbTI%eJlZ%>VA${Z?h7c~y8@2QqdNdqeC!+Mt|JS)nJh@huqTuIr zk*Nv&StSJSfT&;auWT->@9Mimn*82o6w)8EKFMMy0(9@v0`(21*1xdS{?qEwB$e1z z%M%q;Pc&RCch_=g%=7h5t%W0^wOIA6)T7=%b3bl!yw1#p4(}kHBs5@MzbXEOjb*3$ zdbc7uj%s}(>g5;j)~jy|2LkX5DXlYC-#l!==GD13UJN;?bH50FNQt6Q9$U;ur91-e zjYgWWJo)y_@$%^z#GMY7yrK_Q%T2d)aB;fm7T;NmCp|C)`;!+w#2}&4GsKvZGCBRK zDFw*>r=$`vC_mf~>W`B<%ufFNfs7M``o@$*RX_s#E8c83?spi=kX)rMz{9Gq!D8H& z;M-~s-=@`15bfFde?XMI1IGVA$UGk1=t`l5c-Fc9z<8b zn+-Qi<0|v=v~JNKhZ%3u24XlhM&!WgNrOAdQ}hnmw#qzp7a9bAoahujLw{i-@Tr-QUHF?C`QY;# zkx$v{PHsekWbvF=Bv17$sF!B-)!9qXd^O{H7}K|uPq7qj<&R=LGi`UK6kYpiwnaVI z%z&_`Rtmrt0f3ofZAcgMH;y(AE0Q;LT_j^vz{M=g;Uo!-T$w|V<`#b~3B{0s@e@v3 zP22FEr+!o>YQopcF1?96XwW5&!DE@E*vdw$QETBC^qF_FZ^@X4YM={hA{_bMw9w~E^z zAi%;O^&1@!B^W7GT#PBam)6DVcJfW@+obg}Fy$I5eY2h%YTBzYrszv;cO?T)V5-Sc zcXD?Tqu6`8$N@h`-K5d>hQq*V|=ILR>Zy%M?S}X zxp+}pWld1>EOr#7NO#`O^pX8kr#aAIwSlQ?s_mD<>LDe)vrxUn-pB0jJDu+qs>wf< z-p3{)ojaS&_p_cX@rXU3^4OUSDMkG`^9xhPJQynjGCLP`8r>W+7+EgD%Ug z^SVe=QMt$cvH>T*SY&H8z9pO{93a=($;maTmRJR330|F$C%qJD0k^kL zp~}?tPeuRVNZZ8XLP>Y_swEyw>Q1fu^8#N}%uzqd_|JDnGQO~6Kc#j&OsN-^ba0)* zLI#KGPq0Af6@JXIWt^V<4w(4XnAU|lS^p|xT`Q6dYfE;M}g#%M`3(LNpmASD=e zY^B`$RXXj!c1cpvjHjSwy1&KzNv4fhTonD}?i@=G)0cIEfj=$*17bSeT@HY$NaK!9 z4rlXf%0gNoOL#X6ll_INbq|fh);enGwH(p^e}k17qGoheCrzsJPM9jA!Qfg%tWw0* zrZ7Ag-)$U|>1JH4Ih0y8h&F2ZfVZVk{aWfAoeYe}=Y5!bfBIImwKOzzxA;88jDay) zDBYM)pjdv`SD7>;`r9!A{Na%nM!Lr5T!Rn_(=qSUkZ>zV|o!m?u+r&hoijoXL&RsTy0@QlUw z5}SXq9c>bq53q}*D*m;{C8J8$y5E5o?)G^{Y+5u;fv$@0lD6!$ z)up{L8m~!dhW$06qlSjaSYS&fRoG*_ zoZlF3Lm*ULM8_It?b{6978#6oZG)2um}~7TWSx1F^>WwRKk$|xcvLDor%25D0*`jc zd-!T=IO#D~blGmMmzo3wU-`2mWsJeR4=f7)?$Y&?9Iz4j!6Lo_0m*|QF%IjIh8r`t zFTZTPby9!tV9w5E)-O*EWOJWtU-*;y5SBTWuGh;p)=jh6>a*$Boq@HIn5!}ER|c0! z)_;$m6H~4BG7}q~^9)`2P<+|MzFzCJoXkDTK3}hU1P&<0s0)6LJWTwTd0W+k_0d72 zX^0sClM|LnrI9VD?r&ypoC*F-!ha^nPFw1*Ew#hR+kYlFzeQKVRq;9iT)h!N%fOR> zYfQ;1Q{}zZcvLzNZx6Owqdx6AarC5*4p;ft%J5o|M3mWAf)Q(O))WL;$4}-sEh+47 zZ|cPNrV|l-W7*7HUuG3)E`#J z5&+vPHPq)#4pK6r#Uq)GEAf1OZoAbEv0L3G9b4cm|48!-*J#UCd@6Y^cde@=Dr&jf zp2y5)+w)fu`=JEe>Wo;cTDhovA$_$7)%&0VwUnbd@Gh>kKahO!RVq#p`b%xZU=?v{ z$c6_D$*e#%Vm$)=UDl+#R2F0!+|K1J08-b%(Ai>BQ?7o1LiT02l#t zcQuFhx(qa-tarZIiRBp-3j=oxiLfdC!Mo8u?G!w!>ophL)7m>?QCd%~+8*w8Nhd3d z<2Te`+ahY9S53<8vMWtihqrc?yVC<2Yt_eq))emVC83mREi5 zxxf08PTU9dhoSrwB5V)nhz-DY7$LiHmf^KY{INjH$vNQOlBeF*t#!&OU-5M* zE?id9ywcn6LagdfzzOaS^$~v1IC&xaC~pJpa|nhZpPV4uqGlPgAFouoJaHE_9V`H{ z5K)sD*859d^p|W5C#A|3x52pKr7c|N!ELne`n)+9okW+^$m$cm+A$}b^mJrdS;cA{ z_Pn%3h1Sy_x4*QJWX*Yodw1Bgc8hxJxO9?J<5Sf9*8|Ol-_L^+aleyw=_1~Xo=3(o z?nFVry(3Spgu3fh$urp3KE&JCJk$2*GA$;-Z~P8%DsjT=czq*@crTv1h)rE!?M*Z) zPyO(9o!bABeAcU4Ch?w&Ybkdl7uWuJowPAgGl~f4DrP~>!(J3c z+)Hlwbo+&->?T*8T1WUgm)^Cn5+U?S;P2`RevaVc1f0S=mZ!G7#zgoDau#u$(I_1F zes%BXx>$NRRbV3R7r{|pVV>Mow?CZpoOEb=@P2UMyTf5AEVA>aPRPhNXWYekY8ZKQ z$0L8k-W{qxPmr%JmBMB=`W?Yik`$%Obx`yy^R5x~lldBY4r*8=Y!RqyHb=GJ{k)8XN?Obj6<5VycqvrxQ$h}1ZR}EU;Ts((EX{0B%}GZ zwoL+=*|vNhkvQ71bfuU;0)}6q8qRptHcFRv!zlxhYlD!L zg3Rh=pYzLvvl|cF$GY~l$+T$X@=MTH7N}AvjX5HQni)MS*#zwM*?yg`>F-@@lcx)z zeQ2K0FQ}EQSe<7;lhsR1-3Vd*DKcOffuPI{**l#U ziFo*!`~_N)m0NrVx#v@M+T6JX88J8VkHe#J^OPR-`~hGj_qGOA*F&XY@x-46eR=r})* z7$ocQ>JBpYr^A5UR-k*K8H(;JX+&>ZsE$CzxEMrj6|9N)rbzna_+04|;UF%RN>|P*^-5Pn{Hb zhwS=1e~i#>HQ_s;Y@a8kP~El(ykxqa%Myz(k^(h@>8Frd27Nh)mPGUudR2%X60R6k zAB zylC9px2V1^({*eo^5vgeNUW$N%3EcR(7j^fT4n7iRxQ06<>wj4*8QxVPc=(2;9ME68JL9a`fF4l_<0+@{=IMJKuwQtZBn0Kzv4o_MDYj=gmUTa;DO&40d zo?5FXwCr`6dT+>_3k@h9@VbnV#X9;=fwfX(o?gVlNzcoAO?P`PEk7fR)yfMa@mx$a zn*&hMBE5xySz+5-SUecrTb}Te+45TJ@@$-MR28S#YV&5R-yRXkYx=P$E(aG?ux`?X zvkW_PBy7>hk~=i?Vy-Sh)+<=+I%$Ouf-{1aZZ{8!gi#zO70E$;x%Qg5sp^(g&ed6Q$%$^z84rzd_f#bzn5s`>fF7XS3b zs$fF-LCM*ul2N7h=gN3@5W`L}YX86p|5xJARLqDPsEI z6zc%HDj{T!7(RHu+iCicL6{f_T)TX1lWMHu_t6L2noc)uYdY3xt?+YTHj5EOp#QBP zBlra0D<_EZz^Lyu2=HDa78I7U1C1VgQz%s3bSU(?I=Y$hH0%YjLhv5-;X(}OaXmEo zon=hyS%#WI%A2FcBq%=q1ALx>i~4u*`AgxzOng2-je6p9EFoXUXOIQT#AjL$e0E_o zk5rpw@eTA}D%hOK_sWT1#^$9_Z0;Ld+>&eYR5mq-wy0l_`#*=zPl4=KiaNIG{~10n z?fCcc`G03R@p%FYinE3yVR_{UbVCz5~e3gES5|2oAULy>IOQ2&%U=eXfDq z4>UD}LvO1qi2CP-+{&Jrso40z)i24c_uhHH?U+3q<0|$0{sJ=HGh@1E0)Yk7Jt+rbCzR&W zhY5{UVtWNtaZ<5Etzj|6+A}m=CoE1Faj-`iY<4fAT<6H(D!+{)6!? zwf%SL?dKpYGQJ1B^yTsW+KbWgeS?t8o@1O@-d5^aq$j&KyXi9;ccgQ??g1fSK^qbI%iE+Hnp_-cH%_Q2-t zPyZ}FQ+k%AE+yrEgwGe?a{t}&IhV8%d@iFNJ@Gk$klyL<*)P%G8-Uz=fu_HeLVsuQ z-3$Fa3;q4WAG;9sk6EabKz7SiS@`Z`p(2%jjj*1TE+Rx%TBej|b|+()Lb%zbBb!cl z4(>+?-HcR$-fxhKZ!n>-FTLEHMx)KSaY1N@`g|ibzo_S$Y|gHYJ!V&D=GXMhTwO~B z(FXhn^T+=m^!NC#{(gYk^hDiQLd^cYsV6OJI;#77B&9}KC%$$Klo9(c*t`CM1l{djQ+{*~d)F)-(aqk~gS{&v zyr{h^mL0v>yS5_nIhVa_B_n`E?;Q57ai%%#Xw#Hd+q))E2f0$To4xCS*gOKY&1*U@ zoKn4Mgn$7OwRP1iI?__CRlP%1^KUwZdet|xDjo__3oIBASn$xxy{e1TE3m=B%7Wi> zx&O9rK-gUF--wmver@HNt9m?Sxumh32!~5XHy!$5TdzP;b!mC=pwQ~cHpB-d7sp!S5LuYDef%8x zhiW1K_W@8?01i~CX7dt*?;ujq*cFV#j$tqeyLeWZH$pVRbPb26Qbe@$fht9|Pg_&J z#M|}Y*VgD2|6iEzGfB{WzAx2tMJ&vj>oo8+t+(I-n_?J zep;;On+~n!xB&BbOg%H4a`sSzn__R(gPdhs?6YR{E>3Npr3o~)CduP4d%vMY1D zYbx1zQl^=K-Vjx)CO1bu$`eNJY1T3uV-w^fH;qw(TqF?{s*9L&bS`(f*PxAP0W>Z}vu_3aTc zMqLL5Y5H^ZGo7|na=S@j=5Dz$Bted=9aFiuN+stQ|46Y5RNwk#C-HS7+78kcC_{Jq z?l+H|?dHf?oE$mJ_nYBzym6ge z&g427lZ6(+)3@mmfwTQ1Lo-jrlzURjzA^SdY#S@G9NsywaEE&DIgYiscbEsSLSs(G zh(-w8%)26r+v_y>(wQnCP z*LhOP+@b+d5D{%6-!$er#VFg z;6fXZX?(0Jj1k|brb}%dRh&}vyeL^7o`jM`+yNTL7_^L+?`aiLVZNz5Iv+C4l8br* z{m~F?5!)~F_BCbsKb#{f7f}g~`?e?(OrH}NU@T`8VfUNQL})1(hw}CR^ca;)F6_)C zzEJ*!ULg=;#?+sq=m^!2szBQ`zl?gB%dHDkKO$s!>*E>Esyospd_OX5|I7wsdXazT zZn2({o8YQh6I5+Rr9K zz6KdJZLTV-7j^L2WR?Ci+RPL>k+n3&d(nwlk;PlND`qON99{)7`T7WGtOn)Uvqrzw zPhK2_^)EP^5Y@3objf`(MX|Q9nX`D!ChlDnQBNR@%NJKhM8VL8E^9tEd4% z9~$MVV0R{heM9`#w{n7xy0|J(`C0IL{NQeFESB#IYA5WN5dGk+T z*tNbQwj_D8^Zheh)pX(M#qf7ARl;WL2|*ghV6|Gi^gBMgv`gU|BYG0f&R`gp^gB`D zK%6RV?)GONLtW(CiDi(#|C78%ILO)ELERnWwx#tKz+%g=Sb?(5Y%fZ%Zo zG*}~;cB}k+QPm{mhnG5|M~g6;>pEKGBw0{8K#mp-Tq5JY1HRki4SU``AzP0X7J z%LKUP4hm?Kn=NKg(WohPhqfLL4yP$F)5*;EG+hf*naIV8W=(z!?x=<VyMsPUNc#sYKHu(<(t{Q=rOGxC`a+ z3`XBopf|0YXb^)??zT5iG{Bn6G^Aq+XweY0kJ)7E@?W0`hH8UnAy~z!`e0O^`rvWt zLnlNBRPM?*lDV9Tser9lY2Vf7m7mEf42PEuERLU_S+#mtDW{p=y*T@k6$91UXs%@V zJ2J#%tEg<;K`R$I45N;|%G_5r5RBlm`1ZwtrD6Zfw%9ZY@T$j8o+{=Fd#<53Y9But zE_wt+aVK5ve;D)pulhxnt+?lO@No#2OGN zzYE@!uQt9iNIq4NEd zq2<*>>x5@##v*&^pw;$_3zj?8^k~Y^w&M7C8M*e=eM>(r|3>(ptVdRUq~_8Dp=jzI zD#zS+t^F$l2OWH#`Xm~%ONVrD9w!>2eH$zi@9+Co-_1ZhRd(EFS#di`wkbQN% zB&uKg-s&_VS9lAamTapkj(>d7$+OlQ8KueA8yU*MFTQ{!c(3$xX5qHwhXL|c#uFa} zl>-okEU%?IeU&5YrX|QKjtxzq{JUV#I&FCP#>CJoCaZPv40B=gwNCF^&3$)l!H#ef zmr-$sqhu>1@O))~ctwo1KAO(5RUhux5G-Ufz}1oYj~56=eU-2J7d9B&;kw(mGtA!X z2cwB!)^D;{Qn`MRtbsA|xDQ^h-mZ z)zYupJ0P$4b~?J9!Nk`{$+EmkC==6M=%=|7H<{5mZ{A^D(-$6}9DH-@8KS$5GqCK& zTg4XN2beDih3%H^heM_37$fej43OIkI1Bi=tsb|}I|lFN4qvO~Qfh}M`hvn?d)4g% z?A>8&8@v}Twn(}q?A;yqHmK9Y*xCvQ@3na=i<1X0tS>+GA>7`u4;dUENUDWf8Hw8q zizgXVWLyp2j=jj65<#djMaCy_I`j_`TF*eYQ4w%EQxM5bf1dBEMaeSlF*mmIqG#um z43U6)Q=U368nT`c!@cQotGhv%;DK{@2i%RAe4Q~TwD$&%QZvNjSK;@#ea+>Dd&`$W z(*WhJ9}?@7JK7^D2N`6^MKp84XmOwU$9@?~P|y4%I=uAA>O9pjnu`F1G15G}x)0kB zF;_9q?Yfx-5mz@om@QTncSm`ZrL5{QA@V56ZI5zqP;Yk!*5F-tg(YB5sExPup^oz?gH7^+kyg4+E(&|~hZ5a& zPuT3)CMiFr7Tlywn)`L0{I=$rtQj|GjgTYX1>s#{hzF_nER*f4FkToyg?F)}!zwHa zI49V)%~p3Wphw#00|j&6kyj{Sj(~F-`x6cogm(dYIjkGi(HRh|!#L;09k2&4Kzc?s z6RDNa$7X52tu zs~u~Yc}pVxM?AYUaEh;`4L1@ur6zyKO&cHHm72g|TO32CLI6CIJ_xRTZ`B1&!&41s zYWWdHT3NZAE-#hy^*5y^`rfmIE|0{T;f-eaUP~xZ08Y{u;ZpCWrUj|V@{4TR<*1b0 z&OKJKe+UhK1&X6T+vKzVEAS_svEpw+>wU2^DJymPZXoF~*Ugo_Q!(CTvPg0~BGT?F zs#o;YUKQ130Qi2><%66^HshPgm5^H318hThy9WXK@*^VQ+lhltt7oEQV==SYzVh-V zu0n2EyJEZNBGYe*_-uJ0!7|S~XtUH)?dQ+9O6*uHWytFaPXPOltKwH?okvts82OE^ z*M;6aHbv)qPuo#f^${tXOWXG0GdHwbibuA1&%q_GGk@i|7kT1-p1hwS?4Hxe7(14@ zmf%D8wrwP8TH^ZfT@rHh+u_fE2@>`WVV@I*v~f){!oW47$b>|P-zNZ}-SWK%4cENa z5088%a#E-zAtK`ZK%$P;Y)1HKQSv9+CayUXvnj+)slL+=kE7xN7j5v4X&Rf(6A@w^ z;zr}T zKWm!tR}sPYMx9g!-d+hI_gL^G&=>ZlG#NMBlTV z+^a;6;wq>U8H1dsHEIfcio*g2hFC1qT@`PdkB802H+UfCBm8$syvb!GXHr}bJj1xa z^LUJ_qD)%xayxg?rnP(U^}}hhr>X+ZnUosHS&QDbaoYImoR`J`Uq$`2q;96#mUG4^@Mmcs0#En?b3iUOMizPm` z_X;Uk7OITj_?)=jA!;?mtau>e&t0LXfs#w6a~#nMIeM$mDzmsgWY(wm;tEZOzCiTt z9b&v$p?*O0J9mX#xoyu1^_XjcZlaS#4HyOu5(vt z4N%;3otcvm9Z0;rye6TyTFo)b97v7>doMHHgct5FgL(g5ILS3Qc%$($Cg_3)#P`X*% zd1Q88@5K!;A0`UnDpY#s#l!ln~Hz59Y?z+pt zKOhEQ7s6TxLRi3dE^!m=K46IMh4Dy&7WHVIU5lyIR-lYPDtNv~{mmTklz3U>16WERXaK z(E)cs#g77Uw0DTT8e-NlAda28S}y`+Kre~By}agl?|FXTEcJ8p{JeJv zu~8uX2_R1N4v}Nl>m(2-&s{HsK-8-}GY7d)d334HY41JD!x~`CDY87(JH*>2#AzT- z_YUz}4KeEs5NFO^t+hZIxgD7~9UiYa4Litg(3jZ<@2%kA3$ps6*Vr6l&k}pK*Vt6( z7+$e+kjM7UGI(JejH<=|A8X$NA60ejJ(-y#Lo&%3AYg!iQG!x|77b|P5KID@M3gWg zFiGU4El$&sdJ)b5D&*nGXeP&L?QQkyty&e^(pFpA>jP0BLm&@8d1y5eqTwO#@gRnW zNqCs~{%h}@z!`*7TrW~w zg;VHwibBCD1_kR=NV}>{v*v1%U}*z`^*Ry*EVwSs9fVV;BE?lW#V>e@Lcu8p1#1=u z>l*37YWw6fx)U2 zm4Z>^4#Fswh!j_06y#+Yv?UnDpkR&QV0Bw_yZhJ^7)M3-z;JyD1u$YsgdT)b>=r4m z!YMZL6orCQ3<}ow?$09|TV+@eOEWi?6V^FY8 zLV7?ggjjG7speyRU%BWAG$2T|9kb>hi-i2xz;Hdo%d0q!GKsYe0GBLMgxSjgDdzJO zg@VNl3i)IX)=6vb$w;sUkxzmbc|QyJWI(XmMT)DinRj`LLcwMR1?y>~Qf*&ZbH9oN zYY_RQTvRILlL5i9i4<31Gid@W!Da>ptM>wdb=sPHIufiw(-Qe1`2bU||rS{H0)P_Palm1^s< z=EBkTpD>FrktMHW+o}y6jl|e!6f?=s@`@)+0MI;6XS@*OcF@V0x zy64X##Z~xkMd}UCumT|BG*4#iOSc9y4ZV{DUW!+;DDXzj-PJheCf#55Hg4K*v zs_koQ?$?oE4YKa}JukB2D(jwyMT)ENl@bA#;46cIbu$O+j5YU6A8~=tR1C84F^F2P zvhX>2p7--r7|RZxqEIlFLE(B1sZ`sy*4%F+(KX1rXN9QrD(jv(B86D@3^;%%3b2Iu zGALMj4%P*0?uAIO23hy~`y9u`Rn|TKBT`(2ul$CmC=`5UP_P~l46 z5|(wv7~)qhoB|mL!k!zHvbdyZQYQqCdYH#SV`}6YQ?eGIJt66>hlE~sz!dvM3Za)B zFa=q`N^jj6*2^YdDI%%<0C}2GVHDtoV>@UsV_)f$3 zFAnDD{_yV{ou@fR_vF4D-N6GnIupLXZpzVpyEjL7JHA(b1n1ZIUW)IH`*U<{ALi(G zH0J1<_T=cy_|C=mM0`)hxBG(}-QQ5hZ+7SC{(?F_Lmk8M&A0E8VS7dfitNZU0Cfpn zt$?Na>U&}F!*3-@g0-I=uJo9032=x&m=q_d)ggQ0Qe4M?ZN712!KGF|%lpBM`#$Lk zVSY1F;A4bcVc(@T@LNS-Fs7`7)$$az0TB3mOz>t=1h<2*UDk_ZA+UAKGr9D&d6#RK zeAk%Q3zs1y*iY9Hk>xb$ZbG@WOD;F&eTU6!=3CfR@n>Qg5 z!Osf$L4q{cZjH#ugKA_g?-veZ-~qhDW70IY#Eyv?c9WKYB|O6O>Q^X15a~=?;w0z zKmR_)amCuN77dK?sD6z9ZUBrQ8UW+l5jHr+j{*eR@E#E3oooJ|FkTXY?GE%H&Rz6j z`_Tw&|HuCl+sg(<_l^*{Q~S~VbO_zTQe6DfX}w}Ja6a~wP3ed07N?oJcS8M$+ZO8L zU^0HoUU`+mqU2jdsQW908CMM}Q7`$An{ams9u~vr+L&c97n2jiIkJ(%eRfd;rTs#U zOW(UA*xOrSxS*T~rtepF8mocVsA{=GopTxGkH1BeN5KaMUd?#bYt75xcym#MFSmCY z{eH)PFXI;=gRK4^?Az5KH*7bIU9!Q0|q8+ z65YetXm{8QgI!L;@wr?TcfS1VRbT{%yy9U&@hQ5!tW290TxD_vVW@y1Wud{;30sJ- z_z4~ug`l8=a=4%^=-En`x7YX54Nyx_*W4Z&MIl3rK8yZ)i2}%?A3$P+tGeoppRN97 zhkJsu&SHSMPIgjOYV>bM$z!svqeoKK_4G){(q%svCC9*}`%WXA!b9-iSN`-7_=o@J zbQmh*qBz1C`vb;3+1^Ptc^FozV=+u^>hx_Y%@x!c@iTdp81@Hl>|GZ9@6H{-VD)%euh)tSEu z2h>o&^6Jdrh66}XsTEge`oaOU?$L6qGk+Zppdz)Kt26&I96)A0aBr0fmAI${d3Bmp zo%sd@!1M5OfTatrS?)3g50!Z1KZYA*J5U+5LW_QIW1%yuNKn%^vM92o5oAk7wY%mmcG~`W}&2E z6ZaB847>JB%7!rXuuiR50VA97C9HWa03~HLKL=eVmR5 z;;|j5X|k0Q5|uw~Aufsb>C;gkw}G~mtNVdyQy8x^DS??Cy#)!#dKX5vwiVzFrk4qw zcn_{w!Pd<|Y&ZvY@OA7(K+^yK1o{%FSKy138Tkvl2iRgeP$w0{eufMf;pKOHhVee< z$2#3cLR}d-X+7=;%<(UdE{zjb2T-ngMedd`=qtIGwUZ)U-aDAcWsh8$69>CJ+o8QeD|4 z>;)p^F!++^cfga%qv?i+OK?)^>RqJf!N9M>dO`B-rs!MfaK<9*Ik}`x2WRI+n@sLR zM{q9WxV%(32u9h%}5*BwN0?5Re6_dytmgsdx&_(SAr`>5S0IzT%S zXcGA{{74BJBp*Kx)=h9u=luhJb2EeTu*`9CqO{#+G*2817a)j`fmiqvKH*KAfaAd+I=$P(OU%~d={zy6Swr!r z@8RP8Nnu=QqeW^Y@Q&k`0HDAa)wso5YRL~|p&cN?kX05-hrLAu$aB+FbSKS|MMX;1 z4D>SW=2-?tCeER7hh;l99fnu;!zIPFV_!Q0${#Uf(>(y>D zCZ`j~#jsb!$^iDNmvP9e8fxF*9jo(b>c0?No#jh<&U$dwt@GLMF_eW#BOGyxoIc|p z>^KjdN}2O0bAdN26?JS(!4ED=F<@wfOcc4KkOU*4WV%SDL>Z6srIscVoRJj*jSO#r zoUG01EpQiEHS>D+P`(KKUA;|h-h4g6wCOlMKwzeK;V{>=%7$djH$EF2uwRdxk~_78 z^!oHenR&^$z0jMNd@!KJ--+6djj@aK0cVT*OkE5B%*mXa>^7nDj2T1ZX*lwFle8U8 zVi&1NL3ubKhPT%iH1MYeyLZ`uub{yeAW2*lszafHDfk4;z&HvrDAxnCfS6pfg()Sm z0A~5>^b@$T4c9@FGXP><&!DX{vgD|I2kXLhwze{I?~3btHD7*|3n$~sz!PEgnl%x< z3RWDH_lP2jfhl>WgZYuEYGziMuQW=&o^F<&hz#CX&b6~0I5L(j7yu@M0;S1lAPkek zRRY7oe(Z!WIf34F@D80`-xQqkUOd?0~B((A3}-K`{@#APcz z*Q`22mkz1M-A53_W!&mhBy02=-E$Jgg+qjIGV+yb9!7jIYJLt+0z;~i@P>d|3IHex zAyA|ml0&O|2_D=9huHFUFW7YDNtmR|w*c7iJqqcENw;z{CBVv8SN;{^ik%zJenx}( z00#B;OK2rZ9#j4r3UKfQ?B)xcy--%FM+B}zU^bwdv1Ymm_&EYxhVmoG2g^4ZC9D1^ zRKdZXNu7=&b}uglM>QqSW-8@G7{cuYp@S{!>8phV-5bg+sZqM}P6CV_ z5HM*auJeMI@{=`p(kwF@mr3lcGtR56Nibn%cV?!{lZWI~UpS8zullms=P@F*cF85U zQl@x2bKDQ6Puxsg6-9m?JVO^41~^T#PTV79!S$lf%oul1=-F9} zkvFN^ez4c(fLH6_2s$K)rlH2$?94p{yrQ)X{xWZ(AqsnT%uL}_pjJ3t&4N`gL@dE@|y_I;qQe3OdPJN$bZedn3FvS9=4*^dm~JcfG$a83{!EAOn! z>MS#$K^aob*NF8Qvwhh{_3kH(KI0Q=vNx|2Yo15dkC}p*^`B9=Mim3DwgM?D%28}+ zO+8V`nn?4!d~G>BS=vK1_HdF(PUJ@`f+Hvxcj1vvbFP^=X`0T18d={3*HlO?<-F2stwAeUdgi*%&ewG&eyI(>j8sKOI98=_sX}1O4z#<12D)^ zshIEKWis>uYDpL`x3CoY&O0BP`TNipO~Dc9`*s<==XR_vktk)G6scp(}2`&`Ysh=vxD*allJk=>x!koD>$3j1;^g}^8+4N(l zpL^+NrO#W1_T1$@fNAkI^;5rQlqJmozXFJj^4)2<%iX0MljIaceTCD=FyBU~aSlR{ z8|0YUsGPgp7fTm4bRSk4ER#N*uYRZ;6XA2*7yVj)4wQ3GfbdAPHGv;=%jx#C5yVAC&K3;pNHWo1qb$o`3&??)c{JHdoZ>meO9oOIIen5?R9BQ{K5f z@+_1zq4SQQlvNM+IP&xEIVEc)Jw{~NV+=Yu5rKHbBG90NZ{;io@Ra>gUjCxR zgaS64k6l0A6`S7TbDt*;rN@bz7~QG-Ty!|~G1&G(n1$ffbDxY;{yDNp zB8m;C+7OE|GL`4OGGZJm)2!FKNaWsfdl7Ga8 zRbpt+z10uv)l0(d0d#Q|h>b(o_24?Lk($MqA74Ap+PpRi*T*2_`eoh>cFqFT)Cr)J zn1LQQVuBZ`4St3jYXL3yJL7&B;!eHWjTz0_EF0jb-37P*zNPx&`K+FDl5-`ne-kXT z1J5C#$~whuY>b&=Q#at8I3TW#flS-moYrAnS8I#%+4Qdaah~@bFm;l`heI6toxiuG z>S#xme`T$yG1~woMPMFEO0Taz4%}45H$V$KXuyVWu`d6F- zQ5tScuaEcRjH>%aN^H|BjRrN>sK!meUYahx*$pSj;ItZiA@uYDk6LV_O@A{b`oIR) z1hlGagwG#|4hB|{W=_@B>1v};1TO$=d+iYAcAfP2PNI|hlXUMuIHM(v7x#w*D~f^2 z`Nr{iN#hSf)DU+JOFLTFFTR5qHMPuX@JnykD?OQO^ajj0V7;c=7+j2`vrmDTx*7Z4C88DBOdsbazpI=B89IMut50U02+ukue-dukIwB;X_$uCJN`z!V}uww>k1wIJOy0t^tE^O~%bfu^ZE$KmZCr<9d(T`F;1a|^sr2UCv z`A8$FBJep;^M(|ioC>e+5Sk8QW7H16L77v4TSJ4L9RuS*Nl?J#VcmO9$(Z7fX*5jn zt&Z{>su)pci{gaR89|iLj%VbhQ1+dMPK;5yi#Qfn9W@&A%;hs)ghvbqYXOIW-U+ki z8egG4b3>ABXpG5I7bdH@=BkXT@)S_(oQGq&Kc$XxEnj5)aJ@<0`CRuU9mZ;0u!Jb=9+*3MCA1OHzYowP_He`r&u5@Yrfm-8)a3r95d;_C z+!GM1k{r<5n{g1{f$hAHio}uqZG43Q_JS>qdlu7G01Q`a;gGy03+HrkY!-r%OFIGA zKh`s}->iP1Z1{*UI;j~rJ`5bw$#u9snguv<%N~#PhoTJo4Sx7T`j&vFV%0fsEAei#YpVvG=> zB(dLPWVuEz@+PwiD3JrtffOLqLaC9lBv~Gfk|DqkR~IIM9B$|2;CD6|r%t6cIgx?j zsM(O?DlNFlgprcA3}eYwrs)lx7{PNW^c93+$_M@Qvn|eA02jE(4OCZ*+SC2HX6kOW z_N>>dJj_9@6Zo;PMwO8*! zuO+iOsD2^2uL$`Rw6G^d_$m=zEIJQ5kciF$O;EQ%^M|1Uw4mRLh|rtt?D)AW`U@i$ zj$^35sIz_!!4uQwQnAci#I`Mr=&Y$fAianN90-w}4QpZ1fZ^?JOyLBXtU=J|3nvzas5>ul zTJ3B8_62>-7n{?h`|jJE#vPe%Gx!_ZkKI3#0ycEk#i8mE>J#CK@AhUnHkSh3-`44K zp@f=-TdwYx<7ebixE=tWaj0kKRNX#Rz8K17s?*;d<&K(D7n5tNr4(IDi7_k#XTSo7 z%ml>XA%Md2k&wCIu!WYEMRa+FV9D|*?!{s@P)3twUXM45X4)L1Hk4mx^iDVemuP^? zMU-sfYX1JFip^0oJAs{03{Gi#jGeu36xiyyUo+caBw}ZKFm65dW$rUPvI>#VgRFBK zFnz(mp)T=?aRmiiVqG&di)i4vr5Smbzu!DCx3<$qlF5T(&Lh#-d05gwsv&2 zXwnSf9tw0(w%uo(iJjJ6Uvbi2x(s(#m&1P#ZcT=A3r?!oot#M#8`oofPXE{}%)0yF z!7BLH0;~qa1=rrk=^a;1E4>p}om@(_i5onyRr)1*Ug?RJ9wTCv-i~qoWSML6BB!-M zj&|ZM-gVO3Hhtfh3tvY1x({*kVxKKWdfVj08nno1-78z1*8MUZYUo9vTT4IW%(TeU z5kk)?p|q(3rlk}y&P==BO;{0ojxdk`s372n9O#}5=E6fzhms0XFtJ}_YNrnFWb+V3 ztd;I8(bANR>$!)$G3+*k6G@rjFd~C3LwejSge=~uWx}aX^{I8B{(csKdB=nvib#)MkTmSR@ovg=5;8!27-rI?qVpsZ5^-+`Ap zl}GiryT?jz8z8k&J0CU5|F*M{$Ogf$OkwwG(d->Shk4QKsl|Bpd47jqdkL<4*Hu9~ z;(3w=F)oVpoJL2L-%Ztew>^bGb`;E&SFZRx6d>*$@pV>nF(#$vgVxos0Fa9c9By+%)yw<6VB zk*2Noc}@UXsDz@Szd8j>5oRwyNDRfz|czw{$Bv}qcA`dy^hIYvYiO5LE(Zae50)URzQor%rh&8!?|oHqC6<1;>S8s*i?Wdi|ZqWAxhM z)FyADx6q=^2z$BQ&3(6 zM^tetktNVq@Bj_KE2afZ7#k@aXNA?LCW4vtGb>Dq!4eomfEC`1t8kkkArJx(ZkR-Oma{RxFu?{X!6WN1Ov0_xDQvms}HY)vGZY! z1*g+%ACe7Af*s#q*$qY4?+b4RwV-y$c@3^>ST}ZLs5e~EW$*t8cLi?0#;B`z;C8Il z#5&j_pUVY$bvA=z!mn`CxhQAU6(&wPGB$c zxa$$89fN)kJAm?V?^0uz-y(0oIeohFE{5)*vzh6N@ufzkS&zHJ|6)b~?wigSAs!#j1ugXN2*Qg{=Vx%4Q9SquSLIOf=tnsV@#EC z&0cYF3A0sVy}uJEp3h%f{xDXuQSR}OKT5r60J+D!Gr}qV-Kg>w{ zm(#U}XxZy|mH)oMe~FM8gY74i3MpOXY{v-d+Ggx$%cGPF zy>gOzkWg~$TK};agSd!Xi{8pnlABg3d8&~&BC(&YFYk%=AC37w{?e-8LzI?0Pg6t4 z?97I*I=Gd<03~pmaoahbUJng0uV*uUYoGpx|8Bu=4RV~O;`sR_Zn?OxpflQ#k^`;rO@8G3f8Xn5Af?Csul5ElLTdd14ZC zp?gz&gvcOpeFYW;jyZNiSSP#&W*{qSR#>WRQ*)I8AD%}OcWQTbyn}KqH)5My?X6<)y13KKf(S%lQG=mR6qL5yO*aocDF`$e&9`n zKH#G?aRiP30E}KZSp=R&4567C75EiCsOU+eXddXP7hsBtB=1V9K;R&-oEJ-3BZ}oE z&EzHBDd$yPESEDx2~`(YKsj=8t>l@E*y_DxaubLJE~_$DxXsxJAhj!bxKs980;l;i zHE~>^9nuxL#p|2db*{W|D@+STqaM?QXJU8lMq8xK;+`}P=`XK!mnL=Yp0#P@ezgt z%~ctgTPQTh=ZoN*Tkyf&L4MUouwu@si^{3>P_UPW)L>vONBltfsu7MHfy?9>1AW!A zT>ACT%n6y3zrp;lbCgwO7Vspucc^CAL7Uxg^rhX4`jHb+LNeCB)y z{aLA8grjaaXS=PBK)V|(R9a0jUozk}>*-bCoJ}031ghGI1(cM2=uTP!;bK-jboA#Y zH6H8<%s>u>p+hzUC^U7Y!eXVG8&M6lIgMh&qd-(5PnF)z$JKp0q!;o(WYi;QsGZ%w zmrA71y0*zf+>ff(hS%j^t_{u5l#T)gx&pgNanwJ|kTG3dX~ddmva|z8APSa~o}09@ zzafCQ4wL%s1C=m+JsRs^9XRVz7I*+5O3zB^*T+$jRI?wySZPa7mhtePOTXeb zmiPQZUas^!C{Iv&{zLL)A+Cux0yXMt4nMpIm`D3WmuPeQP@q;`W@90H5$o229#l@5 zT@9&N$$DcoY>D+ZJn#r1Z*R6JL=+FUOYN${1Z&0<15&E_6(Ce4rO6~%0B&}zyd3yD z0^LKaF83z;W) z0DcU2rBr{3X6fKIdmE+KK{qA_GfX7cMWih&tJ+3W%g@~@5Vax_$yX+tIYshJ1XDpz zpnbN`=%9n`qAAosA_HwvWF%CS^T+BJtI?|P&$ z7BzZ40uWPcJ^#j|iO<{Ua9v<2mC$6wU9NnlqOKj<)sQX6NIUW`Yl%ernu!A9fG{`( zRWEAM>b%QpFNLL8dn=RI8?7I#9)@j)-tNx@@B()9_fzOpz1k~=GW^v=!%j;}QjT|T z>8Cd9feJkw2>Roj+%euaX~y%p=-2+0|W*cgg~O&_OMc-Ji{VjpAWtN5pSU#LNH zFBh8Ci^_$}iWr0MH3W$YYZD7q&XTmvDxXAsdEfyyAp-z9v|qjzR1;k_6WaO0Ag=(* zUhO)VV1cF3D8nKo_D*ck$kEwgA>U4F6uFgcG~B_?9tYkD6Vp*1UgaSI&r`I~T@6v; zuSpv2P}a*sMI1OPwGyd8nly(_+D2?b<)M>$x*E*xaaVBi5?3nI16Yv)!eH>tO~~Nm zGE9+}CBT!Ye<9Z+A%yL&29rD%7dI2ZMNB5pKeM+{Gf9-&h@qfRs$AZH+hQJ8KPV8$ zl-`a~{Cc}92DUO@A0Zj#zr@SGg)gAN^W1l2@*w%P5IB+bAYbrJkwHdvD(@nIrSig% z`5A2@l+i*A)5Z_fAP2jh1ByJ<08h)h@pKXO4^i_a5Q+APWGWZt(5WNr?14K-iY*iD zAskIJXr1MI(w--|3mxFE+V(tR`TYrTs_$96^F1tD4YAK=A_df`EOQnuKt`fcHU>jXTjH@K*SOh>o~;(8z(%;WRNs>eBZfZ98{Z0B(E3JLiKq} zUzp^TJ7CHP-_l=(WPYSgMmmcc16#+geAI4HDslH|9Q1V%_oJe;LgfY>8QXXcS7c)g z#10tlD#39&J0oYqv?;R79K;2`8Bh`{GjDPahtJGA7`JHTah}7Kdli@sMq?+<8^hNc z%7nC&`7rVUtS`TWEv)otvfYtm`BHpI!;;_xWA!Y;$jh(x#Uf5N1`jN02#&BIk~j3_ zO#?-n`l4Tph^DKS!v&c8Qf%%^A>%ITkOG|7!tZ7dj_ARPn5&BcO%ZH8YB1nET@O!q zISDOD?-G#XqHsCEL1*8B2b7*i+)*|!Pl%nZuBF)ZBBFqm?=A@a!4XNQT%DfESK8$r z2r3DFEQ%>VMPC~UBYi>Vc@UwYFG%+XN<~fY3o`Q{YHnZ9L>@%#>kG=@L2pC`-OYox zL;1YkwH)Kpk0wcn|V-kWKc5?+7}tr!GjJ&2A$_YA4LRhY{f^p>7_{I zq|w*LBN3^}XY-(r$e??9(22;PDjvilgMP<@PDKW7=0Tm2LA!WRPh`+XJm`F6&gD2Bq>KQ)JLI9%POTn$3eOkwK+AhzKxT{KjN_l$TR@IH3P!5a8hs&+F7*-d+Q;> z&}C$8BD)Q&P`FYbeHu)HJo%TXy^}7~Mb#RmoEjKH&vBny22b_zwIf*^CB9xe2)px7 zFwD~Hv30&if_ZOMCYVBY4&9)BKFrxn%8YMo8^Jx?!yOD3bzJRYat&pN;E%T~IE_aJ zN2~yds1(MNlvwR8R&j}se2s!zt>mFl?(af;F zf=f<;qFY5mrO{N~%cGCMAQMM=GU(h%(kIBAMJG}q*s)taL?gzrT0DfI@??Ui4Cmj{ z4`-%)M^LyIyOLQVBw%sZio}Ly(L4dAI#`UB3nQ~p;HYIdJP^*1T+Gj6IG7!}QSe#s zG+vMfHPb&ckTiU#2kUp@QmpagV6Xw@SP3uT#_#BXDvmd3Bz9wFDox3}(BQ4?A{?a9 zoY2?2w)FaCIFy4<0GAI=+f7ghN30BYH!;A;FyzK24GtS=KP77dXx2-hb0gCJK#N_R_loRlqAhn1KC7I${82<6M** zn(Kd}1%unsgO%DM{;)Q#8OEmc$4iLUvegae`3yS@#oOl>u{!`X$tIoJgy34EeZIY@ zJPP7@@a+^J(T5{Tth^kU_S*SI%Pb=R*b3c@iw2YFI9ZZ9gt=ef0 zVi1y;gB^qzugxUXeKdG+egw7HS*#{{L8_s%j3w-^@GZs_oo-`#P?Oky^KVjiI&o|) z7@Y(vKs8(~x6SkLGim(t$RW-`w_%4uaSqMYu&?rHYh7mu{WlaV6YjrVXB!& z)2=|GcWE+It<{BQJ(RnkYJb6~BAPpa<9Xj+0tsBV;j+uH)a&lBPG61# zG8>BdF&x#n8AzH9<)I>5>}(1xen~58#JQytN^jEoABB^lZQyRi1b9er+Gcj|bfo5A zO4??fO?xFgxL_m#T^(})h6a^`KpK301i?{~-o>{$!4}zNP%_!zVLRD9Av4+zl&l%+ zV9#TaRX37}Rhe}<4>VvK12**T-6~9~oktw!&JVw;D4pxdX zM1pWh-D|?7e1Or>5Z1A^?gX6rhGafT&@8@({c4oBo!e zG)rV@+b#^&yd8a|fti2`MzHI^vqEFVz(8PRny+TV(7J!&#Mlv1SXLZT=+-oX~2xH7dzOJtAs)rD;-5Kfp5NKv6SX%jJQ zKn4zW54JDM8d$P`Ft0BcyYWljHkN_bh@EuI0wpX51D%tjou3#~f>G`vUEDe-${nu+ z_3o6e?AcNBNNjbCL{e82c105?Pnoq*PAOqg^bV?h9#ji-#)*wD@C`iuF0RKwRh)N1wb(EHo4T;oA*bROUigD+Osw=4&N8|&%!p+W zzrcV6ouZSTrA|sYK%JzX^SWs`I(twmrYt&=*kR40(J1-e0;)noZ3{-^Jv1WSbGl(M zRs{x;9MRG=F7(@eVqy*Aqg20KfHs~3#T$S@%#h>&SpP@H!uW)68I7$(69 zmgHS(@{aQ^O;HLh!7v$ccF@Qc?I(7?QSd&sfI1qt5u_iwZXs^0A~uX6{VawwnYTvS zs%ECj7janaK6%QFRr2*S*2>qgVc`RzY1Md7kBQ+5C_h%yTIzzsL%D3U{S)9HZ?r zb_S~VS0O@^`AtCOeMP2fKULTQ*{-le*;*h^B3MQ#EJpG~V}^t#q#lHj30+V>(-o zbV_dZm%$yFAaBA)YHsy@9y2zSQbJ191Mep|a@YzZ6P)CWZxtBveA`QNG%j(FYF;2% zLfCd09ObV<2xgyP%oAI#mprvd>Pm%uaVb(|#K@O$`c3kzLMZ=oMVD_v4hQ=X9FJ$g z$ZU*8<*e)$z!EZ?r9(Fn z|7bEPj0;coiSo z=H4&f;4+al%HG}$< zdPenwM}0Gc#0tc=AU+s8iJ=N4xpec({{*xYvARW3q|u?Nctx6Neo;_POJZ+Q&jFB~ zCelPeM`FTUI)W{^xo^z9H~3IHOZ>gj>~Xv*9oLrmJ7NstP826RQq~~;xMENG!(w%PxvBUd9Q+_hZbE*`aXO!NG%JE8C5hog4nWqZ6_ct7B27}J+q_D`@iO)Y z5-fL_k?U4&EoNt9C-V>>dkE|N2^FB3k$m-Vc8tERCljyTpY!4ju92lQlVN7sLNn82 zFeA89e0UUDAUNMN_11SM6+&zo-niM-K|zXM5|>~Kn8oV zuYkb3SUQ&~)lS9J8cm{=q@>{;D*aYS70U5z{l{bYjgd#xBraam%^q#sgfp|f7~Rzv zFu}?-*RU8-a5^uT5E3|pXz~m5A)cG`l4mcTxFiWZ{3#f~bs5?r6qKf~KhRT~HypEB zfjqLn9j`WaAAlQ!!^+8CvRcXO)Ru>Wnh%~i;;{EE3PJK4kURy+lU;_049i?`Wk_Cock&Gc|2^&0O*j`)CO@PK zyO6Hgql{JZ&wP0u_TaQp!4b*O+raP{ve(^!Oz*b2u#`M?@HDiy=N-5Y6&HAsbQqkL z92}8?w7?zSvLRX8**zG5Ww^=}7zu0vV}KdE&x{jJr?d>@4&{WIy-Qa_Me#D8a?e+&bry-jF4lzbgdb|Ce{2BZ;;x|I4)O} z8Rs^djha5~QZ3Q}1ju=~$<>?zuNPLoq$Gkpx)LC$*Tv2lE!7-HzufMZhu}0ue&Bh0 zsCg#}tUaq!Z~(q4BhH=RiYrx`^tCa8zag?VQSGs5>o_j*?XDg{e8O&pQUV1Q_P|Xvq;5hX0Wy@##@#YT!&z-H5ko)q709=d zX9u($;mSq*`k|pn5juf~8el*vvWU!{;+6hF?mwI6y za7k?nJP2Y_=={;cs=HHddJbC2o$3b%dwfax){Cnbx)Pf5EbvUATZRoTCz!mPPBz!^ zPHzk>&#~jBjfK-NIEWdEcl4MuGYY0^38nUBA_WdnHPSv!c$`$-oopn7byU-%eAydV zj_wDo!(UyeEAAd==1}zV02JTKDZYa!ewm>75LjxD^mZ)4CDq)32&dIg+w3tK?qKbJ z_b-tCL|$%*o(fI`gxI-7g1{GOlhd0>jcC#bev2aRb$fiQQEeF>a`4c?Sv*E!5r?V<+_WmoN-h4xb@TH8upJ=$}9n zNJ=!>6}{Bjh&l21vDTl6e)Tucp zTG!w>C426Muw1qt3S+LFFIL-1puD;WW;N+E9H%g*+n}W%JI{SiOP+M5&N!}?4#3DU z^GI_~SIp{bVHUbj%aYzw?8qrm{cv5S zS68jW1cx1ae8i~#szL)JzE{!g+80@SMI`o`SEVefuIM@PKDh`}XF`$`Nn@1?++v)d zR?zK!;h;v#UG78bV>By7lY~cpvvNXWzBq_M$G#q~p*reiXO?L1#W_f^SiN_CI(fHu*TUE~|s^ zF(&g_Y9#@B77}9cEsA1wNLW_piYfKI_!7S1=+jq`Qe7!fI981X{5aL7b4zx%jEYZZ zr91}&N2L-t9SM^Tq%l((FjFfcIuIoeoZ}-(t*b(G#P^)_5g&CNMSY3U?Q_uW4~M%wllL)|6RSMTyZvF&?IOs} zyS*j%WL%9^6dSr8@#taYg(R50aLM94Ql?IVv?Sc05bK<9Io zG`#%YAIU+^%Ui_b4e{8_AIK>-KwLUDI~!n3SV78t>hzvo_pqjD`lMQ5f)B}2o2>MF zcV%!I1&_e_IhaG{b@pe#(Z?$`FG3p^{OC^!*C#-xUY(3{kfX+dmp%$my^oq`G5jv= zg}g%=2j8RSzy0lRJIFuYq)TF)NaqY$^1z^XH5!tcWzcXyZ`t52n(ANsAD z`4KFihk8RCf`r6Sj{zHT)i;CXbcLKwYX`MP8 zBd~`?;K%QP*>qvBF#F#T(-Rtk^H$0!nQK?b*QjgDLxb-EwlZk&1(2fw-Vo0p#WzeY zkOJAQ(@*RJ_nU@s6Y@eCLxV(i1R9p5kCGqB$t2`|-VUG#yBj$>6`WqUj2~2k! z8Azj${2HK~iaN0xXRo~u7+bwdI@k;_Np*EOi6Dn)yn+QPA6R-%RwCY1+JupVoCDE684d*pj_EQzBGrrN8ml= zPBX5E`A1W!c^%%;asj;oA6?liv6_aqQP&ks91JUfWB5Qb_ID6$K+-JZ#swI#P|j^E zrKdt9wy~UEKOl#M7os;m1Kd^j z(lFZq5*tSpQ~P**OFp*CzTilP!4_aQxrg(wEYS;gW+dR@GX>oc0{#lr-wfp%VD?bQ zHfZB<1j0S8`jW_8%$i0an0+ypBr&dD(c^r76Z`b-u$Ch=a&QB+(7m%;v3()7&=R4H z!3Vt52L1wgv3ITRhw1&?6PeF4z$4W#U@S+epNgO?8vCnGR$pRRY)CbY_^Q4{3lFH> z@h-jHfc=+LunXc7n-D?{u3=3;9E5@%Ll9?VK3g)BTk`=qogsmK>#%n_A%0fo61HYu z)rQl^d0M`%t{@60d~p@uL3k?M1+W5k zJq8%u2zei;@~}an8N3y6gyyRt#u#`Hi8Hf^MboC%i%83te^3hr{(2>FZzfYY<_Gdl zOr;feRS%7kyc4KaZI-9k<$#09lccwX5|6v4`XY7u>fv=ybRd~&_Q+9KH0DHE53KnD z+zxgc?hml?&R6Q;O8kTpG_KlGjD|Pa=n#yK{l|p}kU_S-LNg}yr%s`$WlmM^o3D3z z;kZXX59gQJoq!pgwG5ntpYX4}20bYbDJ#N*Hf2K7!5?@wfB26y+#t$?J)8H`F1gT{cM*Enxb&9VB|VLK7j!u&h5Irm?}|mjSt0`L z>jB!2zw6j-5qbk~x0<8RBVPr;UEnv%+1PYawxx zhp;gSTE`-H;VOh{BYj5N)rZjqdkI5|JGyflmqEWxju(&v`)IdeK?ea64KiX+^LY3i z*7`lcAa{UGids65PUSxX${`x=LTa1ui7FKB%$zEZw3ol}SDVhI4<&mHiXQA;R5wwF zI5TgPN0yXtiA;bs3?(3qtvEED7^)4Ic$0oT{c!mnWW|}LQ6m9196;|f<7l)K)xu3o z09~MS{kRNEUiTFEu5o52%M)CZ-S-65=fY0n&p4cDljLb87`;#6I&Ph|%e%x}{u%&r zu*+bTLYjyoi-60{-v3J&)8k!}OP$tMc!tFsg2Y(D z(`4d?p^HA4XAYE*{4KmAY7!~dA+v(;g;Ts^Q0nrY0bVfIjT7SIHP{*+UIMJBJ?H+Y zq+%7Jl4WRUd$;;jClJuj9kEK|{yngR4Sc6~JxLHryxT1JWqbaFdAs%vG~MeX8*^Xn z7XF^f-^nep*EbpObj%y`E&GD`xw^C^@M~ zp81G$Dl=!ff8xxv>D@n4-y{giH=%Pds2yxFPs(@dZbVXu$v8fMeS>Em?3al0dQyGj%BTvKt$2(OGs@XNN^Cya#YR^0t4rb1r$vX1}GZ6dI;e@ z50(tDwSMc*IvpL0=3hWcG24a^vBg&d%FqR75_r5iax9H=vqYXG4L~j9Axz_3xkZ=i zV2|RAA;%-^?0EzLHU~$LwC&~JIMNPmZ6h4zmpIA^yby4A@;dp**pJW>=xqT>dVlCU zJQVEYP3ki?sz2rrh~X_wKkS6--5XWj2atKkX4~OSTOd2|r!Z*4IJIi4RS)$7od7ne zoWRhRLtVz;n}pXAwh1~#U`u_0dQeaSgd$MDO}r#v8N|^LcmW+#%vK>R+fK|*K=&w5 zgH2L*bbz`p%)EB&7DEK*I^nV>bBU$mdbnk?_$rOT+d@2Q8D;_AiSBrt)oodSoUanZ z2&3b<`F_B5Q>ZVc_!7a$$j5%nAU-9~_qC0_@7xYBYR z{7*-f)7W8$4-;9Dgm;7#mgqk6)jjM#E1)3is6*0E8k) zohOK@Xp5d)9m!+MN&kUl?3=Gbb6Q+e^h!+mR3vKn5K0}Y0R_;~6w`nQz?HCNogC?1 zwv~GKjF7-io%UmO z!x7cpfl8aHSq!45Rc5PkV+aKl&r5)y_Ilx_@>f!M5ty4Hu?=UanQ+4HEPyH))Fm0+W5Onvu( z>)_$-X81W#>dm|^6d0V^SCtP{QRJ@FtS3?7`sWmQ>~YUg=cPdP0iDKBq401I39e0H z-z5SGYJM_!hy`Nu)IthQW+y2ae05ohZx);fCVS^uy!lB>g~ukyzdYA~1->?LflrIz zX(&^aHZR3HFG+hny`_l$v1fLQfl_^HA4ulP`GM~i0=3?4(=apv>9v?`EDnea3j{$1 zeuSr?UY*^P6rj6x$r(j9k`nQOSYMby*|67l2?5~scf&y?f2+{Xl03!Y)woRZkdK7& zXZf`JU!Z%hCj%J;A1BSW+C1rlgBQHM*$9NX00m-@KBcCfW{ zhHLCYBixLy+g`y4vG&MAgB(Vs!CZZb`l;eW^-Zb|GcTtL2Rr&Dl}c?^c@w-Is)5f6 zW?plmeU(iK(xI(M6o(CqOvr^2Yq$jEq7C^aUxAS_UrL{wIRz4iU`L{IgkgN)Q!PmK z`lj-iv=Epdpl2!IX&#?%wCY8HQqB9QO7-zz5%~h;RCqxqHA=nH=pC)Tz=0Jl<^|<= z?=h+zu_2g++Y25CZNvI!R1dGuN8Ly+kUTq34^~}ksaFtYcU6`rgD;R9HK~J;gmstL zf>#^gAwJ{0?#Vn#nXU9xuKK#17RbTrX6Qdc1+W`7ph2R7U1qXb$tEvV^T3ZE6Tu-E zc!N}S&eIc<_xsyo9E(k4>nAla05Ahd`Jj zys&LNg23>_Pw&@8?gk_Q@02`gVRK+!4eDI~^AGJAsOBJv5o!AX+gWJ8ElL4aP6 zzgI+vdieTH2e48CXE21tQtfB&k#H2p%5II)ZLESUjn!?)=1eGzl@pz9E$u03?Tgb) zG?;ZQ4Nhs;>@;{-9+qW4ZiCC-VI%J4&jmeaB}<1>T76)$Q`%jR7{^7t>@$e#80w=Y z^rDl`N@X)V^`HY}oXlSg^-gKxG|z#*+jLvMsl=YhK_y^`-&1`7wN**VVbn@;tZr*1 zT#3RD@gApi$xqdo&PgYccUZzE)Q`G9N8l$npvopBTQGs3`+n6ao{qODZP{f0LRFVe z^K@<{a2yk;9cQJ|PUPN)@;*d)A6|>{8W1}yTGX=mDBiO+^Rm$#g_;HFgcqr6;AJOy z=!Ht&ZP`mN3wEEC7BiH55v8^wsULsue&=tKN9 zB4{jCkHC*9@FS!@iL_04N12PyI;EuGgiatYi+&{K8tBHWkJ0;-&O0!#@7(%QHZXf& z%E}o!UHk#)eK>AUm@ ztD}I{(5!S+Do3lT{Z&YW)E%e}MU{@t<_$0V1XYILgLrQ+)QkV9>e*~mh3E7tk#;|j ze-PE~LA7O{pcTQbRX_tuL+GxHXh|t=?@K)SgDpt4AkB>5CUp6kDahF}8^vWkg?Oa> z$|;q#;kg@VuE%>PQg+~p_tIWGLHK1D7Aw1{Iux-|qnC5|J5ITE%=@1%i)4CkhSjz4_o94Hba1^u}CV3Je1h_b#p?37O9c@AZ-0cC%(Q{4m4xyV+1 zQP9jr6vf9&RuC0Qo+mNz`k(`P-Z|e_>VgCT?COK4k4hgl_5g>zzjR*GJ5lhWY-p}N zN`|(|Ma!zC7{u{Z>R3BZFL+1b1~4Q9uIFrJIsI0Eb*%m*6)!z%^91m?$}EnA$Nn2o zNl$!6Ll5nL;@@})JdQBVBGA+ITRLZPdaXC0#4A`7`ePRgJcg1#qLMpMf;=Cn4l%9; z(Vn%9%!8U@j|+%J>OZuK!5ZdBXN{-X)w{adLi@dwaD z7-6eB0n4xi;yrjSNHd<5mhRa)206I5XP$xj-191e+D;R}gpgO#0@#nBJJA83w7^9$ z5>#Ihowp6|;1>G~z7plc^bm-aKF?tWM~HLA_=ptfPr zL}*(_hpM@q5^f!ful_N%cETz+eAcZU2fpGN2N7l>3pQg+gdN!gi< zhf(>;M;$HGm3Bf?H@>!IaX2DFstV5`O=`Pa$9v7toj8n3f(q1PaBd zKeq=ELZ~Q(WUyc_p5SXGU5Gn@xMP?cF2l%=T%MeN-JMq_`^y(@LmlCk} z+m8?uNHD8WdAsWc*=yq0pbeD9OQ904Qc@%2)&7#U;Ohq^ z5s@-JeEGy&kG9j4?C|q^ff_1-U;~ZHBPemJ7~|0$cn0oAGzHQJK&cNb-~lU-&IVPS z>>pJF3Pg{6(*@BRWmm0At!@#d^dQNaol^z^lk<_?9~p$CRs9vwE^6=_(s%apGY`y_ zgMqv7<9js?CF??ury4yg5!f5|wcFa((sl-3+(p1GU$=erSqtNAkbUOw^4y1HQL?d( zeRig6XSQ44_Fdf9vc9b&@Gp!aX2TGPhqccwU*irPS>N?<6n0emM_>~o12e%p+gsaP z4^PpZu<1kv`i8czk>6@u!1SYnfrn`P)1XSTNM3ofrH6j9S9oiS|t$_sO#e;w={XPg$ZYd1u6-5~X zpP-H3Z-#+@`$srKLtA~@(Zf^50qD=#zTdK^5qY`4wz{VYqirp%NBhFsj<$7D5tm!r ze*byLi|)~F^+ytJL17ez0ud%B@KY`gqypAyJ!CKQ*-C zc9h`w{}0u)wTOD!z6e#1fY*7&UmW=E!0x);sJN~p0*V7C4xEVF^+Q;gA4tP;vbBMc z=TfbUI+f-yda#DW+i($4@bK&O+vWKMezS4w1s@f$A+m}(5-k&9wzUkD7%q5dOvU25 z3H}oGnSa1SQ~%$y3$nWxbyjVovUQc=vg2Anfiw<7T}tcaT^-nSCx$22c`%(bPg9RUR*~s7nc&jHGhY_Xg<@RtizhUJ+H@Ey}?wemkn*dG_~~83N6Wr z6ptLm$n9wB1TXNneb?UB0O`^Q=@P?S*;DDn%p23TA}yu?J5V?^9Fk&Nm9$5{M?z+eAA%bf{~(I-E<^j9|E);?a^<1FX5}SJ(Nqwlk|KVpd^$a z+NCemhQN?UGqz)ABxD?cQXt)SR>BE++fgu|&S2);B=_x^Zi{<;8*4pCzR#l{k=olp zsnHV;=Ia6spi($h3(JRO$<5NQD=&!r13kKk#<#W* zC)V9dki%Vyv;uekLO1cOWo4}g)n;C`dw73qXor3@CXbP!(jG~o5EC`F0PvvjcHFdr z+dim<_EFsEnQN{l!pab??DH;Xf@8pJ#uA*q9{Mfe$4za9$JL(cxyYZi^-qA_VcjYH z`g!nY9Ld8k?Bb+f8~EpW#JcqNOxh{c(E1yPhMMqDShQ605AjVVsr*}^Z@ExmLnP05 z%nwbtSOGyzc}L!;fL~DD%k8$b<7sYR($Nk}q1N^#tUbowzN9mFkhX_C0o2Go?CHgp zARHrKvRVF$e+75uKPE${B7&!51iuoN`7dLrtLcBuG!}Mh^H**sIfR!Z5t+hW%lj4mi|zt$bbUSSj%VLO@86`m4x#ONo$ceemyajPa&=GP?>YRv zg1^7v?``~9@z;RAQTR*1-+ug^#os0T8PRmkg^C^9F)P77V!8#Ia`w`ED%nJKL+aeb zEnKtncFXu?KH8ktX7>X2`y#O=QZa>YT^vofE{;c*Vx9z7+IHUsdfsKx9&!}3B2gS} zt^XOZv*?UUILe18-0CNVFxZ$Y-euD6VChFuE%+L&iFa|XM+|M4qrI)Ty_K1HA@EwE zjf0axcYq3j6wp_Ih3N~Z$`oN3)31Mq{m?=>-B-SDQ7(4IO>E1xQMx4|`G+~B1E>UolV~Q#QF^cR z+FN_8S8eV6`mpr@(of9;11Qyi)ItFbiuQ~{F)HREnE8Hd@0pN@+V=l_eOj8K0Ds3LV<^Kz5*7v9b74rv48)eBP0>r(O; zIDH9v*)Cu3$4o{yoE!~MNJ`+_&H(qyH>E6EP*&w$;$Gb9uthpV5O=dkI+WhWT;W1r z5L%=&!xscdkZ2_7a;HsI12h#4MXYM{%S$rs^PQ!&DjO7ucA;TfDs)@{h&O6{3LDDo+KZU zE|;;`Q-Ol_W%%TL2?xkgUy*?uEQB2-(F){5Y}R^0e&uIzA?=zhahPV6FPNO?I@HM? zdzx98)nbk7fwK@7m9|48Yrx~A#;-4iLo~U_K7BsJK>Mlpxzg`&#>>a%oJi^^z~G%L zzpMxqjg;-lO!u1n1>$2EVK|Qi9fRLIC)YY=7rs%~qfPnfRD}5-y@DSktr9#LRZH9a z7c)NCpGsNLAp`foy)tmdNOdCL@P>H+p$0_N?z&5IpUUYC&Ug6x9IMtKOLBymu3)jlUC_0*(`YrfvN5H-y z`bC-pD3!v49unFuz3hX?_Z)B|o^hZAk#gsXE&+(&6ZWlR9gJ-l)ChhDiHL7KZ518$O$59 zEtmQ$>c|rO*?PWUrM4~bjO_i9eoJGs{uGenY!_%)M;x&5YTYs{1nQko1!7ofk!PgT z$TPCaTAPOE@P1!pPaoJyk08@bCHyN$Sv&Qig_;qX8d}24&y{e#lyIGt&=EM7GWPi< z=4J7$RglX$Ga5;H*elIsJG5wR-5hY&;&L&NKNZOMV1SUlbSW z*XDJr4h61UV09b-M-E7yL*B1prrzp!maAy(SNhy?0gvvs^-B_}qJ7&US6WBHloz;C zExA&g9396i=mpiUC%%k*n2^9s{G8ns%WKq6C1I0F(wG#wi5Goo`P-js@d9w@(e(^6 z#v`XR^3;1f8FKkjN6e%3@+03Ky_X+wDr<%7H{jHz0hwF-5qauBM+5}`F++6#0ZM;= z^2RcpGqro--8n7Zi}-W$w=}24S7L3c1>nei9l(mgXpz1C9;mXj=$KUXi2Q_?M%GqW zSPO_o^h-%io3?O;+3vRt?>DiN;+;_99c6W4k~}JPw(q}?zId84q>+!vPxyS^q}`9m zPdJ4)Y5ODcgXdD(immFS+t>JWo&__s#RSoFElkRv@HNH1m~enW(u~k2CMEg0YW>ZU?evXlBF+9K3Wf zN<5whL+~rD-%D|A)}sT-t%irdYAi+k_-^2Zu1lG8bPg{=t!K=TdNM4}K*rFs(qF7+ z{D<^c8Q(Jerd0RU2DnZ#vHh90u&2V>@F>hAusL8HgEF0M7KIe+VqoFVr9$LGAu{uqLQG&pBy;xc0&0ACT$gKE79jEF}#baXnR2-%tidGly^3y@hez7{&^{uLL0>R zdy|Vzm+Ay&IM_KMTE*9B=c3yG4U!>$ecV*#S2UuCuc8_ked389PrSHx4}S+6 z?kGpBRxt9;F&m_vrfO48s~v;NkjuLdQKcvTx?qDexD0cj0^by6jVD0?RoaZtR%vqh zVg#6;k%(CGcR9btlD`2&qFvk_&9GpBCn2DEEe*mHg{}ot-2$YjYj)Lg0r3_JKwV#f zae)Q_``?q%F^UeU8k9W{3dwN|jW-p}nn6Kq`-d71`3iN6d*{iZJumR{ep?q9RA0a( zv5RrCz361>R}}wu2FL%rjB1>5Zx3}v8l;xFrW)PQ-Fya}1*hYEc{rfMC0#*KMpX~L z@jxpYPJt~7vDBDzTc*LGQD7kRw-;DX{h49X1&i*74`IlBg%{q7#PC8uLCy*eO*Mb6 zZjq@<*T64WvlO&8SstXeH6(frtN5uY&B+q|de%FuI<(sXs1{!zGA`P;#eAURs ziqP;02qjYPzZO(YSX^)l19Akd$?pvi@e}U~VGgwuZN3a<7!Am;+n}k&-Rp@vGwuD4 z3u3aZR`6-@q++6Di$pEO@WpEIDIe@X@2sZadBG{4%dfBxEwVg+TRf;#6DZ)9@TO1u z1suC{w+B z#q7vbfT6Rfp65!>@lo^X6A;!_EqUF+;6vyHRaC54#Ljs^Fep{zwBSNyFGfFF$||Ck zRg?u^O)*qok+UVC^w>s^l>jjB9vj|Mw@8n#L#ss!wzotH(hx#Ch!~-(6f7izJt)~U z8A9fia6!w)--XYQPAwIK9Ti7D0ce7(WX;ay~TM7mJv4-k+9Qhu+i&D0Y%;JyR;p#C%L7%tM(@&Mrm) z9Qhop?N`u2br@15e*@mMmbpX&fs@gdR#Okf???$73#^s@BWEwsI{I@UY461xWuv-X zM;vODUTAas68)WT;j0EkKiSEaIwSKO>xP?Xsk;gTap7SLtUTFGvn#<8 z0x32f7bTQ4TOjV6^7gR-5-V*f=3fKB44@kfuID$*zq`!5`_$XHYqa5)3+Mp|v#9 zA8Eq8GisneyImUo%u4mUTz-S1>LqqJPZ}`&pKUI)es8CtH2vypgAFRlGxBRbBM`!E z>_?sfg2B5jV9D5B{*j-qq9~Wi$mPcl{ynj>hM2B4p~pe8wwb4VmXf)c}|@jFE*+ zsrhMLLVq)2?t(RD*=bgH0wGaKVonqE_g-t<|6`_)_s9O;OYkPuJJ)-!zjq!UCwkEm zkz+*v&d^#~)l?jZ->nJ3`Dhkj;k&@>izxjz|KoyaHvi*-X|_-)DRIALsAQM>lFYu) zG>KZywpN}$@U9bYTNg`sol$?GJga5&{>P=`w#%t!9RVCS zXJEIi+?(9qAV7{xzskogu>_fR`cTTyVbkc-}ikicl6!q9S?JeNR4b|*1ti; zH3GRc28AK1c2!ui81bL=BvN&#tpgtz8_#uw(nM>;WtFpJuj%ksne+cEeyv{ z3mY8I^6LZ7OLs!QU?s}y_Me(8vhmjtGq*7#oZvr|x(5N(qGIb?s_nX! zZrufZ6{?iNzwig8;UF~B?@{J1o5N^NJZ8&0*mQiQTT66ZY?*j(9FvI32mw~D^*rYC zl$P)ijl?pocOKQtJVY0Ci;As3ppI@Hkh*k3tjnavuxU0p$T!Ffb;Dxbr?nXkogng{ zcHnP|h*)vom*Nx|H4ikYEV|f6$B2K>zF+l$7^2H#)ECe`)%(Z$`y=SOs~!)k(!1-x2-Xt6q5-^sBn*4C{=59#_hsZMD`1^r6@3OxBf)7yXQ2FzETDl!_x-o;t1G3vji zXn=sb-H~|1Kj>#muk=$jrhHIiE>Vpcz~{-neBH-?Jg_Uz6c|r|8u4<^YNpRQzCkc2 zoIcQhGQxP(bt-sI`RxA*yx&ynNvex0VI#%rD^~RlLbdkOF?i1pDC`eBznm5_o4q4+ zrgF;-Gm7OVHXfOj7oChT2 zrUqFJHO&`NDR0H0)zATE;mFx%6z}xBZBm4+zIg*AO_rfJ6K@5QUutApszo&z>i`_! zn~5*C9y~4`h^AHaLaW%>yyt2Pr5t_fIc?M*fw1VD9C*%@Xe9&Eq2x}^SpLZOae^6H z?K{VRYS?;kG}X9Ij%fZ&*eZoc9_nl$_Mw~ZIBNjMB}WIpHZ zYS?Sk&n8h6s7Axr)r)ix^iSqR2CVUCnM7FEeqVDGCYlY>Hr>#|fO-k2&4Tqk1P9%X z4deM7R3v1lQm95C(8j4uXi8+ImxBobXww*ZSdYELv{K6{V^gCwuxkQx(1|XxKb9j^ zi{Dy_<~S{q+xB{qYZ3o%pYcsaoG()qhoG0SB`APSy$w~?L)^b>pm!t4Wd_su)!V<2 zn5jN%WQ*%&F7-K!+ZBv3s2waiEG($+?N|HOx|vIih5#MIRKA?r@D0TJPtx>Hr!5xicAcdE6C%&e*Fyw}7Ke?&x4bYEh>+BSG@bL)LU zNSQKF)qbwF6jfW4RE1X1#cKz*Mkt0+S#C{2qDtmkH4XRVF4mMSH0kWt|8MjPqpcaB zfL^RS2_*a$tQr;QjjN3{ab2j2O7%vsF$0vBNsqmAJ+DJwm-CmJ2Dpz?(^k!fA(ZTX zOEpgAEa<{`5<8Cpy!5ao&?JmGZPYL3weEQ}2{lG|G2())flvt_+Sz~<%2UxdQSDRX z^!bsU8XZ}5R-3(!C*#7C^?V(=kNN{g+yM}1$Z^vo1wNm}FEn61;FHbp44yKKH~RZl zdcUXEiv$jGrlM(z7uzQT9l6by&LK&kojPTCwQrwr5jQAyv#&k{54?Pb5OU*-+_j{ZcoUF!d@6j0{fW)`)LX?0o(!% zg8eU|uuoI4e^5c|3JP!Ow1@L=YLMXYaIVUEF`_qez+2h!xI{b3t$Vv-z2IV8rHxKC z4L5&thrMk8P?Bc=P|LmuP)jN9^MLx6Odkb+F9*~uFwh18N;M=3BCk_mB31i8nA`K# z1(6AAM_|+oga&3&7CQ_*i}b9~eTibScZ{CCQM_UZl6gf}S= z7dlKdhMPkBc}{EbC30}}#T9>1hi}0KgzO}filZC2f;L}`g7uhcy<%qSY}QqzbaNIe zMgl4Ob0o}{TsL!~Zx)W2Z}k;4CCQw-2}p)7)n{+9h%ZO(;$s3OAuXdT!pTEO%q0Ek z`AKvo4&y1$TtmRB`4bsTb zcc{y&9v;yv#{Z93Zks3p7L{DNB{u@&f9}d{<3PPup)TO7uiPq3if*&gNXwPmtTVYM zAci3~j91V@;WZ)2C3msdcAVA(q3U&ukvGFK2680t4BuXEj-Af65q7tbT98|<%E;@? z!!%em$YKObyB3IVSBdi(nrm2Q{rWW^i{f=HmO#f!9^8aQe7&Ty{=g@Fbta6u^SNAq zPkDoXFEM&!Qnx(U@foJP@D?Fb51xz_n`KlAen8vhM)g3zcQ|_Mn_=Zr8UVGbRjKOs z_j#y@-2VQIXYBU(M?zWj?eDR(Z+|l!a{Ifo(pr0WY)p!d4P41(aQ&MhEI{QYpwhaL zvSc^jW&SmsZoLDSA$lG97(v=%zA0^7ayWMJ%K_hJz8=nzi{B;KWjRo3J;OeX0a~py zSU*2g#-+BduEoa+o{Kzl7(|ykKeX0^LOJ zsAa$gkH;9&yTh-4DX@FZMOcJT;KrMZfP2r4OoC?RPtDlUOj)T`FEdh)T=?qf0Hj*) zs7L{Pcv?eb-NhQ>t1nd0#`BT7WsSm;i*~1g+IODy>w#?6eYH}P^)MB-&90j!9(m&N z$m7H#&$-jr`Wij)i}ACQ*FD~sk~~&?_k?Q@qzH$x8*roe*s(Ibi~KmUnd!H#^o}g? zULr3U%B#&h2&-7<@qG>7Jdj*lI0w0{^XG1QC-NdB>Ox*p=i&vXkS`fV=!4-n-KxCisqx%z%6VLF6|B2{%ur!vNc`HltV58Toi>r32wf_*BD$ds zOXI#Mu)!5W29CP$V?rZH0w=n{y5=tojEu8lOf$ZioxNE+z1)c1l=MP9jOXBqri|A) z>ecHo`@?y9rulmlVdP$~>XxFdOcmq|fmfc`WE@8vp8HZLLA+GKOJ@3F>pUgk3@2L` z4P+kh47Na|3Qxl$$7wGn_e8d9L@#9stsG{3_zJ|;79NW%8lqAuWmfn(PJbA+?6Ur_ zQyn#;?(aO4R&)p|ZMs3|DXyrN925CGCw>SNMvb#pQ4|#F=RBgf>6`9U6e?6054O*9 zMEt`d<|3EbI&tGn5iY}Kgm<^u`hu<5W}DECmHBq8%!3He>@3;=tIQTyva7qb&SR^R zi|C>~G%r|$9B9#g{thrNNEks!^Y*iWHh4XlVXLN*c@T-P`qt}o0bA0oa3cu%PpG}vEhEg~QUvgla2m~r=YR#-cTa<}nP z+rc{!@g`MTzf+Og%-2tDXF_~IYa6Sbh-wysuwS#*?T$?`>NBSZE+Zc4+aGzu`Z}Mu z#B7x#W05nE8(2p!tMvPthKg_?=3FuRRLUlT9fQxsoXu!Y;?9VS+9hg|YJuG2n1898 zX7f5`W5Sr!2Xo$N!t(m%VtL&@&-XFAkmN?g3hyH0Md6naw0ni=tF{@Ho`Uaix|%xL zdkXnPvwt3Rp^bFd&Cx0W;cE$or%!A37EZs&I~wcl*h!vj_FF$ARWwPC@}K3+Btfc5 z;C)M6M)*F!Kb3d5`sjNhT1!rUyWOVtmHje#7pn^-S!|VsNRHAX?4Tl9=~<~U4TsgB zZY!R>y_$%=QPxGOY9%G(SO-zO7FU@0Yn`m+zLW*l6y#Dx{xy7MdDD5uv#)p0nRcGT zJ9bvndE7;eYI$33yOKNd7o~7-^TS9uxuH93m$!KH!&&u*e6K7XVp}@q7&9{Q0E34t zSFg~A?;=siO(rsErJWE6Uj{eVia?xWH9%1G@T^!X2okRLk6^KYG^rHcH}gx!c(F2? zHgO{)S<{D?u^k!ljTu<6Azu~Zg2m#T20X^dq#TgSN+%A?uv0A7Ysb(M#Bnq5B&fVm zOytvf<_NQeX%r&^1Cyv9U#-VC30})GEd*Z^OP6)@qbRXVkS$IQR5DXFNbe$OoEn~< zTVrJN&Q>9tnPfXVyYMIn@ zKEPTyftZ}SsKj-6o08ZCM`D)wCa%$fn1V3T-Hj9nsoUWs+a{G zNGK?EtvbpN&0BR2m08QE1Hrqws%_ykruFdDJ1{lhpSu<>hcHcM&UPQyVi97)znPj`Xh^ zgK6I~>zE`_eoa9KOLj$b!%u)s5IvOvm!=igzo~NCVygw$2I9%|VcHb+WIC#mCJj{| z6eFVu+hflMt&NY34UQ}ej_+V|J`*!PQzj_J=Bj7q-%+DZ&nAp6IeO&Etj`A2$6bBT z*)o;Zdnlf_mThQU+9tG4wwhw^$*Bo~`D)T9ine8>|sGp<*O$E3;pdg+84 zBQ>H=UvZqcx1@IBu?JI)0iXqkCe{I>7AvWTH`gg3BgTe4jz@d^b!5%`EdEC9HPtj1i>!m%q=N2&l-HJz&x1^5_fXpz{c(I5kSe{9w=}{ILLo0=B7wAe-sEHl~w3rBgK~4HU~x?^5CHi8h6TVqhnGvW$`% zI5$b$UHmmuvy3N$T!rZ)H+^)20#zqz%}baga=%L86Pi^m#(4HhfbLXdmZ185{`&d* zK)N!teUneqK7X&n@Qco$HmI<0Ms!k4oT^%sohq%Ry3?`NQWhZ19;;it^7I@=_ZUsU z1kuV5KtRkX@WzOU>7h}n533Hk!H3rNILi5q4-EavX7Y)pB5cs;6kvc@ZdiGk57kF$ z?I99is7RnJy>Y+8M*)2`__Z{w093^kb4zJ&%aC-O87ZkiqOcoP^=JMtDkTGZ)(D-( zw8jiE$YI?bG=5W!6N*sE9*MiF~bH*>#wsjESh;4#q9u z{Fk_v_l+JbuH5=5-HlNGsPsAo%TWo9>bX$h%bc!i{et{hB8nzn!ShQJRWcK*+P8N> z1438R$H59t|CqkI(u<{4V>8uL)7K2bVDrC7Uta}cf)4+PzSgBkjZykqr%Cx>`YOgc zK2KknwWDV2N>fdpyp1O)_Eq$?StU{QwNJrS(^pDnidG3Bgla1zV2ZAOA`s;7i|OiB zhF`o7NZ+V7x@!DIOsaInF?chcRP9>b!PF6$jhZO1Buq6zcQwr={5Qf-tS_RhLkOw> zh@=pa?6GyaBLjq$Mh+pUQCM^muL!EZC`M2P2&^b${u~AJs}$O;X?&-X@INA~QF)Y8V#eJKrH?Gc-Aw0la2syhvWlbznF!r2qM)YGkvs|YP}vrY(fhR{ucQj~5g zk|~t3N=(aO@?`Mrn%u7_W4z&?f7alj z-B>x3L2`+aG@)$GyhG)x1wH=G6Y7?ou6wrf2+gV%utVBco{~074tc&ygcx00D#=rA zYF=eejshVe1_Y{klioA?-93sXl3PF`L>bd=*>CWU@f>$A;exWj^=Rli9DxF?&Hs?)qvyxK{yp{bT3{oJyX z(0_*L_9!B`K`hR6WI}yg|1XK|{a;RWXZ$ln_YW9yWe~rV=#EN_(zJ8|8z~bS75HRv zSpNR)F+n3QsOgZyb%Ekfysit{DDB{MHzkrmrJz^{o=B zldpGmgYcjQHYeTuWwXC@^Orq2m#>7`MpjME929m93fl6NaZqO)u3QZ3fCbq)%53<@_73O_k0{P3Xg=0V{)!v9R`fFHD@ z5)yf7`U+aPdOk4;%LwtDT&SOH(5u+5u6}(I_5kVZh66!IA>SvxwCz5XkObB2Jlsb} zIJSyROj>44+P?64+r)ecX&&Ua#F$#V?5J7Vz3n~bfEaI_>b8$z%LGdS3F>QHH${uH)ZyP4CWL%hfK zNX*YoJH*m(`A-i%U(MH_`e$sz`aHJotjew2`zw>gc1v!&7w^Wx&yg*6kc#J`yTV7# z${ovmf%Ozhs5*08-Q!}`_%o|qm!V4-oH@!l`3*x9{+ALjEywygr76T%U z(dG90XB_`C4ff4HvPC-5r1W)%H+#>Y9`P=o-tV1BIFB2JK*K8O{U6ATSda9zM$FY@ z#LU$gJ5>TXe}{tn9Q=)f2X=EBzDcQ`TUO;|x`3vuMgWU4Ys7yE zuv4yIdZef+G?M&h#xgn_7t%0>BQ^lhK|J)d+>yYxVgQ7=?>JdrV-}65F%KvJ0SMoP zHJe_EVE1T=c#_6%knz}od^V>uW#wobiQ%hHg(Qm12iNZY-s)wN~}G|tuOe|x5^ z7iD}5CS(HNmbVk^ERSx=sBVKIrZ(y_*usoj=OlA8+2oM-Hi;}Yl_budlB)}s{R9?2 zC0=crIPPS#TFM4}R8-G9**yF!safYWlH`)14Vh%fP=b+`o^WkQPO^+1cXg_a7+{v&)9OT^i! z4`-@XaY~(jra!5lW1H1eY&nXF-|hDoD(lairw_LNoUQge!c~zm%;ubGE3574BR$lnM!kr zlhbWeR;uuR!mE3}K?bp8NyZ9xXX%ouWmD@0h}EV~*P|7?_B@J`6FVPh!r(2R|M;k% z9$cS=>?B5ZYc{=Xv)8~Xz_6Ivv9~ve=G``>OjY&PM+q=Bz;|J)X(P!WZ)%N3G0#?? zyn>P)vXJxiV%diE`S9*H*)w0Y0cQ{2 zwQ==jvYC+WeBJ7EXktR3v1GP{EG+UBP^%ix{*xW z=|!Tu2e9p*;Wb3@j44utg?ziLP=n-jsr`UarD%#u?7QVh6#Ujx1qmS6ElV znT%EkT#C&J*Ki!XP+Nk^S3%W>Q`l96uvj~X1<40_;sF@^);12XD zn+_aV@}%z8IMuCex)qK;azMJ)D3?5XSLOCyLf0-nLKV@jQNic>*2M*1v~Sx84)kp? zI}tV z>vH|+R7jjybhn`k^`CT_oVb1wZm`E`86-hb-u;}7oA?PW-i0%4Y;e3+&cK+V&3isd z!r$Y{=HQ9L=V9C~VJha|FnxFDHQ|zhSb3kMKZP$J9DB!w(bx-z#A4A(Oy-57&x4&u z&hx+5AKE7^Yl}DPck3qNmHkCRSPkfIb9v*6DdR}o!4qP-4?977v{5^; zumtWioC!7*L+~YfWYNikCuY2g^lEKVUgXW)#o@{~59aOD=I-KVn?;{KXGfOS--&}b zR}EVjvKO?}em!TGS#+|y=p>T`jk~zwy;gToZybgty-rz0>Im(u}k#dI`#uD+`5+{?4W$o$>?hI@l9lN ze55Eva0lWVofgaq94E(FTfsaK2e&U1|EE8KppjferfIuo)>uj=r`xUX8g+R@9Be*P zly{ z#^_5(j9Bf)ci2^kJOb6q7Xe&zpu|ZSb6s4scel8_$-KSDFg6dXa&))EEV~+K90V1jRbKTjOwAS?a{MysNv_Ar35^ z{*n}DcdIiNc67J8V&VAi)=YeL7deMj(X`k!o1&-LMS7a0FwMR$#W?Ml8lk;${?2~? zu6Q&}k@!)spCz}2yCg~GX(@#fW3pyqvYS~d*)LMx-a#}^r~jLOmm{1UeF^(_Iq}OP zm$YduM-%d#n~rv3Tl+E@k6~4r{@P5NcajvaER#FmjDs?f4pt!x!q;W|noGv`mo4Ly z^Pv4;RatnNlshX^5zP3&tW2>8DS3_zSa8WLML9tw9dpdNK1a&ONH2#^EwOty<^vZ7)~-%UYbJ$q7(?OEP@zgsg|4f z0H)(&R&7O^5VB(yWk%R5(P$+svA+qhh>EihC+BX~TsP)zmxeEh z<+46uA&|*BgKtNQGD$1qgLH;t8}tU#sg+_yaM+Q9D0lyk##hvsz7fXO_Fu_<%eNjX zj2*9@7NnzaoC@{xym|0Np`x0q%63Bp7P^Z@xP0U6h=-&!?7KbDzdMeQecybE_gYe9 zV3G5=41 zz_ICXT!q^(-wp5@u6_4)pXu$#z=P8n+@a7l{$oGjx9XG&79oPj<-c=JQ?FT$j z0Bl0ykewZP>PrHC>1>;HRKz!ocQF);kWuM~?Y#h^X2Q!g1uy#?9(y>XFV=V)*;AAv z*YdyxB?ylr{HmC%!=hTDNH1LWE!idlahY6NTQZ{WDgu65718L=MIH?II^m z0L5USHW%nk-ZXVAw9RYBK@fly!)yMDDu@3D=Zlg)Dvb~x?(ck&6;Gn9R4P*aKT>=8 z(A&m~7+=#vyx^w?YsEh+mG|t$&2G(v($h7`*cfaHuu$BRGB_!)gTTglhaZ|;MiUP$ zasE{3tqT^1jsb!{%$`lc!TmC49kasCeWx@J^%2$wVh$+ z{E%(Q0i6TGpE)qR^e2f)tz)Omb8M%6PTZ`YQyx~&g`6QOI3-nel3|B!ViyQP>W7zI zta5Ezd5Jt1c5#-1`{?EhyIL*|I2(zxl5BESwR4}!#@!!^+tl|N*^PUCOJg>L1dHrk z*Ie=;jp}YWI^5p!oQp52tzxNvjvVDub9GIee6rIERajl5b5|^@jXR7= zI5f^5!Tn>+Beo2A0_HC+|Yh|ybtS!{vx0*K1P`+*u5Ej001qW`3=cE;ww(^(GETa6CS?u z3;3f>T?S0aui%e9sSE$V@pRWF&#B|GvQ6Gwy!6VGkfAno$H|g?Qyg)WRk>vbT15HVH3Jj z>-cQn+j2DCF8p&=(ro6i2-uC9@*tmD-gX2N3RSNbPJUPwm3=^1<<%q;zTi@pCGk_2 zy-rx(+%n1)aJ96rB$pQuFnifD0-co3-J(z)#fzNEc$Q@96jjU0c!SlMU9wWgs3f#8 zyQDChM83jM7f{Pbccbtpcp~E_0)}#_??P5VJpH7LJcB%C=|fzlnlx2=_zqKX)Ha6V z{cdE~lrW)Oyb8}>#Qu;Oson`T5d_|YSKo~@2as$-ijPZl<%Mu;vjXy{vDfPkFDP~xrUuw{Sr>K7R{krkkB|LC*KGJmg2s``gn@6KK=}P0AFf-{61C4msuZAfk9G_=`^?n1Ez=V&WJed zkdsYXUQvo-edftQ==7OwdW~$sl_;CpQZu2orl8d~nda)F!jzNJ&k}E1i8rMr%KL=x z8Sinhci$#QJ6NAv@!Do%RBPDPU#Pb19A!A>;k?7>Ie5Wt|Iy9CO)~0B+9-H{<8()g z{@59u#nC{*n~pl!(*!q(S)S;L1B=F@H)q{PL;fg(WOkKDrg&wT?j)+ltQ`TIqZZ;{ z6AST}hu6-Kv1@fixqCP=j%wxLb0cS5HEP$0&G>O*Gk!LoD(LIEeD{p8TK=xaE%a}+ zmM`ENU*PO;k|RF{P)sZzS>m+hv@G9uWQ09ZxH2h!pamvnG=nM*Zs3WK8kO)INktJI zcL`I?<#r!&9liyITJMoXgCH`>*t%28I}TIFEC#O}L1|SPV;{8{ul8J7XPC*1GKUZ# zg)Lp|JmZ_Czf5clFW>~r85lNeSDDog+efS(X9`1e*o!iB=n-5IEVA_X*1O@ax#1W3 z5;%MN5?+)6gTpDW@xH`OugRd8KF-m{%|(aJ(#{+d6{ZEQc8IcRV3iB*abR|u*}B;B z{+r^n>I1#Ye>_d;o#0A1;!PE;v6&Qv>*zp|I8tsngyPmOemtP~)HoM-VUt+o=U~PK zOOIRCPg7peak1a`N7|JCq!1ThwldB44ngBt$Ay$uH~m6%v4!b57h7}%KG&-Elxqf5 zJ!1+(l?`WAsR!ElQO0~c8BjeIoh^`FPVqdM3b|AxA$woqT^-8vn^%T z;c-Bwp&2NM z^yG5uU0@{YBo|jy(74aF4IRS=2C|0j!IhCZ{I&~=?4+poV$n_OC@MT~0Y#TUSe!L; z`-W-aabX8wO|#}xh}IgpyojU1f5#d^PdHD<2Xf&#q77r6!hMO^mcJxpEouiD!pPXT zW|%cnCusVnuwT7))3bfZYl`K-No+fp%xa7}SP2V&SWu;k9tjfrt_g)2IgB|4184d{OypEP6+ zC_tdTfL9`4mjEPCdz^kI0}C;Wn+e7%dv8-|Q&i3Fd>SR#=HQ#cw*A(L-_xB^(0o7o zdG%t`B06eUh+)JBdEX8d!an}F#ytypYf5T#TvlTyfpgG5qkau>a|=FR^=9a1LjHB$ zINx^}lZ#MLeZ3|>*Nf+Y_^W-N!01Wg72910a^Bi|>MH6BF6z3jqI^L)8jpfT%H&~} zIII6V_4#YNY(|5iyIEw_tkX0v7ml z1*cYR3VW@KIZ&UgV|^c>4w@@Xhw=>p1UZJ& zYhMC((a(BuXei88Y(^AJ_p=T zp=iTeK2I$${CG-{-AgGZIuS-vj!cASK&H9^oA(`75#V;%gj9Op5Sq8Q!c$#tJ+=$0 zAD5P+U% zZfoHV{bn>+6bwWm$VwgTGhS(+*`+SRDy>xvslPb^Y!(LZVXlKs?r2lA;<8gZFIY|1 z(-qbW3kDkTD~YtGklEhuW+!F!-Ot!8g6-Fvv)BFx>#YqxY1``Akpvc!X#qy5E-1UlP=5>So6&2QXGqp;aD9wvrcDi*HMMuYaA|L5)xI|Rf zyzinEc$zIhieMwvT1&Vv!o-;vtT?F56`(K;fBPHkA7}t0zrRUZdD_~;6Iy&mUC)0! zMcA>aHhzP;3uK5UuvN-%{0vE{E1C6r*i|@X2N(W0?Ssu@iXcO!^Jc^l_xO>WLRX zpS12EB=ctcRNPBAt&5pGXe@KZ$HIita>IXvG}a;#`B77Ecz`$X)tdif+1&`92}NI# zd(Tx|N7HZXpGLP25*0f)t!&I<9Y zCFo)~UQ*Zsa>xQOo@o538ujV&-8u!7puP4kcT?i@YTpD;b)~hG3~oG>c&}LTSp8Kt zReq)Q9Wa@ugvU+4)O*3;53}?KY5ggMjlZN!nS(#)Y46%5&6DB`|9|tjysXkP&g78G zU#kf8)x+!0ls2<3MLwgVHeShzP-=a-c=(C zNVnFxDye|9YvQiO`da(a#jP$a2a;0Gy+=i*zt?TtpAT9Z^?iUIE1|~y{eao1-@z|a z+oRwfeuPK!#Dz^dV-9cUg(|GG&xuBTP2gs~55vGaA3ko4@y5Nre zKG@|ARZx+nD{-+QY0Q^&eaXiCxhh=-4jN@(pKcUur=jmsn@XD@X;W3&MB{$P=hAl2 zme3SQTbkyX_#HE@CcwQ;hCePo;tWlUmZ96Vo7z>onhAJPe&X)w!%a)Pxpz<-uM7P_ zz+jac^{-R1ClQ|`M*SlM@wHaQ-{cabVKey$-*J251r1*i@F=^H4Zoq0A&+zxLkF|P zlw{*Iyos6RIXs!6V}m6T96$|xo*ex=Q8Mi^XOEFQ8NAH6h4vWtiNxPah@un{gkJaZ~eh8BT}2HFOC%{k^vs z_cxGClNi7C9n9!U$ySn4k9RO}(05Yc=Uq|a%~R-7My#q(_40DwqP-OBQ`A$ax5^%S zXt$6=`n4_v^Y_E}N@$#MzwBBE_iSZ~RSJM0^@=3Rtt{i?EmV5noY`0ll>W7KzMzq| z?fD92_`gM~TM3v+o=i7cux#c}ijRz_s0?KZRBAI=6qlq`LZ4k~={TsWQN}@=$d`X5 z`)bK=Ka<}cAHf7%=$LN0!u53v!1ImQ(5(R+KTfVtZ4?kC;P|@9{=USG*@M$OEot1* zG==O>)(!LbCL8yq&@^iX)FiqbjhiawR+O;-Wty;yd+^fUj5nZ}mZ$Q7GeLnBv4^2)5`SY=ZH^Nx}=MY*=|_gthmslvH%Q_A0f+Xnje&ws6!V}+lWBP zn8=gY08Q~uIVSRZ!9wd0t%Ra;fg1YXB-dMxx4x;j9C#0wICz!^@Z z2uhq&;&qzORSR%On9o(oGtqob&XoS=**!A#!tvW~=31}k&;^HT?6cF5t2*BE zdW|{TRZ~d#1fdk3AMteZJjv6+vzrd)?YoXpuN=jTK82rgU1iyIk>4+q8s0$|$ok|K z(5(Y+n&IKr4|TW*77-Q+jQSL+4yKF>&O(A~O-V3q5m=8bs||v^Hp6VSCLm{Ip2WQ! zTez&9iF@|Y{L$B0M-X+F{g}lmz?-r&@yHP>Yj*}<98pbzSQj9NQNMvkTvt&^qZ$rj z4hn->k*B0#xW2ufmV^_m&YPuiwB~BFpA+N+bBUM_@E6$Dq??G(s{?l+PDUd6>V|zA z--9<*f%$W`BOqq*h8@Bz6QM@y7&bBFEC=E89-{%ZIQBEPzKs2=w)n!D{A6zrZ-FHi z%@AAC_n$*EjQV#4Yx}-t)ZeQDv%Kf{`%1j?{e7k0i~N1xH5z_TvK0=_F_qS%q>ny$AACzdBk+DL%atTWk4>9PD^@W)W9b*wvG!d={2?< zlF5?dz13XOTjM`9QNkQ3du6ndMngob=rpc4N&=(2Pkc@}@xeUdAatzEx_}D?-P=qGl1v;_X<4)HE+7Cq&-|D=a(t)5cx^ttM$09c z?d;rfpA#5Kx>5OyPOnY14%W#Udl3x}yR1FxWi&6=Z*x&_f{P`5%;Umt5i2+`PM>B; zj`Ca1O;d$r`0z+@^Ln=aIH0tSVJ6Ik6NX8?^^~`CrwFCNQ*HBam}B!9o-C`P82Qw> zbA%~y*ql3OX-r#Y&mQ=`c_Q|mmfT#MRxGE0o&!C9^&ha|7F;yb<|G=@r6+KoQ;J}eVf$Gq)H@x?tI#wLwSn;7cR*X@{ij09{MOyS&;ffwB=KGR* z{@QcE|5w}MrPp)9klW5Zd#K7u!LH<8{HFMZ@$Tq3%IET)qv}-S$`Dap| z#yN~mDMg=^vTMUYnfnG(O7%J=Vp?^-<(j4`1H_u~7zXQXzDb@$YhNLUU*QwPB9pE8 zY!D*}k|bsf2H{({VB~Bg$MD(Y5YM8^q`YZT9-c+{a8Il#*=Af!mM?TJdaV@0Fcc={WX4aqC!zk9k^H9V5B^>+Z_y9R@!o?^yuT3jR-Aw;|DFvwy@x*C^ zq#OGL%{A)xP^+;uq2k?QTN%%V(dTv1XPG=}^2e6&^o=FKH3Z5f<+afiSIcujWtf>b zB(A)?3|6pJ!+Oh?cvAUyOp=VYXvTi(m-o~1q>7U~edCDeC7`PKG@7DYo{=?amEqBR z9vn+0#Fv*_qC`UZ;cHQ5z)yJIAy4vE*Zcc!s!idh&olh`ynUX8*^A{Sj{lWUx9iFl zAaELIy+lrBNr|&+S$QSJiBr$tsq(7K^ap^LIpbmZy>?8&Uuw@UczONB1+Q(qsNm%d z7Zto#dm%I_daYIP+WL z1|}Cwc~FMpXU*Z(f+>$l@S*1L)^hpJS~Q8Bs3&*D!-6d-qoLGd#1u=EfhE|Z>!R_31KtBQG2#T+P8=~av@Y}VXs zq1r{e0tzSm3QHn>g*1^BRjlRK4eW~CVsL8*yU2A@$9cVvJj|fTnHFwMV=Z4P&6^#I zh5FsPmdHx$f3NG06qZ>&nF8)dbtWI)H1~spw$NC9D=V%4Ad$NeOZoEMikl&nT$rl$ z!UNSnDa?sPVx^~+SuQC(Fjms9c#@w;dKqHFEQGNAjQTdR1*Qms-;^p}r*n%~Wt)iA}gMz zEDngf^gRdpT>jb)*g_jxnjslSs*IdeI?Y0dG>N+7l%K<(jP1N9pg)JqB=scWs`;u2CH&wOA-Or~&HuLD-rCX|zo}*7 zL-^&|X>Z4gS80auZf@P^+RzKWG+mWiQG_5F(CK-wYvzj|$lejFt5yO7$KU>LnYpfTEJt4|^4@@y+=G zw-gj$rb+@(L_KdGv8lyYeQ`o0RY+=h*3w_h7B zXLlkT*chL6=!R&Q@L7MPwk~IwtRobY)tSTY>H>CKJX3hO=HDLM7+oLbyt2GU(idu* zrG&)atjNTJ+^DvdtbvqqpCJbQ)J{e?i*j)iNw4~nEa{)mg4hOm-v|MKy&MEI7jp5Nntjl?~k+#tWLfI#2G=oP})2 zKJo9yeqc^ZNkMyXW!!o$w#?Eju1t3OQmNduLCG+sUz8;#9Bw+_ShEqtIt=sW z!kq|BvQs~&%$l&8o*xiL-otj<)v>(|v69|!0zY{iJiPyjoRlyzG0K=VAVyIyXGPQ3 zs~RaBKe?pJUP>JIeHn7Zd5r`;MJ{0+cqf^qnR00TBXI?N#{Ez6#9H&1idfDQOJpCC z5&BO)G(MB8|@v@J}FUs^e%F4otidQ;@gsQvhHomvQDnk~>U2e^jhq*I& z!|?$&L+x$UsIQC_I&mQEvL5D&HR8)g0!K<2V*gSgRc5^mZAJ&}i%`q|Ir#>7O69^z zDoRmYq@w*G75C?Gl?25u!npjm*++}I<6-crHWT=rSGmCYrjG0lkK~@G_HUqyix@x7 zGtqv@lD*LZ4sUAx6ublLfDxB5M#GO~lzLE=i?0gF>f(>YZEy;pOq7|>DVQ~uuN$r# zG!5p)rol9}1)rM+*RMZ&8jM-_Mbn^D4&yD1rJ4gUqOv@3*pZIP64eXSS&)b0|6$Lt z=m%*N3P?u1!0}6`=CRMU>6JU9$b8`$WNsAHQph}P0GZhp$kc?P#c*A@KsXPFhfGc3 z3>-r)sWXP^`>m_1)a+tcR$<*EZXU|5Iv!@x0eA3*T?3f>)Xf8!EbHVhxS-6Aj(dHeDgxiHi{h#s$;P~-~7o2@j6&VL$+J(bw_S6XLQ25C6d?(4WKj$N3 zD;yWc=-_oC^dt6=Pp};A_H~Fr)B#2dTd&(WRLdcf;WvDz4rhm^FzU;R zUf~taYa~6B8s+>XqKaja*H_d@^rKRV{w8soRot>6agV9E<%8oMy_q*)p0=X-Oxfgr zbOrJ76(i}(Rb^_rk}e5aT1~U_eaCSb;iw8dPp9O?6;Zv^j?f^y?>?eR7+Xj7f}P&^ z`7@*@Z!u*@rbrOL0H@O9u#Oq}ODQ`ivX|fUgQbTTc$CKqj_Q#`7IBH9kj)}c>q5}e zmT3#;J|aD%+$_o?!PrQLu;9Guk4W!`M_xqY6Ds5b`J%0pxD;p+T-0NHl5N_6P}SlU znyzS0m;SA+FugG0Waqd(D+TAod+qH7{cD^By~bTD*!~*ZCt9N6w>s(L$3o{4v8k^P zfEahZ%FviQ^V)xfPMHmb%Iy=&+DT{J^;hAT=v>$$`z6VGc9IYD*d}2tg-iH92pik~ z7QZ@`JDrdtxW*Ak3T|`|WnVoKYgZ%uA!l%n6H8h~aHF%Qox_dI`+Z~kdv?SB2f|q8 zxb|dl4T^5JEp_=rNbcV@H{oH?BW#*eAA5M z@_JilaAR)m*5H~Hr5?(+YkPxhda*r0@T<$Fd4tN9w?Spg>pXMHik7R9;uba)CTA3~6Se1gBvVB)1t6|T3WXlTflJ7jPk>_FJ zoCGHEH(@Up>yw)1#x)klHO-Bu+{WT~f4iMyyp+CYpmbn8kXEINgVw_WmsGzPNVQp~ z@wdlG3wOrJ*T#Xbl<_<2e2BjX_`8?Cdj8h)=jHEa{>(Y|9Sh94{}@k;r^WL)&*MD1 zcy{qT$@3)7UY@-?PxCw-bOgaTo4w248wH0MkxqB;cbLB}Rd!a&((BtcOOH9<4TN>? zqcOB`2t)u!EN#$x+3Dc={L|4v$T~9!sh@q!m2%dWg&}$n={X3b9Rw*WHYR;~OgMsO zHB?z@OeFE<6@>5Q?`i&;=GrIhZ7jC?PblbpprH4MK}Ad3qG`^I)rolj2Wk`=2c_IR zC>=eZhtV-RVfTL^BWv7+a9?Cl4$5}YAQyib{AKc&&0j8mQ~CQgf53VSupYxRgC{Ki z`_g#A+`rGolWT(eQh28Dbn2-4DA=429o z^JQ{Y61DMqqHg2z7;06<&zfcYY*@*M8;RR^8*v-g61Q>V%HXqg)DnDAey{dD$}%$F z+}rZLsHvYDl)EV#S*dSh5_6wRaA};`MvC*B&c!KYqVdAM;;`M5`<+Ht;k<_38$;CA&iQorx>0pAr>sv6+uLoq zC3REty*>T5x^G?PONh*Ngh#~j?BVt`J?b+quY6m5)_H7EhtJMZE|l-;7(_h38?nHLxe~~aKyz{_b`?{ zD!HNUD6{0I-|ZHw6fj#x!mADXI?U6LKVI{=HR)CW?K>Q}%4xh%^mf3GL4eM{RSvn% z;2c(AoyZb^WznzE()_mT1FKU|*ja4tTB?UT<_qIIIKhLrXWXAI%8TUv9ik;SsOiN; z|N0ebHoLpCuTZ-gbX}q^UF-S^SEtwYWv)&%e$(;#`)$I`tUEgHbqj7{-oB_T5F;kW z;CmtyT7=5{s`GT$X|J$JQibhh<*bz%$LZ|N@5ZQ3%SfR*;qzf`L6E$|tR&K4IVHjB zUn7b0nxO-pcxyW+UamH8UePR_2eQnfApDZ(&-kcbcn9mLc>T01EcYM7(WGLC%wm6$^Rx{@hqGW zD0)Dg3y*6#niwd3aIlLq+>K2@kn?F$mRsM?pxpuAX_MqB%K3{~^kDa#I7gf>%X*i2 zs=dDvB@`zj!7fEmCtb!>&uH^R3E@;&zv!GA-YVWms&w-RZqt9tb z_)?-f{Cgo@Nzd6bnVAXC*;pHD0`n6C;{w++FDtQ##H%BK%+1>Dz}iWHwV8pnS%I|~ zfwf~KqcgBJE%ZZX4diLFXv9g|9lGtM4@RzlL%f2MmblBAP_#KvoEDlxVpKhf9f9KE zQbDljVK^wC&$L{zx{UfVQX5-~IXgWZ=S@n@;z$urNPU(^O`sV4$EP`$_-nV>q#@F> z@OOemjm+hzIpQsgmazv?ru}j?5_>#)?tGS54`nmg`FT@*4$IG9<>w9g*znu$}E5FuWdMrseO_ zolnH-%vN5ts-ab64>RkAqWG2aw1SU#=C@3Qz3I-7kDq%nq!URMEgE0va10p1b5duwS^HB)| zCn&>q)b`j`Pi?DJ`_NN+%Bc!gttLQ|fRrz(qEefx)g6b{s2GAG^ZxF&_fCS^^Yr=S zx!!ZV*UQDuUUTpD{c*2*-Ru51ZhUwbh*3MP-?J4Ucykaem1zDeRpIXQR=cXg&0dVW z#4dcnLEVB0Rk#pOf6IC&*Utd*9}|-+t&?jQ)=^bi-gprpbbQYU|7Mf zKNo#7-yu8zwjk&@7}+JQ6K0WThs4y5S)#9MH)9#$6)T94Tr-jFB5m zZnu$p|FY0?Um~+^-I;AooX-j8II>NC-bYgeHxpXc7!RcII?nKy8n zU>bYwYm%@lDi?A3l&D;zWw&yny@U?JeVyFb`C96))pwnBC5PXpM^4rsk=@z8YUDB< zd8}F~<5Z$&(^3hXxxn+lZ3LVrxp>u%^vxo;WTAOq87GqgeKXPek->UbAqkAaKg&F2 zw|(UpeJpggh+0Se`vS%rHpHtSZ2{wB8!|v3>RMA>jLO|6no#O)GhY48tIcrJ$Ig~V zpc6i&RbitL86c)Y0=YS%4vqvJ13!Jl*&WOp_$j#kqqmvZoor$L%txG^Y&cT}?wwm| zyw(HG>I;5Jz(vDI;BV8v;7g?9xZXTaLgdWNt?q9c_{B_kLW;)ZHX#dPVr=mpixeFV zolh9vt=y=G28*je;~uk#NUHkxk;lL<5czro!LHBneQfSRtf#btJv^S(rDqcqZKF8)iO?Zp+IlN~CQWhYygJ$7 zA?y^Pzwx*zB=jyY7HjmmPH+{X0WFcA948d078TC;pp`UV@7@>}b)NP9=!^s;Y9gBq zVi@DSG6#?m5gE8O<#h3vrVw20g4Ruv2T8au3te6se@+tEnE@JB6<(h>Sp#1shr1Wm zy2IVyU5%ADZ**Q0UN6Uymzh)e_mU4(c&YK(%{0rT6V;jE34Zt7Y-Je_48cuvQ>d)^ zC=AW&-9Ig)?wQmbY%hFMy$$@LJiZkpJsz$+;Sgajb*S;8DW|4gw;W}lOk4d1WJND! zA3qd>MLjId@5Nkb-x~yVog0!40uKf=tJ{TKn(!&}>EsW{l;k7vYETm1)Rgf~}! zoABT_yin~jo=r|pFv!I#J8zrXu$R{C$irM(Qm(%}WYr*?IGkCOB+VqURM<{?8ziX6?ph?I24Jg<;h zUMxvo4RV(9XO#vs79cQ&40(1-=xUy^JYJrGVIxn|HnP7u8k85(6S4DK{U?O`8aoF~ z$rwURC!|p^XQbkU-u$v$5RaxXu>XcJp^KPd7ff{!A#|>3lX2Em2Z=Gu_-U+=<6~mb zxga)msw0UQIa3`742H%`mE@t(#t4v5PXn*#huOe!$%yXmApxd*cOmQaLIcH%WWK?? zq@_~W;KwJuIm2=C;7I?l#wLB&nq(+dB6Kz9g$4<0JOMAG4u>mTbizd<`hawhrJ;V< zZ6-ZY0eAr@%ciI_cXl;OvM7~4`CtKVRQ&~$m;kG z!v_Ws3Lh*v?SGS4?<_SQkmOtwnC^H%ZOnp>y9Zm-|Tk%lPZjA*s_x~Giv5$|5xT6cjIACgNv*!dc+D36Pm)@zKFoYwioD219~o&Xt2IU{#U{=j}s+o+(_ zY8bU*KFOLM!cFdI6>@LfsDP&%RKjvU@@8PDJG$6Pv|_eEz|nsq|NO(z!pnNm^5ef&%~sTHno*o zI*N|<4?6Lwd0Z+FC8jrLw{TKC?rf1}pIp-0?EA<?IbmJ|Y;(pV`p`hchNOIEG$o*EJ;wJtE zl-6Qj;=y=Tm7S#sG7zY;mZHCb$@Z%7252K^w2Tv1YWUQxp``Gs%e9oM@ON1-1GZ*p zsV_SkIAsb&^KxDu+`RUblLtow0!cA#g5%55I*rn0tPDkUhGta1FSoqL_vQ%^$R3tc z-KhW#Hef{cX$4qe1J0>-&yyqzEP#e`e;z)d4QIH0Qd*~kk}<8jFs;R89gB9xsWaG! zSvg0Lr7(D6li)><^7R+aPTerQS-3gfNicN!a&yA%SXnZBJ&BA9;zC|q>}W9~gJB2^ zT)eG}y9oc2>N~?;i1pY8`~%(C~o7xEXB4menC#Atq_y(uJFy zW_8{)cfr`9C_dcr(p*}G37%+C=G;C;P#{J_8M#`z_U~qzr*QWExGfCuU9GWib@Ht0 z43UN@^BP~^$fIKg?UiW-nQE#uDjswTSRuSSv!YkkMDa~U9bF_Ph3Tmj&U^Gm+@cqv zn4TFH(=$UcJw-h|1FkApOQYQ1qMn{f?CGej*y`!&=#dE1(`_<66{}M*J>4)p(=C;f zUXJGiQ+Zv9_I887-DPr!F>+8jwG=`I(MS$)u~4y|RZ4%{8k|~eTv)=K?u=+BL@EpVLmpDvH6NJw1NSlgqdd2J{nBjx3%(GyB3S2lz~UTp=GNL;2Xm zN3%-IiD!_e+!ZJ`UXY5YgVLI^Fmlm1#V77CM$V*xhzONygCoOsaN-MJ;^?@S38_!t z;flC+6o#klaA=>ZqvdsVH9=Qt{3@w}@gU_%bY&l+TvaP}ljP*tbdcnmy4EJ1d?#9% z=xg!ZhXzc1D$jMEFq>375Oi6Y+(0HO2Tvm>PL5m_X#&QSbBU@N65@u%c+UP%YK-V_<)ssdG-zkh62|QCPZhL>}`BMFpGL>Qi^l=$`h}-B8)}5lK zkl5JbN|`b>fVt4le~0;7YaQk0Mw{QHNm zw^xYqU8t6qnKT<>)#`;2oO6vez29%)d#p&W@6>WMcj% zc3FJ2JAk@Qbpi(!F6N3EgQL=!Uh2@_7~7Mbm>N5C7V4F5v{m%f^&~RNeS0v29_(Oh z_VnNsdQi3!v5M5M*yze(8`Qf&^*_6nRv%?fyd4_kJ5zT~_Q|MgcqylRWAAje3Fy1j z-6&dPV}8TZyZK!;6ZKX$GX12`FCQd3Cn~Zf=k44DZtaR-s&Ow1O4N7c_n_kH#}EaI z79v#p3I{#rO1#iE=WSdnt6lO6e>lAPLwjEgH>EpUIkYTeY`F+FMp3A6gvk1bX=J@C zm{skN+O+(YI?Bz)`{BxCj$oSc11VXCFs-FS+_wPw0#$pZ*QP{j!(ZlkqDuRA%1?w? z5lQHmsYIl{U3V!+k_f6H<`jMuaMuj(QRas?z<8QS>HjHtafw3C4h0+>bePSXLG#*^ z%;wE-3unc+T-xS8Ms3cI^4V;TWEi^0{JoOqgKHdIz=_-Xegs{Wh!)-oQle&2zq0Z)bk~w$u^xw-cbWI;X zzf28J84wR$6P~*57$RdxnC{7L4rPcgwl;8X@I=f#H~4wXIXBo9AGA(*PA-);oSqCi zj14wU`enuo;blBeP&TQyc#q`(MV$>YerkR(_Vz@MAv(L}Vv9<<@S9Xxm$aPAe3u(K zZK0K1?@4Yf|D3W!&jDNp$|x=)yIZa7ZsvOPRP&cT)7u=9DJgMnveimizS@Q14o4{C z`jGp2ZD6T1v{V{e+S^c-E;13}&;>MfKyTp(WR;o?op0Bm4|^cooNkSuvzZwvGfu_o zdutElMKF?rbLT>ZgeNPy3t#aC)j`wT#+dMa;}RaVGwPV9bsm5zn(dZMI(0sW%q{Ime#vnx1i<>z2?&iXH6P zo&u3n(b+XU>pYjXd`pUF`_b6-TbzwenoHjs%V_>9N+3;G^Dn@@DQP|1htH6#b_JP4 zDw|-XWo}8C%}(@r^Lu8Bqcc#{uJXO9ohx@T1N92T4LqCV9x|3Z^l>LR0vdtFf>KP)Ka9-()Nj_1y$tLQ-0Rm*Ra zewMDPmajO75z|YQrmStAZ%iua>8+b7J4`@6a2^=$XkTki8*p{4Zj zg`VxsNQnvXY&wUiTbxz;r}R%pyNrgvdy9j9OCnC}rsfaKZs;T@++0+NHHgzyzVD>h zrt^2e>f|ctcQRs^6F#?gEPpdsj|4I}b^%T-PLn{2%?Nos+Z)~IQIaH<58Isu1S9?o zN;PYCJvF1qjM(j4oO2j}6U~*URnh(Yrwbs6xO*ddl`mhFoUcmG&xlO|a&GOF{LNfF zuF9D|ICe27lnl>~gUpeVjIQaa=Vja?;^vfOR{`H_l2eNesfcsaq3B2a5&Fm;(gh7D zpqXRZsf1W;A#{`K@HBoT9aWg_OUnEM1ThMp(Wycf6=wL7vMk75CM46BG|Yl5H6dBP zB(DW2GaaW-E2Xg6NpXsh!_j>BGTJM-7u0OMV-0&)AWQk zBv;=iZ;yToi>oiYN!%fZhJv`UF|HF%G+PfglUZX*krg}CfjYRr_v zV(&mghF_F?;aROsR51&P$;6jE#DmR1;*}ZatJnv?V~`2DN)QMR5+fv{v;3pk}pt-@!G66RL<`|B#-=@q!aIhVFX)ib{5FBEIZvq_N9k3F>j6D$uX4xOF>P_Pf z1Oqns!d~#GKya=NPU!`Y2?Wo_4zh5;j85*GdP^7^2%e`BKuNvM7pve)1HqeZ@GHIG z>_D)@25;>Jj|&8qDHb8ezIRskmj!}lc7oNt3C0J4H`(CZdcl_mf*b?rP4cXeLKjB{W>?&_0X35Amqxaxk;}v#0cMBQuiIlIG4Axi18O zOpvHq6}s%OJ5uPj!xJKf6YTKBNZ~~4rVQgHMm+kzHIN&Mg9u+H>&=~>=0v?<4Tyah zK{oV3@n@ts<0={uuUlhW#v3w?3<}mD)AFLEj;JWTLT4s&3UNB~CCL&Ly!81bg$G=| zq;w1No(XaLk}@pFArmsemo&+OykbHo`jVztkY5YrY&?MNJV65f&fKtC{6liX3K@7< z(tZmGOb6$SL6$&iMC&%La>^4 z2iRI-INZuWDTS*{g+0o9zf1^_i8ZUI0q&iU69eR9eHo*^$b=j%wXR{*0Y?4oQrp=*8>)ySc&yNhfb> z&Lj}*Ggt0fch*EcGyPr)d|akXB>8cfC(P%^Wp*%^AD217Jbqke0&}=yF{Dq= z`P z_c&v!dQUK}QtyezW$Hc57_Huuj0@CziZO_Hcz0fWRi2TmK+}yx^`2>bah25QM&ncU z2Fa-R&BnXxJB%RtM_fjUiDsL>{Rb+;|29zX8cyYR~rAJ-XY`X>b=JJclBOp ze4n>eK}lB|jRM`S*6vyrlxqy5Mg{Ma;9?bgP=fPRP-KaWA{BgCg40w`&XLA7D)^WL z$El$31&oVSaH|9dtKjnzOjE&^Bn- zp9(UKCB5AKN94Eel}R^Mf}!*z`2fz5?I^NCKJ*HiiH{8tZ>IwKPo)BoZ_4KKK5p!b zja^*HDJhYArP|RMu0Ze#Ec9>^=@t^~tLX@3&4DtT8@y7$0ux!yv`%4g1aoZIpMcSQ z0?r5oC)(hbt*DT1SJ|-tuwc+`SKF{30xJt%Q%riZj7=&(h`Hzt=(S7jcr{j9h_!4R zc8djrQX6N(3N0Aq)nzv9%HAy5CdS*~p;lBjh|6tQiUk`M2%?f;*66bWv(=DP({0#a zfT=AeIsDahCOD%Afsm5u={$mgoe7_25VN-Kp;5FhApyS zHwS_@*svlCHZKso(T3&rW_fEMSY(4oSW#~a1ZUf@0TygYAUMZ{eSyrQj6-!GILU^6 z0L*ImvOqA`1|P7Zt_)z5-0HU%Em$ZJ^x3fgWx>`2f>UhRzx8IhF2FsAm3*BU_1*?% zZXkHA4X(7ng1ZC3>um6B3oLj%5X_THDWNQ{>B;+ELF<8FzMbF#E5U;bJk16tTi}Nj zxWEQ~oUgj#UP1kV;20Zx6tLMFYEcG)W9=_5TL~Uh35MHXu|!Xo-YYvqAn3Kh4_aW^ zEdtff4A!UB3#)odcv^kzXMcp=7fbY#o9rys>J_SfngxDIfz8zmINJg@DX_VK0cZA> zARCaqhWYXZroJS5*^KOE4EQ|@EW47uk^%3tz=CN4tY~Qf;O78aU35f!F_$!7erzR> z&C6cZfEz9FF$FdkHsIwJxLtwGwGFtmH+$LR?B&guDOLj6@9Y&0c(esRsKDkT2Ry(6 z%RXqYbHFFB>#2RG`eH71zWkXl(m5v-*j(*^TP(0_m-d1O{2L2=T7k_q5BTBU>>VIs zd)f14y_GoyNw$7vdBsxd%cYs0MD|(vhUld0dS5578*cMgEcXS^=6-^zL=f9A`moD1K|5C zu#gruY5=TRU?DU5pa#Cow@ScZf_-1q#}1BIE-4k40AI6jGptMu+<1lai;@dcB44*} zbCeq`VGxV)^z}Dv{1Rgo8&+coW@wKgmJ6>}PsYbPw*Vc+hUskNYt-W`o-FGVVvSkgu& zM5JgRcc9gR$Ku;%BP%v4(e+R&aGu6L0!DLE$)u22rI?FNC2e0d@^U>Xa(YvU?Ouf9 z6V_iL7H!7q^+%&cA8$#3>3o4_``kDz>6p94HQ77OvpvCgaQz94uukG2|7Us|u{|k6 zsuD0Oq#c#3{=;9X86e#wdRq!3$Fe-Yl6clv@#OhqPk_(=OO_Jz_M6+gpryQ$tyQ&d8bC7^0 zV}$UT>ln78W5EsRLrPue8jmZ;`+Xn}2;~0FB$J-p6Ds673Ti(=o2uUak|%PHfXy~U?s-s!CCiA)T~vQm zuBbLd?vea-bW;jLv5OI_G_I9wwRYob^*&~dSMT?Yi+Sg~O(_L&?WmE(uX3afV$;ob z$i2JOh{#W+a5!Z@m8P_+;@6bVASCpcV}XRE(1wMC3Pbb}Gw`$$kiUC04K`Olm-xa| z6N}-P7tcPR4a)vhOotD^WU+#!56(Ui`uoBKoA#_t;;9PH(m^)p8EYIjy-W z_neRjKlG^c4E5~Dz{R&H^^~}Ak=~=#^Ks5$F#?mDp}oE^yLrK(?)fS!%+v2gvw77y zu(Tz!etC>LG7q(%nk6X-ghX+`E&j(#XNCrOc3dg98~v+%S8z7lnMi!>IP9R>{m56P z+IV~$&F)1!($wfC(Sr+?1UAm*{*qBWa*muG(LG*&srfBn6k#AOFJS$O-VBbEpKCu1CLbuFwMBE8bA55SWx>Gb)xXSG?8!L?? zhwt?*fD(K29?_Hcv)N|eA0vT4-sI%!n-fF8b=@8flcwS>F@c^%ku1M9nZLKJ>KAVM znbIqBPK&*(@3cBvCSz|$%jwx2d2S8OH-7&d*f=lxu$-{<8#1Gt)&V{FW+{VpnS=XKmcd&|t|fq0$EIF97ND~Y6#lfGxuR`MfXm0dtDz)&)J zk*!H0H4-Q_{@u>Z(OA;Vo74ch}<1wajSsIcP- zYSC5TPSkwiyO4XjFkYA5o7u0>Y@h?Fa&q_vcVg@rVkT(kV0Von5`Kh$6Jn3*1@7O8 zI>5+g#Eu*9Ry~nnW^?k*>IQxf-81q)U}Lk?o~}5C$T&Dax+1*TEy5nL`MssSH{NQ( z6q+EDAt#=DS4QYY`9*E3a$l6p}Zc{U3HvN&_J2PaQt?no@{)mXZ*&WFj6}t0c z`DWAr)BTOhtiBjgX8iCuX>D2Wg8eA~v$Yq6S?zjZPm5{rLRrA8y>^4`CERkPcL{@D z%9Js!xz(99hSu{!+Nx}t$Vjc3UCcGj%&6;{N3wUK*N%0rFYkOc zJyMj89yE8!@d-xFIF|208jngN-^DLTlgru0M?_Rp*Cs{l;s|8jmEb$G{@MBHfgfIo z)TN1;V5XHB+y9G_<#N>=Z*!0$v+)7phV0-zB^s7>5V=3(Pbuzzg*h?sFwwim_M{ zq#Fm#bjMCHmaLrhsM3l?TQn?-hi;2AwN3t_r|~#d+IYr5;sYm1DlF0;o<=#3bNP?# zh*!xjS}takxiUOeieNJ04!!i+eLPk$?O6kxEV*f_%vH9t+ChQYit zCG=Ko97B>^P54j^DDGzs}q>eKl2!ojjzU9Z9;Hy z5W+ZuPWy#DuE=Wi(_MOsjl5(jE3)QS!_2kpl9f_w_@!!{@$D+JyZ(!wwzl11LSFT% z%h>_%*+w~>oaCIu>g;m35N7w9l0eK@7a5q76+APU48FFNUjj%t)b?)9MjpPy+2)SjKvm!=`Fh$U}y7 zaCie6RL*Z-G^@_*vuM7)O`ZP@r4LMzwYnI%yVA8^9{?Ei7 zrmuCidt>vPD15{Qh{XhcCvR*%#cu@Z!p#SpKX5rQ8L7Os>@(%GdoG7VH!14iMvFRl z`csQK@LANsk1c?9QO|a-2!r@aYa!KFKlWvqAW`vd5iF@3A)N-qj} z;B1UETi93%8t2on8KXq~FN~|(G2;2BD5;Ns=RWj_E{2Ze93aa3 zB71P$_^)3=@uiyxgdIj1fLc35v9pXH@C~u^$Y$9^kfTc9Skx&8G-XjTdQSthEs_(x z%Y_Pc3JYc0oOLr3K4vBQTF7Y!Yn5Ig+KnMV6B;1TY zKu~-Ymuo8@PbaHlvQiYp+g-9XgBwCcD*3q>Z|TWENQc;PmBDPuD^o=|MW+ld!+NU+2_)QDP}F@vJ?3e-Yz?Gq}MPhKGEldzBV zsWSY0g=kP@h-CQ&`AmQWQM6e=5g($yD`1m|(4_&KSaw|_YZ8-TiiqghlZOmaYfxB# zr8#?Eevi6FT?2&3EWrS5cIF=@$*di?=`aN3N+BR;CiwQRKL(G(+zAyQ_C+(9Pm*;W zIuk63PyBqIhL!`OC(rjFyMbh}K(bh$;QMm@L9)OpgX{BKi3m-j_SMHv{0)tur1mDt z9GbId?p&B?XK^~oF@CUO*EgYt+QuRG9;J^|UMxf^M?cJ0?A*}JQ$;{NR3u9VQDM!s zscS>vwQa#$U^u^$lG-^HbqH0)!*-&_-F;Gsu(6rDxgGi^*^%Cqu%%+3q^$eQ7txqt(eco}pD^XCU| zxh4CQND4a%d3LxwRlZ%d@VR!bMyS(^Y zG+OA!!;r=9i2C8dZn;yk%FWOrpWS zd|;|unpID#u`|w$QW}Ep;9z4Iy7T2-NiR`}#Lz{-AqLWwf`^lWDdm40%X%8*aO4G@ z#+jXJrz13Bnz@ksdS5URiVicUj2k8#?YfxU>=?!Y1X{#=qUgHH?UL|!ZC3-IVRk@V z4g8ZErpqn_ugeWnTkj3iN9WKJ$~yZqU%z4cxnLn#h{k-%G!LNOvNBx7Yp${~@i8FF zE*KZhux`^8j$AOszAqhpmUn^So*ul9HW~dVs+kGYya=$WMVR@Gt0hK> zltMs=MLIl}()`dlBDb2UXrNG%@=Ay1hq}R&OF1dyoe31|?+iaLry#}?>IkNJvS&M% zz!UI~r7Ua8)(a$HY!>F?A~DKJIu8r&`@-@In0rdTQ3sxRwILmsL0vdFviaT_D<%8MuEN{*LGmDMh<7-BeoJa^p2F&-Uig4Wva1 zcgyr^MkO|RUWSjBe&`GANU%MZPj&D$bp2Fic;ZrCiH^`nUYw8u4SO0klZ9?-h;5mE z#i&sm(-SuHj!pcQZ+E9B#@=oC>jp^^8)!!`wdER?Oc=#Am|I8>Mo0>+*md%)p@%oV z>u7=ox?5_ac9XMiM(G2H3eS%GRO~Hbee+ zV9N?3-$CHRhN|!U)vtaPOq}2G z3w}S|2Es!BGFtzBq~zFBGr5Bc;kL5XxfllirF4BKp){)0|WjSQ%|LaPal3u}{n z{7B=}IaII6eSCUKv0R@AkBqOy9?LA_+R)bS#c zN2euVx$PV}Mc*Hta&LB9PSa<azYPxWmm*S=ClD_7ro{XzYJ^jFY} zkBfQPtwAS^U!dRolTJ*+?tJZ|w37CbG`^^fUIBP*b${VzFtCXfEMy5wQ`5tpcZjE+^KXP5a_CvjB9~XA2Qs!_TIcqqB!{=o;Q!81M2kC!v zFu(t8gPEcRGqy^v->0_*hd-jml7Za#p*fOs!Ux)C$Z%vI0M)+sq5f*5{+N>?Y4@RR z-)@cJu{yazGzV}D19*`+fY#`>+oRVLY3gXdj(YSA+>f&N+5^{rALC@VRR(}= zJ*%xapV4={_96XU(SGs`Jm{8ZzbfVe0pGl-Q-4)Do$eahO3z1brYDPz$%JfYzUC3n z*Rt*nT{}wWqYB!dD4n14cDZl=I=9*JeR}<)j-#CBeEplKk>|djOK+;$F+*~iScG~r z$6KHcJ(Gp}&#Ehpjuf>8y{~;J<0JK@HEM29NmWB63SMhxcAPagw0-|l_4KQbP_vx3 zomWW>3p#!vHB_*Tl`yU18=Rk*6*gy1L~Bzs`s*F4dQ$(4p?-5-_AdHSYDs54XLY8` z;8wF2ffdq~;WJKVFcJxLqSl6zb8kN$__lu3>zibTCdtrZTEhR*Yahsk1`HubLt5wK z2LY#HDzZCI!@1cBm|T&R7#6C$T4LlHh(Fsj^Zm&^8_oZV7`L6;{{I;Fzr(o4WSGIY z(o_FG#?}7EW8C){r~g?P_b4^8F|O3^e-6eyPHhC^N)2s{E3iHoS1SBJ1>>r^d=uju zRPkTIxWf{9Fz&|V4WR2jTn16{Zh$P0_#ei8Z_e9!n2vu)h}KBOgJscrgaV(9R6G`` zcrsG)Xr$ulT>mHxja;78b22{%nG#OI?i4nyk_>%sq-figRLr7>JPoJ%g$SjFk{V9+ z*AP+pa7*g6%|*XbMt|>uR>SE>c5wlAfDcS@uTI8)@T}OMaV&=u!-huL109pyT2e4+ zUaZyga&y?oRveE3T=ID~-A{61aQ-@&_&WzJnina0F!mUtJJdQxLm&|zQ~9pi`t=-L z@J60)IF%jBZa8&W=;DS`6GG25oGS3#KbbrmPWjiTH*_p%IF;bJcPxNla)(|gRA#by1 zMl7L_ejx_5zxK3_PXJ88oYe{FAF8agJ&jTas_kin4HNJ1^T#Y8;(wCtIYbU2BaYhs zr?;feluSb>BmSrPBsCaq6+cLdcU#5JwhI5iE}X9MG|Ii4D#xh;1?#bN@AusQ4=8P` zckU4u->i;)s*REtRba$D_a>4e=<2wf&(UdTBK~dW_@)H=bqob&4dn=!R~c$34+9Lk zI|j=u1-Ukc@}UlqaSWz(FmKHUJ{ohH^$c}4bX?JZrr&rI|CapFk$zqKC+DUdMMO-_&`t!riH!P0RTan<@V3sr;j{;W9KickWO1 zysZ5y+;Y(Q$;LB*OowAbe{TomjPlTl7)i2qT>+3D1>bpMkQ%GQRN74>{y$sm+Z%{)=r!d zm}g~#iv5o%=NKao;wy3_di=+tt|6WsoeCJy9)dOM313eMRbtL2kHGU^y&}IlJ@;NI zzh1dc{X)*5VqeEK5)q-7QH3Y{`vGQvJsV8YcSTE%E{yo^i_||HseckzlJlb4?&!oqq{EpCSUgq>>QTK2 zw71!_!{PZ22jC~2vrhYFo#yac^kf|_>hc&v|K`Z7(=d0ORq@2|@<(wJl{jZpe8cbc zqHTKp!@B=I#3A$Mm?Z)KzOFr_mmJ&JlGsFV*B^HXuHCM{v`iqR|iVD}R=%Z_ZA_Yb&r}k5i1J%6`Se&dx5J zeTNbXIGkYYv3)p$b!hpve*SH`e_QikhadX*&_`*f;>_8SW0XSn+WsJxpeSH|=-euX zGCrd!t(6`6x-2Q>1{}GXnDH-b+ONg*7;kJ3KjGc{hi+l zKP*!57WVtFjTk9;r!ba|`=E&botTR?_ZG*ISPFsHrMpU=R?~vPuN{ssZABZl_1SoK zBfNwcU2t=fqk*CH?~6?)@7L3`$H<67{n3cOB~rAvDx&R=dBr}s|FLf*AvbcO2wi(t zY^n@o>2s3w>?i`*qHBAbjf~ipI5>V)-^J$lT6AGTY^18r%TjVMF_vL}ils@Epo`zH z&DZN2()vEGaz@F2XXhBDR<4#%DA`-S5!ExpAb7m+Zob7Pko zcYO5~Mu3fncv%*0Rv{Jw)4bz(aC9C{Q@@-vFCpY=ad_p&KT3ns7vSFFA;kDMQcupL->iQy~d;lnc0GwD_lP@OefdzWt9Jcpb&p1Beps3%-N_|BH3 zU;@Hu3``4?5RIW-SSI$j31ipOco8uW$a)Pa?C4Av7sLE@?iWYIUF-c~imY#WPSaJ$ zmV6h+H2dS4ECI8`5x8OErM-Lea4b+yqof(I+F)xj4ch3dNfq(X6;a&yyeblLE!Y8( zGj2+&-wpC}_=?>1>6_0dWxgbhwPQz`i$e5$PfOQ{)q46v-1}m!@dTpwgpSL#)}SYa z+uUigT5&xON7Hb3f5r7imzD;P**NnhKVoyj@x;*0`c;n#Yc{T35QP7CltkQ8m{!aA zF`XZJg%L}V0pxb;X#91sNX1cDyYbgip&eknMqZ{>G`V_Rs95U7Mg%nxF;N;mp4W$X z6X{?kfLP!xEqnR-U%;95zmL&O2<;#Y$>Vy$QxQBoRW!jkd`kF|CHr8FRS`TIDQSCB zKeZr&?;q_52yCeSPQchqqw;XN+Bo#CQ)DFslAV?_(&SNgIV{ zrZBwfQ9ac){`Q5`TzXVyT{lQrU99_$idBz4IMj^`C8dVD2WwRw9oBbaAccnOk2=bu zQ<6h;=TRd;`qtSkHvey6&gf44Gtv!%TZ&-jNvYy0sbaEK#bZ?Q@%R5E&N8*jj9veB zCjNx=Yq(zVK6QcDf&5Ns7psvH>}~N;j)G`n18*bVU2UDb`f(4L3L8mIS0edNzOKv z*4f-JX=_-J*ZyYzzBPF~MWP?$9ODImTd;Dd7ul$$otSJtv~Zp&PMH zi04@C80nk`f(dhtswT7ZkE_nVmUhvi#?R?Fnq&No1#He4<2HifSFLgA5Opr{z)Cy; zN9;Ls&=5*s7{iks`W2e9Fm^|L0!A=foVbQ5HNJYm8Yj;tC8gjukT_jQ*A+V9W*V|? zCIs{ZJNo|K=vh*hik?q&J9bO5w@^(Q%9H^DO%}>9=o4gk|D9UFmd(aXhS_HXN{vPR zIj_K+-JQ+{XJ-uO-^byk#&ak&f$!zLg+4(B^uz&rf(nLrxytEb?SLW*rb-W^rFE$t zU0YMVn>7Mx%;j>aCh}&LY#{WwMAWEp&?2ZOeXP zTPu@O16P7&_O0Z!V17B(d`B+#b0AatUha2OO$?DG3z%x|?5So8?uz4~$q=ofwpfm5 zhm&EO3u^iiN1h@YCW-KA4CpW*mNa{z@_ZbZwsnX8=-KfGpzMP<)5HI?vgg*v0k!Ey zZEp0&QSNLU8G4+}L-(XLmv;m|`6}K6O|hXj1C`6lROutP@e>YN_Jo=Xo#a+Dm=Ith zOE%?qA|%AGx&id>QqEg~ew$PA=U>1#Rwr0ZD1ryH{Z{+JZpOChAc4!&-BRlcbBfnbxNB8af^14V#1C(}|NJu2N z6eM02bMe>9O&c>nn=D4sS{bU`dbekjkYG7Yn{RL;wXMz}a%}iP)0wtW{axP81&Qc9 zwHCN0I6988_sH2{(|u}`(?#8+p(>KbT^@c6TujUXA_tzQrBt_p2Vyg@uiv2DL|~a- zOGD3QLiIuU(4Xj|`FT<{Rn^P#i)$tBuZmnt1T=##( zfeGS&gTrx!|4Cf3IUE;hPjcjTwnpk7o6Y3sbb-M;U4w*gj8x!8=tQJ~i>GHj;dvDH zJ4sQMTl5%*Wly*eNR_YtLC?KY%+S+v)H6E#@(b+jopF*KMNV+_&uYk~X&xcDRO3JD zxJ)92>mPGygJ;+WJ)-p-^tPJ^y%P%}N;4^ytMsH4S$1Td-OFc{zP9rG|KfVGuSMSAPWqdA8`HQfC9 zx6y~w^!moelFm@V3@k{Bp;iusO^22@_47B8w;T%paVVDdNxW#Aj2V|WTBmkEEMPWm zlG@G>d8+i5s;j|8p|I$S8P=ij3w0>$g3ecm!cXZ}>qOXHCn-NotACP$+vRe}|1L8+QuG0nx++rt zQS2gh@O$#>2R}m&a}i>yxv$qh#I2QRtaXVXPto(SWYWvh@EfnlO+KeUwkprvKVxR) zGXq#!a?$mc-ko!bscd=%l?h${(kn8LM{;-EJ@P=r-{?fbt6SBUUas=BncNKB--uAy zqn>b%OjR|{g{er$n&-mg5FY{m?L0q6MQYN{V=)}H7ejiu{)huzH*vW290%5-6G6lS zw#~!WdK27?`gV>p(J6@fJ{YNg8Vyj-jug*t4(j<#>_?+hGJM_4=`VV?7>`EQT3n0- zy&e|RgW|*5NF?cOa$q+99#(|z--=k?>2clX5zy094EldtPhZYJ#J8Dk#v`GpS$dozxLcW;8QWP3oa)Qor=Ale){F#fkVn z-T%B^pNT+K=<0{%djEN_z~zD;6U^?*^*8ixmHFbdcyqq9CEqn+b9qw7)6fDW2@aa) z>t2t*16di(A7^a*_zY_yrzz-;)IUnyKH|W-5R z+IxCEcLOh(i;N)z`&@JKFKiMF$NJ0kikI}ghyDS!fsxRTPuoKufnyHNAt@P9-@lTU zP0?JUnlvd7rnRV|)jz7UCo$DL_qUN3cw65at=~-*o^l?3>M1UyxwWQJ;z!Q^Qgk9h zFKM3u(Dypys-|fN@u%zGjUV0gNi1gH0cL$$-B9FB^jCd5P1zKYZrHR3(+Ne-hY|{f zIl=Pyf{5L9358CMea(8w^ZEhuGloY>I7+uiGp|cKNF}1KtEqJwr@O64);AkjbZbRS z8P$J!CiSKjEXFjj3p;VgwSw_!Uj3t6@O;>jIOVE!{#d_;++5Awk}%gt)auYd8X6z@}AANCP2OA^&Bg1_|QC7G~`auk|bpp&pxKZ|90e4CUhhU zQ~hnz^U2~Lo97WIq?{)4F_rXu104D_j+is?+wmSuW1AOJ8t6un%>`$$BTqsxYLy6 zTsU&?g83Uy5ljeWsdpj?_G;%SFP#;;!fkmKyZYxf^vGzX|M<;%4Ey>|=j_>(QxF7ct5VyS5dVv?5DZTvtWd_AcRklYd+>I+>2_IN=$Vbe~t92Z>Obe+Sa`{e5* zKS4I66IVu!qMuW0Tt|-*znmTW&E?%R15&9wr^{@`0@aHBW-Ioat=ONaTCv}3#eQi8 zBFFmPk*|g>rK_ak!+4t&e^nd2alfkg0NxZu75A&kzpE;rK@)+}05Vby;GM`I*tO>; zDsxlc$3+3YHY3vP4~IM5PS1aW4E-l^_B1xFmG1tfWTSfgm#PZV>AzH!kbb8lq~Cuj zCEH!EAAC_ppZ>m-N_bVZzi{O7mi``W^|$n_S#8zl!MxEj>hp0wH}tY5FlHz{dzXq# z^9Ep1C$a=T2+lC8!XyxU!Bb|%Rf3Y3Elz#cP zro5t>^2WC`<&A&V6xEP_tsTv2`*T`;Nw-^_hgWiZ&Fg)vcCd-!Szf`K{R0fj zK%~+@YMCW$8Rus~z^YmDgd~M`@_x=Ajiv8?74O*Ja7K={Ff;{?BF&p&-1>VNN>kTC z02Uji^MIVQvlJV}1mRu#Y~D(ys)JPEw3wC_aevtBqJWcKZ2UD1jW5m{ml)m;glmAw z0*7(E3W}Jlk^Mv5`TRvfx8DaAKg#j{0;rt@fs+b8^!u_{55z=izsQR&)-?^ z(pzJTjqQJv+Kr$%Ns|(r<7-(DFGW0pble>3Nom;s8^Z8^tUp1 zAw{><79gHXJiLqd>@gppHu7k9Xz zE2k>xxHZZ*O!;Ks3=L=jqT}$Xvp*TSlkWFy28B8{;8MlY_zIzFS!(!88T34ib3~iP zVG13_aGYoS0r$@qXY&Pux=8T~AEMgPz+^VozQ%Nrvj`4^HZ`p~XM>x=2f5WE=*vr1 z4zuJ$nek@?xH-H_Wd=QxE%8-V){NZH2FK}w*iPdTvJX#6($b8%s!SPo7^(d#xYz0| zGdxlf6Ym6U7Q3RiQqdU4)D1Q3&@rm377mQeNZLQBuOB#U(_uqA>!;*E^IW77=E}*V zGao_0eeD7sCazn0`u`1Jex;NVWL_rQOSEs=jd}HT7UIHHH2P;)xsYqR_RX94v!=y97zlBO7#7u_9ILppgy%TQ!Yqt ze&>f~@Bb5}Dp7Yj?zM=TD!n*;-AvT5WLAFaS=r>M@~k|Fkk{xq_0 ze7vX1qqXr6H^%#oW>(F_aC1iN5Nr!Hr*Qa*))zozh%u5xAI==v92=pS+B-v8vR*ty ziv8U45*(V-&bHVaEG<}-46UvCWKI1ti(iRn4!A6yMkVj9SpP$emcR5YhOH0Siy=h+ znM`)2p|&9V5PmmALv4Y{YgZa-E#Kg^n;L4Td$`!miE~6l&EmJ48fuE&9vj$q3CYgE z#9e65$>W*DV3smgp{MZ}-xSv!S1Ln$tNPXbR#lb4uxzsN&|lPQJ8rab*UHsOuIb`$ zE~_m2FTwmNe#DA?3DFPNXyWU2$CSD(!E{(WOE)AMvlVpp7BOfrz8jyn7 z(P{qXYGww%X$J&2OAzinmd*M!pX&zLe_gU5yD?^O`&Y5)E?dPX#13+XiyqVSth(00 zQzbT{=!m>(170xto$TUIh=!19ZEqOV20mAwpPk1U0DjEr$iPUkD^l#vElKy>Bjz00 zL`yOvMer56@f&~+2u!PHb=HRY3Ynts#|Ho>?oK&+b1R>AN{+buorj%WK+-xRxD|Py zGw`WMoy)m1au=3ea9<+FTc7`ZQ^McLe7R6>!Q{)z1mr7{!eDKYNG1Js;B&c)PSr!FLSAy zfs12^y=Z!(x9!}>U21iGDFf+Er0q&5PqJB55eT*ar?L0IYgrMJxvCY|$e+94ceLHTlXeA{|jz) zJhJx1!p|lQPoC{lM_sE~E_HgbS9T$zNw>Byl{_T8=>lsr$g*fk3`&m?17#Nzu7Z$DkDl(cbm*G<#rHJ zjKp<+4tI~*a1?sTvm-b99WM7DK4nEvqgL zo)?|*GbwG=h0IF&I{c5st!`~0JCT#l&w0D!5o!=TubgPy#u9P0GyIV^ymJpDL<--b zy1igs4%M^{+GDVmgFazR+oZ6f|UhtbOl=s(x0I>|W>}qjOiOY=-p0erLm+y|AZj z*vGWk%JF+P?Cl}-eJQ)^YQS`043HIhnsoRf1l8=FXVB06zEht=Gy{oA7Vl;v7vvEG1xgl zVh#*9PlzS`?tInPWb+^LNmSPN#Yb0z8sekZ@b~{a|7xB~am@Z*iX-@6DUQ2&?&tXh z&vu>`p0{|8^EiH+;z;MYkY_y4G@eqPWjyP6w(va4b2WKg%<~rE<2>Cwu4hslSv(VY zZsb|QQ_HiS=YF1_@;uG6ljjJ}M?8*aDUauTo^d?a^33L`=2^pYKhI-4+jw5)d7a1L zIl+@kn?~_W=DC^YZl3S+Jj=6>$KXlcPCIz6=9$G)%Cm&0mS-K$gFHXy`7O_Gp8Y&; z@qED3&6B<(#W9R$0#6~&tvo@Vbv!@hd6MV1JZ(Jh@;IJPab)m};+euTi)RT>i02-j zhk3U0yv%ci=L4Q@o>cnsd>;At0(~j}0x}+7``30$zGM7<C+q+ zEtuwb{aAia%rCE+=J;Hn=JZPH@)e>5=w6=0_ zXhrQ>?-LQ7Xw7O$-fRW852rfP-b&dS=F%4IoMN^P3=>&jNFzIr0{TueRB>dV^7MWM>CclPC^?H!Za%H=Co zeIt%^ym#>mZP^lU&5Drs_Db)f#gcpF5{E;5TfL}GeW_ZZ)d0({@p`qIJ8M>~uJKmZ z)~=}aj$Yzjv3RjoTe-x$x>|an7IrJY8#Wa<#+{$%xR3XUp?Qvnc_&_w z=XjF$7~T&z7C1&We3t=OT+OPKH!DyUpcJrhY2DQmZ(kac@X|W-J9#c%&6xAq526EQh0jy4NnfqdMdrQ!v(hMQ-lI%UFw(yfToW1KfoQgGQ# zOIJu93k&81%5S=U#>|@?j+y@a=?WIO$zPOzgMVSE|EA*W=gbz^*?E%PmE?Ivj(66g zx)n9vKxOTnOKa-9i)xm5L)Dd_i^}DT?v(e6ASh1CvGeRx-f9!~I2O(+b_DWoo~g>t zFSuEXT$q2;jABVQCm_Ei=rFNbougP`w>q!%(PZ!FI-#T&t~85WxO~yln(@mt!6|l; zcHWXlNe>>bqIB;!lKH$^U8VPOt0?aZg+*Dsl{Ma_HQptaw`+HJm#+<4 z)>c+k)(SS4b}w16SX*9M!#IH{y<@-tw`)s5A0%Z4s`!$a-yq{fOU90ONbN`0UH*-F zNL^KFyqTYiD{o)4_)fxSMU>xmBMasf73I%X;|JJLrPVA}l~_?XzH&|FV%9;`n8nMN zjPs5kKYnZ~A@5YLIbp_E2~G%^U$vT*6f|a0?Hyysrq))5wAvbPZx#|~Y%23Eb=iu= z>;yfftXx!ESGjQ6l7)3E7A{^C$oDY$@67dMlpvAmgJ@ zh;(<|n7$c|O_g3*IL^CDdS&U#Mf8Imp-%OTH3%TfWtHOvA&wEqIM&wJ26PqkK?c5Z zSzTqS8J$^NSsStvNJm&PBo>b~dwK_`&ykghnuzBoDJwz|EC4tHz9w^-rfu&3m(~UbOU1 z?OQXXPUKm)!pu?v-GMuc%w|-{ik!E#+86nBi|NWqSFI z(-(eQQC26JeekbV`RiGEnH?iWeA^`X#`ur8oE`7-WlQLf@v^u_j8*f;>TUHJKi)fH z@(2)&ca=F+m;`6d6sykWP>r=mN~JX`Vh@knZdtl&p?{-@%W3Y_6DMD;)zzB8%O|U# ztRE9FLY4om344-R0XvbI#9A&@%R_w@j8$#bUUTuuoTcZ_#-Ai&;h#OBYb)!tWrBH_ z$Wk41OCKZYWsQZZ6)v!1{XguzX?Gh(k~TV@`77$cRugO!AVtY)Tedtck(8~sweX6R z?LL%-2LwS976`BbP@=};|9+n*mRza|3*}|}-rJgK6IGQN85tQ{Mn-0`x3CkUbM2K# zoXJyAwtxlJZU+7AHnKgMa6n;BrTb}l2vHxJ+kiJ6!u`SMeQyjxL$UTx{7J=FdwO*5 z?xX(nyU}EN@Zqv2`g)~t{lz`JZ>+C0`W^k-xj4}umzQ1pLCF3v0Xy;i$}cN9RyNQS z93thORujyXDtT&Y>60Qww}@5HQ(_*}gVy2Kk^*@7VtaRg|HVsq72DrEIoRDkc=d8O zssx8)LR$tix|&{IO-HBAMz4E5Dn9-0#s0y;KldEKN^CHe#M}Zb(z;+H7(Xk*KTc(r zx^vKW3fd+NS0cJF9=a>F#d>+28B|KN?`ennnlF@ef&3>Uu^CNewDsbmGkLpSyzdOB zVn&27`R=|rrs8Kd8|@9(X#4~sYYCXAJ1Nr~!!d%|x9`?6;aq}4t&&>dTYpt+=pSX! zV|Dv=-^{O<3I;fqq%2V?(5%j>oAymQ;t#?tZR!|Cr;Xft!;tnEdVMLE#? zpQFi%nvudsUbyfhNe($BjU0L)wc@9=0F%z374;yiTZvrwXmS-bPB4w5YKUo--C>*2 zU#CMYd-7voVDAgyPMR`(&~BKGQWxGVr^rS0H93!<175FfL^in~>)#i<@F@2l%het9kHwoelJtq#|G%uag@qm*3{%P|v8$aky$n*w%KFBrc1*<52 z#^wDp3QAj2{cUbzLDUq5*VPHi!#@G*Yu17E(8Su^LLa00-=VcOj6m-vHa&0O~j zHPrM#j3WptuucUx(9n!`5GJ;NnKVxEsWuL^*)P5+Zi{-vPp}f(7Nc%0Ll$)o-v`wa zjL@Aq2#parkHqMn-xhQfQkaT&&$q(st@SOjVe$^<3-h7Wkp+a=t-)~jhK#Yt7R%kA zOMC5NxqoxIR~_n<)Q20dU%yfJtUSurlEx!aYF^IhU4POg7dzXkaEAWMv;wBSOWoL> z$bT<5>kj2eU-R%nEdv((X;(TD%D@zLib&w38HyvQf!``JIp+bn%gbfZQ~Xd^FzjAm z>)#A(_Tb`=M*bXX5#FUKg+MjsCa3PX;CXI;tP|(x#2-j#DCp7ExZ7KT+T-pYhu9e5 zRoIcs^|6Tem2;2D;pCUC|QP)e^~Ln%-P+Ot{h#1V3vQf-Tca`Ew* z2&`mhI>p+eM0WT?d%|D$&!{sS&BI%t-D=-z-TJRvpWbTTTDx`Y%^JD#)~#E&p=qps zrP{5(VxEX%uQxa67qdcJb6^P!3Mp;*tOikNi?Ha9v7Flw?Vsp4HLuaWS}dq{eK|mc zzbPS9p$5i%08ONGtCm8BV65I#=E_9oP{^jj?k2ABdz+E+x=yF;#?c z49AJax2+*GJ|ZfD0fEzt$R3Wz@#?$^=df9s)0#3`0v;rtMZisPRw5cvv1R@@x&I@CZU zhSr+z)+F*HnEFDEwfN55!TMe(`XnR&mo>YvPyGv31wz>49?*nY>6}6xt|OOV`o7nL zQc&&>ln%l88NIP^okT`7F5-}!0W98Z5HV}90@8udPWqXMNzk7E5&W!d5{TCBJNZ*u zktWUQF#==4h!YVmB|7;Rb0S7)Gs-hYZN-zIr&TX2s1ke|5E`!(iyVV6UL}Mly^G7~ zhb9z&4W7ixAWuLM6Wh8@ijAKib4Zs8Jl^4qr?6|fpN_8zbk-dm9}sC)f4zV?w27eW zm%6^Z?R}V#P|kIHhIai~Z3{xQ`UuLs zT^Guwnov7@`1TF@Vhc}X56Qr3Hu^&_R{un~6Q2kmrxB1yTk{k6RfW0jDWA8b}Vg{UTd>LQoRI<(r6=O^;H8a2=`7ID^f80 z72Q(6l`xW+AS`XbAyb<#zk-k4@q zrd){suEWy^6i(zP?Tb#}9AOr6h9Idfk=-*rZC*@}!6O<8qsz8P=Fz}H5lWG&%1e9TLmr6Ot>uM_h0+{eb+RKt!~n9r~f)-|4~C z1>tLG{l4Fy*WMhsKOT>+@7&v*-IZ%Ca5M0psv@K75iA$-lMw?SXi`{D#I=?SxMT=xBfY<&(VwodM~(lPx)^rZT;+)i5cvm2v*ETm(~+ofLp}kI^R@D2NHs+jRHZf%v>b}N?X{x(SeKap8 zz^E7wdC|<(wV0Ju6|17m4KB?CsLDT)ch!~gK|torg95{LfdLk8gYGu{kPEoE5(?Lc z_R$e5=Crw(+Ur0F>#&cbw?*41^Jmx4n87mTGFRLBvOPeKxqV4!A{nY}E7CbRX?8*v z`p9%2p7~18X}_l_oR&ZLb0bNxXOswOw`XUtlfQ(CIn8qCP%6H$9$qL=;32O=2Zb8? z%}Sc%YCU2A0g1JR}ACFC-b(Abh*m>7FA9rGW{g84vx7j(Pf~IT^r(7Vm~@pyQup)}iUU$Sy<5 zwgbG>^kJ!xfVS5WvX=v_Xx`ZXh}&sf`l}bWN9aA?NIfRb+y8*ksdQlkgwbnvhkrcmT5*xF4IJ%dzt>VMEP9*>dU$M*NRa8 z;-@p2>A9I2xR|Aai(Cf-~4JOT{3Mi)hlwEDd2reisdqG{1y~u+kzV*t4VY5U3~2&kE?5B(SJKA{3M`ButYz z&zJNDN`2=5ptiayGJ^*~6J6R%Y`B9% zB_9!I&$7yt>4DWRsi-sCehdK^RU5#rQ11;&XL zxHL|OE|E@_z*hi*uc7ZVC_XZ!1Y?zaCMG3RkU3_p3(ZrAicu_p378V*Fo1l+2FMKh zhBd%=;X(4>xo}~N)Zv5o)V7nvHaMvQ1voyT5v(IqQ5;r_rRbUgD$rvdB zlHR&66EuS~o*{+FC40S=FFh#2fgqufle;a$Awym&s%c~CwQO_`O-FrqY~Eg7duDZCVfv-zt5pPmO04s4_mfaory2eTk5X1}09El-x5^sxQtL%Oa)5Rvk?mjR z@PPQhQ$%-#BM^&cFs>&e{`1OcNLWD7du$|6+~=$@xLP(#Xm-|*!;GM#T=I2sY}6fs z&8YZ(@qD*hyI;Zxz=!Ck#tg=z5mu4=!~V428JzSkx!FWT8Y$1f8F7-469);pb=^>S zR@d30KT*o6#VMRE1zVIk9b)z;z{p|6N_ZB+pi#l3@Zz6j)Ik#8rjDUQI>zSQE&yTK zR6hv3%HfPV4ygW{8ty5SUBi`rP1zC(_T56VQa`40Kq zwNa2zN}uSGI0N(!@GtsKn~<`h6;Buw{oC}ROSo4HDr+{u%gWp-KMQ#yon{m=A0qV^ zJkEfY(?Gw!o;zm=g~W4^dA*VqfenMF((lHj9NXXw0j?3lAnu6-m@yUrP$+^Ntt6I) z7wbS+V`mQHP*--rsyfORF*1fE?^L1+MHW^7BN6wg0^xv(0lqGjnP_@r( z#(_^Xg_NdG8&ct$%7U$-B6gAD2~4DA8gK7{BIKMmc}GXdoJ?@fqjA0Y~F7ODXN&cp)3*${A1=(TZ zDl$;NtmaIGc>F;@a9|C}ksPd`)SMhn(No0c@wqKLB!v_=?-y{CPm~5&Q5L0L2+n6< z8mCe~EilHU>xn1O!%bO+k!kvxOlig3d0<2Ftx1kabDpd+Jd(^*YAp<$aXwBmLh_z^ z!ZzkwM5iAbMzC0g<`}R3Wp%wM8h`0rTz-Xxu(}crE%C3b(R4lh z2ntWbu~hvm1>u`n%3%VM6EK)AncGraGarZGG6Jv%HLNg{OaYaAr26WX5YH;sd|zT^ zLrG;PO3Ja2wfXyit!d8@f*|8b=eqh&4E%o;Ofo$wDw-;YDq41&(G#YpK?IB#j2^L* z7+EVMx<-?48vM$MMTxOkKhQsj01N+&nY1^ilnhWq@@J3%VA6z|B#Nxd4VwuXVg|s9 zD8>I`0p3Be`@uH-gvD&v(kM5>3k!r{W8P>qUF$rMXmXBKC{@`&!x0mCRFnl#M359^ z#Z^<5L=c=1k~i!GllUAts^oIJ~fELPhcyJR9D1V!wkH{LrE%! zYK((RRyGHUAek<}*Z?{bETo6N!zQdGhbojM~rhLxg-MF=Q-Vu3By=CHbv z;STRYHd~8Y3UPT&%V#}#kJTBFPMPHxg)n?SW3sNaz-yuno}nnPkUokeyB;)3AgfTo zXg{?+yQyh&k)FUmye3F*i}h;ChSzD?Fu2{$ki=-}lnbFqB)Io1p+|qWrKjAHzAu@5 zPjRY|FnHxNPetqUb8g6Vjf!JI4Qr}KwqypQC|d^eTD1ioJ4S^vU?-#LFqvh~g_oBN zAkDTduM3cVdSSy1PFC&Q=_n8Drt6}=-Slt$k@Dd>q#w6PB|;h+d2p;=E{K_EaqU@D!j`8T2M;(?M6PiR_t=**G;sf3cW8)(Ze-`MSwLVKz4ue(@9p*W$fiY^OFJT=8CSwNwp@H!_)*YvF zq;gJmp6^zsAyu*A(Tnzrz@o1MI!S}>qXmtgj|Vk|^ojb{7_i#7?;hzj2@_LJ?ElrKIMP)g zS}hy`kwi(s)lQyrlq-Hny&GcVB9=%^j=LNcFH;C*KIa>;f@;4YAkKh*HXV%y0w^!h zz|_5et!xv_SyC-fRFOy+)3UHa_8;@bxNe_Ifngoqs{iY6lg3GxIs{ZlvUNCoa`1)t zF##`if2GG*D-dfSx?Fz>M#qyAQR2xf%zZlJJ`O?@&hZaL1AKdB)KFuhDn>jC{+wxi z6S0H>nxPjkBHmO&sI`?~(MZ3Ne-QdCU7=hxq&yjtBae-LXOs5x(US|N;K6TLkuDkD zjs4(G2%nLYql1IibOgkv)}WT#+6YU@mOzTb9u$z$PqTu#9MX`Wbb}PE>65g?{sf(# z&_Q~nkN}4pMnzF#IJ6NW!WA3w2Q`xoIU9W2c*A)q;VK_VNqghUh3^ay5R;Xk91z}? z;jkd(%T#Elw4$ZP6#E7Pojc1+ii+!4ja>t{WFWs8+ zZ?rq+4wyTY(U|*&bWBnTe*}*VPHTANX81vnLk}P`L4x&VS@x_b#ZQE3#RMo9T76lr zYei94f329L<#q8=1Ayh~;R6K-4<^B3uFFL^eGs`9&ZAm#7PLmKh<#SgWxWEQ=7kwH zV@p;CY^rBJ5iB13y;V}K z4qmJ#G*^`HTuYVPb7Z1NRxaV*S*fAWjY>^Nh3gf;uQNA9fz0&7g6FiH@%J5K?QU1) zD|g%ulkXbpR2yb#_%T3H_OIA_mibP+EP1`uM70=@LB6WCmltlIb-S5U22SFk^lhEs z2XkKW&D?y#w@g;euv?8P)6JbPwb^Tvmn=x-Gmuivi_SIJz+T=*pbXn)S{lC6Cs{1O z7TW<%edl5F3_r<9GLt+=DJaN^zX&M#A_MY#t#lHq^!idtM}@1Cj)NS<%FeZ`NBM}$ ziNZ+0Go`9LEQk8&8Lm8<$4^qH`tW0btZ&Jg<;-{DXd^`e>rvMVP?Dk|GCU`UiF#1& zEA~mK>Ruk-m5)`-0BHe0+0r?V_LrBR0CRX9Xg?vYAsN!nSdLlCIQ|^P(}H?}ksd^( z%K=u3t4(4G4PBw64?f>19%+we6mh1K&tkk4odgIh3MUlWsqQ&;l=QgbAZO;-Bk-0) zZFE!sX4Hx&_= zjotrnaa=ZPv3QvkD<&!Q1j&)GVlWB_3^FbcghfLW@w&Om3OgnA0vWL3Lwb4=NTDRa zhU&EIm~zKfo}!f*om?L4Qo};`z^m0!$lv$~l-`p1!gj(g|KYrF%xh;-Zm}2XcjKes zNUqtE6CIAhTAJgi%5!}M3%2h^IMbV+7j?usJbAgyCV3N<2}2p@QNV@{!=a4$f^!S# zcOZ`wXY5K=l*03}Z+ksphd37wd5}{MY-Sx>Cg0okX0KXd5f~hLNAGaw1>!ZT_J-rV#fj1Y^8D!&yxc*xrVVvn{j1fLTTD|^LSfkDC)|A z;7KX7bIxPz(<~)pBPDy6xQC0whLtK-(Zvsam6<_Gr-#0+4ZdZbHZ~-oL=MywE zYx>gbi8p1#615}>c>#tK9YbTi9OulcqoQ;V3su%BhFCb>X{DM*>4Q24`VADMt%_El zZs+8utBEeUHeFPan|Ta-TvLV#-dI?eU=@`#?ZaVDs3{*f{E`RGjV;w!TxPT%w2E`FB?o9e_6zX#I zv_6;%yam@&0(&4mX4jqw2g+$;AGy0>DQ9GHe@(9;%S2OX{4WB7R!VxNx?8e7joSEzGY>^3jKA>GIxP}|g{wrRqfE$D5Mr!V>!OVK8bnS; zQYV>(Vs*Gm!Crmyz`q8ZCdqEsOd7u`8*x-@gKXlhxdB8HAs_~o9=;3zl$gCc8Vov@*uM!9+n~En)_5tZf;~h8@KgU%MZ3Sw zY+r1j!T@=qDR~^vuG#5Ip@ffD#WR;-2^TL#G<@({t?79mr%ebCA`nV=&L!xSkS|fp z*t?~|;XAPsYcF+=lmgPNilI|_p5XpUHlsyxmga+ZvXx;0s#-kJ5H8Yd=!1}<+qo!k zAj(K?&y^0rFFfy!gPC!Ckax&gAq{WNQyw$%OnQ}|X^jRf1w+9qCv@hY&>d4Gy5VT8%-Ek|g%gmTV1G^adG z#J5?hFoCHSG#h6a4D5u7Yr-my7d#RElv5yr?}<>c+@;IduBBfBXs#C9;AGVFIEsg; z;#+2@U`a5E{riFj)#9mw-tvA3F~x zF<((-lhrYgtD^aLUrkNL#(x(@Y3MBI53 zgYc7_-18K3LQ$59jnv~oA9EfAH;Hjz99_1iy)=)}+(c_UkinXTO*?rt>FtewQp-k! z%^+dZ$8&IP688XQ4wDwKDGbWehiD4^mhum{S%tsc`6NtG9UFD+3?laA(96H^3(o?e zbD%R~GBzm1x{4~CWkEFkVwmOzHXT2A{(L|+07i7M0ty>T`ExWiYNwsBQHv7yj* zV1-CPP>P|nNGdeFJDiR=@%tHo*h!YL0b0i^Hp5p+YQB-?4fWV8zc^dH7!C;0ANlDQ zw&`q|A>=5qKo)lp{Dpcyaq=1{xyF(ynF6sQJMLwUR0_#BvDKmoObR5-t`HMUo&^rv?O45J)WHACauQ1$Q)@j@^#BC%K*Sws$ptnbUf~d2 z^b%`kIGax4+PEN7nzz;!!U7^2gVE@1VWX|XX>nzg`m136SMQm~Nj*2otPIB*b2F)d2gN`esNMCpQ=ah`9FI z;$zyIFkEDT{(8EST8p+!Dl(^0Ny^?iG2YNQ;v6(=Yy`n|#N6bQCoQHRBjR&_5D-IL zD!82LyhBo$s|ljArt957Ue55 zRx>GPsOPyPjz${)fgC6*?vF%2Dq^IyWHb?X)&$jkl9sS{f!xR5SUm;}<)z#uj)1xWVfZOO(aPPBlog`^ z;;xfvKP{9iW~&0#U&Zc10oiB=%gD_E-Lb~ zrN+v-K9$Mhg;LdmzM3Bv|4}jE4wT51vldZ_aR}CfcmW1JgaiU212TXW2+H+IQc={2 z3^O)?GHTcDP2|WTsT7GI?6FkOOqnh*>j8J0Ldv4e1OS9p7cRhD$;kM?p2qWS&O0}X z@zn}NbC%I@V0qt{@4uo1OE`MhOF|*+6&7ZCizkC13&&RQGrgS2bp$ONoCN6FS5R~h zVaGAbVjXKM{5ZPRagVy*ioHQm{LT8Z}YSl{Q6nYM#*o1Gx&4?DK&?UjqP?J<{2*VZ> zZEi(Lqhj?(W!9hpbo4zBAZf{8O}L~S@RNmQ|Af5|`SpaN)usD#&Vn`V#dR}U_Suu= zWsG2B`H{$Ff^&v()hjr)&9v@d_YGI#RUbxSGvbA@)etwV!~Eh zt#{oWT%Ev?E@f}q`-Cf5+urH|R#H_jonx2mrKY%f2^E>rnmlMBx+X7_AMz>KT1TSr zhkg)gIVY8pysV>!cGVrj?T0)#LJhT2!h7o*c4($4$kG zyO0yx>>Spc3=%6#Vp7aVN1xeMk7`+hg&q2eTp>d^H?xFeLY$ggj)6NC^OFSwBJND3 z8G~$G40{}-aC9uki1>=UXLPW8-&p|l^MPk`Sz(L--bMKFJZSa7v+Xdv091qpup2U& zUtd84U2Y_7KtXYZ2e1^cyeO2`#3|-@gOIe`qr;U=oP!`Ym*O>k zUVh-;U0jBH!XMAE*w9${h1a)AJKWxc2)-a25R@2KT{Eba&G#16^VuXCMornthBMfb zN+8C(wxoun$2b}4si!pXCUE5^_ch+;01Z1=U*i{YcdrneySmQ}p}x z`pQo4`08vIL$m{~f5wZ}OYffyvHpRmgOuJKa&>fnf4B6wWzW~hp826O9zrfaL&ATI znd6}JYB1_dA9o}I&k}sxhb$q*K&m|J3^C`V!NJ#OkGJ)Q=I#Bk4TESeM)&SM>QA@N zd)#g(g`bahPx{k`!;@`v2B#ERU-0_6i>pFZvIzD0=<2M15|_Y2=ND$xB&wkqatWRs}8 z+TS%e8aD}psgEzYPg-&*fSvCam3p|IxvMwK;LEGwq;qN%!l5zGpz??il#vVSCdiRWVf^^+*7g~QtRJFRRP^rYej5l*ToG_o#H@fb&*Vs?tkoKu7qqt2;F#XX5b23inRUR>)fv~MAs$i{&x^^_ zVJ%%sRp~awZn3##YQdl3@lrQ#&CZCqNj<$i2}CJb6koWiGwN0K4#T9Lz1z?QJN{QkxwCJZ&8tHA6Hu>#EONGuEpS;V=x~n0%kOkKh7l$`b4swG+hfvU`4xRSB9t?ct9-``v#Y~g9(NxYB^F~;!e3UKF zsyDR-TI=SvK5-Oum+?Tas4Z+L70<5fkeCHhc=qKcJ(zg+)vO|o2D$ZL?4te7Sx*X=8DPt-I@6OSC{rjNWQL>m9>aub%AaF*>OcxQ{Esdt+=B zAu%}mOrgt0zpU_5RV0TXCFQ|gP{{FP&-G0 zN@h=Z&HQ3#K2J!oPFfkCn@WO(2BXsJd;%nJTpTZPz;U7ozz4@Xr|<$_TzOqDKi1#a z8;r|&gxG>gIu!aA!{A-PI#-Bb>!y)rIL05NpfnYHF^dHW!aGS$(s#i;8RDS~1j@m0 zUyXTb>2?4qsLd}4GikZcf;Na&YA=kH3{?Q8OY0W(v*fGr5jch4VRveP&t$IfS!FKt zPFo9K0SMkvAgC&li^*A38s1kUi||y5i1rE90Iq=xuJwSLkRZWDJ@1I0{m?8?VG#_l z0yAi4w!u|O+4g_~Is7Ojqq}iYoCk*9<7Ogsl>%MiB%I>Y;l`O}m&1biB~u7!#%!m4 zB6y~Golr>P7&8juOalkgcv3(m`q*n&-%)KeZsNb|1rJ&SRwTA)ZZDaf7+{q`Uy@H~ zoa+dLptyzJogk(%>5c8k44y(Dx5dh!l50fvl=H+ha~)At1XdDiAi^|wdXp+!mm!mq zWuh$5W9+oGAnPXyX$)n8SLe}sR7Z$-ssfS0|JczKZp58_{(A9w=KR-ztf#7-SF?a) zq!h=(cl5!1!=a=$Tw#znImo9ZHytQBf}_0NA)%8DHV^8k9W#(gl#!8&D%0lfL0QQs z^C$l5Y?46_y)}yGEm4@W@l#qrjG1*;os{Xo2tz3|G@cBB13WOngq@}Ry>A&*h)^oV zOh(5MP&{iQ=~5u5a48rg!(1S)7dR%-td5KF9HpS>E0$p-lX~za=W<7A>twVXGht;? z3UbacyIF?1f1h?Gu0FeAcsD}ES^6`aK^26otRm|4W~L047ZcSGwPsc>)r_J|LX+ti z=uB0a8S`VUln0ooGE5r`VMOWWSGE~R)AzV~t{3e=3R2JyIL96sPL|VMwJtKBz5xp8P^o2^{AtToII4kf)Dw?A z%mNwgNfl}pxv=0l3tBW2%Ybv;KJU=7DiOR^v$hFy(CBE2%p?myZbg-&#f>@oHXJwS zddnxS)^NQ9IvaW=&Z6Q3ttq)0;-FLA%2O@H;yf@`PSilq5@kIVOUhN9`uR>B!|tP?V>-vS3(2C@LUI-**|_l8f85%ehyS)MY8isYB3x1#Kc7iqC9gT4A8Ce@>50Q^el zY1M-IS#>x4^?A(zP6oFM4b6az)%Qs&1mckjIX8^94>fpatmGovOa9S2DddYK3;Rc$ z*)`z|qWk^?;W|N#k=poA3NU(WpW~sxQbxHubO`To29S>GS8+x_vY|gglgP@T-4#q` zqz0iTGrtA26BFvJ8%0f{Q|f0r8)zlpfyoW$Yzr_5bry&;!L?kqq>jt2Cmc?46aNL+ za!#0I%b}IARHvrgL>Sn>ZF33>J^>2=E=947sOEQ6BTn$YO=HS_CP0c?9)`X5S*!?3 zz@MpMWZ%wY06L$HdK2ysI`81TW;SrG+s;0R` z_GANub(d#u)Dst^oT-px;gfq4D8h3=^iS*bh~moKZqE0Q4PdxE!b1WJyAG6 zp?7gP#c4EWgV8Y#sm;uu3NuYfIws%(Nzmd zi;W=P%)%hyIPVPIg4m(C*3?GsCct4?3^>vHmGlQN&1-IU=)@=k=$kflDc`b{FkUFH zsQ^v}^~73HF#&Et?LE{6WH5|FOqtT&L{9Pjq?-Gc>aUrcbP zG%+h0&-&a?GCG|WQm@&abOZJmqv1$O;k;w)RKmF@&+w*bUUVlYgMZq|O9xgo_Hf^& zOxlP8VG-C8edTPu+SA8Rp4JS|OYTeNC+@ZG6nn}@g%09ZSO7Zha0vqHZ`GLWZhLUm z!P4toa-a7GmsM`q=}(ZoL(2nr$&T;^z($j5Z)Wh@w>_l(W8#Gg9S(w$o#q{!uz7ll zV_;zCut@-4nFgyz7XogKjQva=<5j90O8gT>d$hl^UOewj7gFM!4BW6-yx1>(beK>^a&;?;H1_=(S7i=A z1CgI1uW4Q&?s{D;2A({%G(ow=O|xcj#!?q2a$=(&_XIa#Jts(c0E=QR<@kf$+_qMw z==Gf~rznYfXRCGG$EhFMq3e5h(~tM=rXO*lE;rV722!NqbSjVut?#i0vjxu)PV$V3 z()6r~Kts+Ba$G`lf>UiRZ6>RAIk}VeU=RE)d#d4C-YJ@law2yKyw@27O>liYy1sJ{ z>A=cnKjINdJezf;7icvV(;D%R07gX{@5=#iDitS4wuk!*C)tJi&%ocD5F&(BxD@>RCk0IVhG+ZXwCB{}=lb-^f? zIt=e4*|5~qJyGhABe@PhI4bl}LaLpV(f#F~@y^+~?9WO-NkYc9ZDnEtgn$F7+FidQ z+PtZ-;mIS|wV-qhR0aqOoY$6y;-I7yb|1b(7UI>o8?V}}g(=&sGLW4~2Ov{yJCndJA&owVbltfa7`mv`43uR^rm6Jf&qCc)DC)M>Ft^XUNDYqd6_lhM z=5>lpSuTo~k;c^<+Ui7TbZ9$WeInhaFRGT-D*78uVAuUEM29li#WaL+;sDx8)GHLvjvYH%k*sm4jD60YiKoasA~Af*gF9Y9}9 z8QBu0G*KzhRdxu9`c3(9z*1WGdSZ3ihCypA8Un_QBkkGWuu28~gTa;*{Zfy2lgp)r zs-aGr$X}|2PqXXAB>z&ar*hcnhd{Kre-b`4$dM{nyy4BQ{D+aE&mHuxe7e5&Pe}gH zK4J5ged>(;Ztt#8RT3TJ{xUDtx|mF;W5buN>fEt}RVDD>>v4r{Kjz;m1R zRZAO5FX(2i3{LDNsr+=~YBJuyb`kkRSoiKO2aC?v3|R6Bu!8?rfrzRZW^`GL{+%!G z6)pLB{l&c_AfgYKm)-cQ0=uT~i=YD_ilhkEbh;Ku8862McbID*RbDyBpzh`6Rcr8W ztA)8Spn+qO!Oy>$#FpwodQr`XT=^;t`Vam7PLEd?BwV4*k}-nc35Ker7W(;$nG!{fz_N0S|**@N)m?7PBk7^*Cu3r&n{avb0`heTSM=g6iIuw zjL}syQChmxinF^eElIyi{qkwBO#L!z94%x`uf08}_`N6FdB^AZ0p4>{%{T75)mzHv zu(nK!YMUYl!BHcJv+qKswogT5$r@i1ig78mVat#GmJ1p^oGn&uE>wMBxoMpx68ZjXJ? zSY?_hDa*}Y8Wm}uUJVAEmhsbSV|6xpQl~F4t_drLC?G6DID`HdUP{|huh&%vl(zpY zA9_;@&I|kPzw-+jOWuf!WxMEXg+WfXED;SSV|~?5NeXUDZ%Sg@vDgi`h6tN*=M~~m zqwxpsNJ5TzE#|AOr!rti&mR6`w~e^8r2r}Emhk^w7SaBkKHRhm9(lx02uv`o1K{84 z++Jv+6VNy#vq|fW9>2_I(3wI<7rb%xOg`t&gc@$S{JA05-p)z z43YWv4T^bbP^PW_99mv`mniAqPsy*go?tsFT|g-J%oD7VPL@A{UY!if`WSDL_#B?1 z)|5~6@i~6Fxzhacq~AGvK!sEjW&07Aq)+AJlG+NM8PaN=k9bU=pV)+{I2V<`7`l$E zYfSskx5J`(yH3CT*lj(igs|AhlA8NN10)Q7h}v~}q{z70%aE)_2>7|8YD>ZEN{GPc zU!lF3F@h?)RTQ^l@T)X;t#+P;p0pI^@iXU(wJ>iwSNGoi5qjL(T0)WM^G|mQ=1uf` zqHPn~n-Xo}lVjA+8~`Juc5vClUOj zB5P4+qwePsqV0d=L1O%Ci5V8#lqAC58 zBma!8yh2@XS|tyi;RcY&j5-LOrx@0uNcT&I##)Uhdvk~^Z%Z~a=%R@xb!n9pI7hdE z=HwI&uKxS_JV=X4TuNF}R7Zh3nKFWMx<_MHhv_oJ6hnwn%m&$bkFX>CXONo!ZbE7) zFZfYP4T+r~v&c&_gC%V6M#EI@m0@Z>3E{&SCBY_c_mKDX8f7{f4Q`@NoLe)N z79dKfknF*B3vCz-!xFY#mzkgle&U#doW~7_9VFzLD~EiIn^5P8ibN@KetxOWZ#Z%r zdSC=lkzsqF5xy6D2Txu+-&YiMJewbS(>0S7F014yAylKdVo5KIm1);7PC8$#j26WB zJ+o|Oj~Y(GJ#9AjEP3}9`rgrtUM#r}mp4jYEc4-tqrI2A42d6ozx#3@2s0U60E7?Q zFZTZV^2xW~9fVJfTa)4z(i#e{gkLqK5PI;-!(^A^eb=^ki^Gu~N{S1xb>Gb)j<3Gr zy6pt#iiK*)Pa|jY20y9D6;i5j^o?k895V%$QUBBP*0G^a8gJVR`<$mOT9M?X>~pi% z8u=Es!%>l%4$srr^cPj5GC^A0Hsuw}xTm49RAUp2q#^J@`b217iLD%+-fsvLrz$4c zw_3M9x52RHd0I)xx?d|&w6i8>t+kZV1N}mK3|ykcyVjzR0o-b`d${#RBiU3iUI(L; zK0e$Qsi+R4@j+seO^MKa=rB4+qVP9E!3u@G+jB~Sto@|n$6y}pjqt?RBm*5A)q1_+ zd^)|nzp()efvdB}{n6C~4oe&PcN^w3Sl8P1O&RbulOGPeCGy zi-|?SK>+M;_wtCSM3+QM;QKbd-@pF|a*V4H^9*B_L5Yy+pB8ohMDzbq5ey3-ON_X{ zCUO8Ca`v#8m@drXF*#Nj5Ag~QM9vjhaM*wafng-%pj@W3YY8!3-rS0;f=E#;D6%m2 zu}G>uPTVFQ;)Md1t9WrjAx*laC-4P zf^J#03B1)Ny2n@qsRND=0c;mO{bKy3lw3^0ewBNW16zb>0xLr`cBQU_-mWHUz=%b} z6{P?MY5QP&lkZe6|t6%@;^~IpDg!cz>*f;bxMg{Wvpz-S9aqEi) zZx|e&bOz8V4;mkEwR!QM-ymN=s*{J6*^k2T9#MR)#O?LfuYd4yFQ)zJ05D*Xzrpdv zKIKy5h#sYu(fO-(mZMfuFOsq3@}H3K*!0EXaSusJ*lWRlh6{LX%3;SEujwP*ZdXH8 z$k-8$P4UZ;$56BL)f@ul0gMQ)xfIt}yabT`4qDpjL0Mv&>gqDIRBW&%p~fWK6cO#5$;E9d7Okqk`hcJpi$X?Y6F$^6G3jZrD{3r@P z4_fSm44l6)dy$+N9nuv7kFWxAErVKV?tj1>sJTKg<`6T1fTexmIe% z0>^s~nbRSA35?9JZ{d*%aPlBWtLl8g#`}f1V+9fir&pc9V{G&7jRkfx-mu6a$wb=_ zIJR()hJ-8n6NKhvZ-XEo8aI07?Rg&m_Z_tM!;|MbFL(DFY!c+b+#tg2mVt4HlaSXU z36ky4v)Svql{K@y?Jeju(qkGDl5amB@lp#y5112l7`KJsp@EcUF6wR@6+^L%Y9IHI zXehdl0%rqYm{RnW3^~2t_xfk&(+7>s{3~~)smJ=D(Yh|WpMBQSW6YDD8ux6FeO%Be zFm7UNWTY(%$-M7I6D&9!!30lf(dE6+$KS)U%RJQZ76@2!HVM9qo}%Pv+B?!O^xZUT ztd^4*34n2sE7L$Lm2AD^2YS(-C;trz|Aa*YzT(#RPjJs&dJ=C|M6{iOl0TSYtRkU= z>9d8bTfUQN1E51}089FX9e;8YOsMgfPp&r~ezhty64PCuqmw!s2=fKk`lx$xGC4Z# zLFCm=36~I#@Bk^ZQ6`O1jr_5ZQd07hY+L{8{%AbLUPBzd#4V&3Bgy;47+^=DB>noH zM9BmYDC}Y-LwJH4bCI$qK)fx@$qQEA)bk>)fkDzxiz#BR=9c*eqHAXUxAPKAah83A z$555u>%P3=qVv-Tng+sk(I4{Hy(maDIf)@Mc`><+?u81s_lN41YRI&0UI|L$=mOMx z(AbL2(BS35o+?@ohG4nd**z>FCS-&*#9>w8i=g0c4dz$LnlQXe&3l0#M{Coktb~ek zd1LD~!vl$MhtG%cSO3kM`l|soCQHtRQxwYMP$#u;V3vXgo!x%K>x9VDL0i+6o9LG#t7Sm81v65N=dK5n!6`ky<@xw6VYL%nT)pR6E zH*7MDuhbtZ0hfJ-rEFN}FVHyXAZA6lrc7F=s|%%ggwLt{Ex{K7MG?R>rIUcrqmz%) ze$*nV!-;fztJWHw*68wCH3&(?;Iq!-olm9>$X`T3^Ufplm)Ep(FRwKhSsI4o?MlCipsD% zr|@!?WSUBM%rJ`ctR4Y7$+z}JLJz?wsSg8SELaS{dqbY=fH;fpyJIL*d-dD9F&JR7 zsTg_AzQhgBCpbkIP1gbm{-L56mC)t1+j)o3O9{B-?k^@C+c5@L<6Sc{MW7b=PB9ta zR1|muOxs%`>dn%#W8P2oYqFYQapXMS&0=6aS&RSk#=3uxC^yL$Bd{_i9~a{`>XqW$ z0^U9a;J;0zJFrX)KYK*{=>6ofeet$FM(u^#Q`~2*5o-2Wka8B7722&oKJa| z%ky^OWBYiR%yHhtq;6s=D-Qq56L^wIjODWq)-JE3!?2wF!*m}uO*kN~Mvzaktfe{J z!fBD5JWd!iY)XEo^4k3~Ub5EspT>IQC=4uc4xEn(Y&?2A8R`Ar05gQ%i7r4<00`NG z6bl$z`G@wXhKIDp=YS?$W0P+(r4Z&XO2_g*0LoCni@$E#zHkb5aLHG;9x`~c?giCCg230?v=zo zpnF&H=u#v97-BFG33N;!3+zni6cTP?i1z6B9o-$D6p-i>rDWfa(>T!hlP(Tx^ojkk z=+pB0nw4Ul_5T9J{d-MOW*T&Ic380~>1028Aa$6}hF46?D|WaI5^GQTmPe?bog71x=-Wn-F@;(aOrIC^%v#-QlB#%2nZ79lX}q zSFC8uR~m%u!ehpDM*$2!g7yVSZX(gam7Nqq4}pe=Ya+lLE;+_k5;)>$VZ1aSX;8e7 z{3(sTa%7%R$zSxa({9?^aU|8VvVes8qz#TN;}Z`(FH#H9rXxsRs=7yPsY-r;6@fnM z9-xu_5N%AN{k0qk9g_7z=`DY?_jm!a!P_M;r-MXjOXHWsaoxMK!x3~VuLX3=-%62P zjG%B%hz^=ZD(--@b@8%?eTQRrQ3_qC&`wcJXQL^&Vo1K?;&Of5r6xut{)aUBL=Oa_ z|66>|EB+$wyf2F*I~J?5G3Q<`h+@wcS7Q*G%m zZKRgjywX@yS0HDH*=968=^rCxjHxpir5d%ZAdrmIZ_zkTgY8|2+JI7v!Qzi=Hsz2T z298z98LvXTRgQSF4IG?iBH*eaWywVY`Uvzoi-UqK91cf;!+(P6G=UCK6J3kS_ElXiLum&ueCDI%QsggeP&1kEC?OP{dxYdQTdd5~ZJHH^|;7)OT{ z-;~kQwgL->3W;#V;r)7J4g#o}#$t#Yc_{MH*Bd@_jjfYb|W1if5GB8hvROHH}DULV(#Iy?#wo{2KpYjzbCkVMPip z!KqGtfm=;2fxk10;A~Geb8Bs8kx%ip*=)Dgq!dbjdaHTswpGBT0lc(GXzBc4iTrSU zj=H1E4?K^vsgX2#{LFAteR^whAJbi#2Jj`4-zEE2Kfa#+z1KY-VIFn)0g432d?LvM z6CBi73stme|CDEqVokPxz5Z!)|GV9%PY?dNx4X7xis1DWZ*eQe^Mkdbds(zjUjm4> zT!$8O6zggNkz(VYKm_4hY=7QZ=Qyncotjs`uog6?Xs4z$>Iru77JU?o;Q-VUC729E zQV7k$=bQ9ipck#n5{MZ+*3#AfqI02j48Qb3Xn>)Fq^;} z^CyHpbUA@F8Mynnc3m?0myirQn$zQfhaQ4<(n0O(cY}U=ERz3JORf0zdoL$cARAl{w=w5AA4L!JSem7j3^9B-tqDaa+W}l3dpE z5kK(&K;o5E^eI64GPblJCE-j)sdf(@qlnKqhV%E|u^53+hV;svy%+mWeq1k}^@rd7 zfGs*mT-^R}_P);p_dtd}YCj%$X8RM_X{U_}c{l~5k=9*#@>2t_%vDG(BWc)mO%1=@ zsUa6LeQZWU;CUw$xtR(AYpNYckZhw;hl(YRT%*V^&dYCzjt$8l_`a;xpb~W1MY!7a zR)3;BHVq;REvjq0MA&^O037+aJ}$_1C&-88zCfyio7KG3?SvH2kkjCOM`v7TXi9*C zPLdjgf(wZos3s4Nv;x}Rzd@O)T6r+xi|Zc8Ajk6*zYZy#E@y=hOqd6;P@GB?&?&w4mGrX4ov zs*CZ7EGh70X*k((-It_@>9v@AHh;+e^iMHEK&qF_h3uVQ;Q2tjlc=yA1u(IecVTJk z;}!4l!it-0AoMkHeUi&e;X$+S;Hlte;2tI%_?$OySnR!g@qc!=5B7`u57fuWJc@`_ zZj(~O8;ldAn68I5Z{XW|-@SPL&-(?offvGlLM-@y-mbfxRvzW<*MBvG1IC-@0I=u5 zOODv1mglxC)xTs6D$~3A{<`%qg9-|imlODD_iQuui&D$l@NtJDNy!d?8Zw2((kl75+t zdvqz5skHeRZuA9q=RuXANt~Svh01x)5{F=HV12Ip7B}c1@<+HN&40MQAiMC+3L97( zZFw#iF;)-;3zkrP=QvkL>l^{B=7Jd1#DG!>(7#K+`#8?aNuwHa^GK(^J{8Te5MP;S z9O;fzw#HhKXE0y%5i~AJbag;T<^(JUp5u3iKX<>E7m7%zP;%{6=PMitm+Tjhf>C?@ zI`Wk`iE*;R?u*+l7-3#vyPA3F2;50IKqT&EhC=(a3BWr~Uam2Ta#R3xoyzi6Hmz>~ zEyoPVKs;8l`T51Gm)pBycRvC}%=MCu*FbhQo+$cjFb&ZmdI%SAnJ3l-4Wl4_gwXr` zWv{!2UoW2R@9pBo8Y@y1S#1e8*(cB=xF-r}b6X>J5PkayQL}&=O%1nb?^l?oSf|+&LXtR8F^u($ov_?1V z+wJeHH*^vC^XU#QI+9<3gMF~w1LDeT2)dOWofuaHtS8U^vHff()`wU4V*lXLt0zx) z@P6Mv*dRG(Bo&c%jX({06r+HRv>YNPq*EA>`SonnQ0e2jVfL^(p=(OS(e*;f7zNjh zJ#iv$a|LY%m2VQ?9AD6{3~cBmPXoX#GCoBSBaaOSnb3Zwr9rvgHP{s8v#76+3Rj$a zgV|qi1kx~CE5;WTrj1Fl@vkaqFXf0Qn#W$bm^$^@AsiDoixW+Glu8>rr)&#D!rhdN zPTN@{d_&tJD|KqFIjk*PlUFKJ*zV(~qtOHW=fuYZb4W(*p(3$aGtq{5U9nCf^lxR` zC#Akq{ws6!XjF^C_vf5w(bQK`lk#h*eNRE zZBbY`!27gmU!8r5B9T})%}H6rCQslc1URW}PJg4o0>&x(WbC!7qc92P`cLLIF=QO0 zD@Zw}(3txuLCfgk5nNHIJhKZWJiuq4$|30xGxIM1jeM!K9zX|rIjZ4J626G1D5 zZrFMXiZiMJWoP&Cljm^wisCa3`N(8JR(rrB@Zc@I$C5N91qoh{a86O+s==qPc36Ez z?If#5ASBO(VyY+(kiL@Sx=3J_qF>aLyLQ#DR-!9%MXD9S4$$_q_osSbJ{YZ+S3|K` z(4Q@Ji@KJCW32ajDg@OKjVEW3sIH*BJYq0hkd;N*;cFaR7F`p$WHzYdPEzwlA~K^FQ;kka-_!ntTR^eJ>H zixL-wG-HiWx3L*{B-j)Od437cM^r@AovJKm99(OYbde{lDOR_>SUDtt=Poq5x{f^RbgO>zh9-8 zrbYWDmM|k>ByndiO|`(r$l1KNmK6v#_5lz~UPIg<>~gXb>s+4A$;qy>4YxacW|c~# z*)HPqpa9wE7!@ss3Sv9~evwSmDaKrjtf>i+LVTh%?oxj1Yf>MvwrXB!CvFOvdBEL| z2$hyn*lhsMEzc;jWlQGCsz5XFK%&rq0{WTl=B2<+-H9%f^^W@M!7>_5?$ zni(OQ&#Vn@&DLExp|XcHAu^n3#tE`9d|8 z_%@Qjk}TY4D_|rWgWhCHPtrk2O#_eEYZsd((P4>RaU#~lCKxGiAwd-Invj98Rz%DR zUqj!a&dM>8rrrj#q&&paf$X2zutdJlz7IX(MX_867;IFgqi5Z7vo)F71E$GTnN8Qv zdE3ul9PB-O2{ze%nuMu1Ee2TD)6kWyB*vHS&qZ{;W_cyDZpE5?kR!|z?)v75i+0|k z@u?XrdZH-u+_IRIq!AM??fKo}^(KPy)A;o74U~J#X8z9L0BES;0lex#+FsAL4f{*mLKKmlic z(TGqHWd2YD{U^hl52lU~7>uvUrd~9rdYoc77FtOkhkz}7F zxdi`FSq(}<^jTa8p_0^Dhiv^5wdl$>8F87FOA0f`_1cUw6_GS}}Jvtgod3&rZxOnYk zF7dXY@J!@%AS~WnJ*(PAP!gs%D7a9WjK^g*&7zV6 zq(vC@SwwGJw_ls^S;R+o1sK-I4I~lS?V!LD0K3+am&VI=ki4@CKyBleDIBd}5CSo7 zGAsepwbIap4{n28A?a|2&-VwT_lUabedit6H}z*{pTpZ-!Ahw%?;ZXB&%8Q6_=AGsZT^Rp#b;{t8f`uDfe?y5XW6eiI5n8rbI_Y7rG*{ zs1D5+FN#m5XVJY@l+hpfz0!i@i#H8CE(mF#39l8q5~BvPU%>Y!vQN}gIH>m zH>zoyAaBTt(sB3QVd9a@bYwnx|R?DfbwSQPPE95{{B!U=9yiN4w8sN>Dn{L}UAyscsBM zgVEWj*ub%i_doqRn$t-ueni0~YDpVVX_};R>()AMWKdp3a0NLQN%M{{8~w8+C~;u^ zfW#ml9MjD0zk+^oXb}%6t7q41YG2b~`fbyQ0zwek-pq8OSZWTc^ExqCYs^+aG0P3} z{@QdXM#XCo*W3u#Q@=)k{SGLoa$O8K1SkHqd0FhqC&gZG+)_Wtw$#AOx$I+VEMZWI zL_m?30!6yLo353rca+oBVt+mrpKS`$rKA*6jYDHbw(;!$}71s52p}w!M z=eTGa|IV-acYaml_1ug$=$Wz_kN42Nn@q1x`Xj`PVsJOUa{;tBEA}@=8(41z^C=25 zem5TC=l0$!Y+S$SU&AoU%V+X)&jTI!sC#bZBMytavxQgJU)*yb_{kS{w){mXbMLOd zy1sW8W%l;(_zV2Riyx0i*LUu15*7X|@EdP`iF($1!1F3}O+r@rIGfMT8U4hwDfszF z?_@5%E*kG9TbrAo`$u10VO%3$83~lo)e+`Lcwz00zE4enqb4%1ZpI|r0F5IJvT+Kr z9lZ6Cf+tel@$8U81!$76?ODT6=bZ`GNJ-2HOW*lscWf2)^*?+FTmQY`@GdJ4>V*%I zgcfM|jnNYKykerQO<9E1^r^=76Py{q61GAVi^g*{a2u%GxC#-#nc1;)17PL|-UmKG zRx&_Jp}Vck+hn<9G03q_)`R>N{dl=ituiX%Ml}TJ!NAHVy85GChdK!IyFD3qk3wq?Lc>4?CFN@X4W2iA)4WVse5GC?6U|};g*Lk#2MpBa47g@c zRbYk6T+zk<;g7du-3TQD!w9hzl0@DfVA+Ko_KLDP{67LqKR5`3$U9m)1E3o&>=Qar zE_8#NQA*yrIA)9j4+!CA0dBkm<}mHYZ-Qa)e4Q#1obbJ+^AE7#t=?c7aH?w`sVw_s zSQC1JC9asyF|Wk1%Uj;(mB2Z_V|+j5JtB+BO*<3Zy{ii|@TwgSW z%bSNl(Iddd+7+@Bd7GkEl$HK>UDlfawauQ)E^<8|f_z>E1cw0jhh(TlwL6*Dkba%p z&Sn3EV^#}%a&n4K)I{~<*Xe0fV#x^vkrE-~WeIltQf@W*GT=2d5uf?jzmZuzyMX zSMh|u?@w^OHtx<8ha7Gu%gwvk>RG=#9!*B4)1q1x4q<-T)o5ZZ#RlJFxoh+u=RE&8 zPno)yba8P-<(#R>u!qD8oIZvBmun=h$TVPbmz~^p_^~~SU4Z4l@p?M(p&cQqk%2Po z;hY?1j_BD+PQV1P=xuPCTY{UO2bG9nM4-zVMcTeYcZ|r_DuJnfoC)BFnjXF2lw&q1 zvl$V}yDreUPROvRuBz#o02~d-gb_3|@E9*|z(*qc9!|INVVG6HzK8Ov4%MsoZ4|FYbs?avmNIrcUJa6|W}x3^2%*{t9k`Gzlr(nW){)27YM1b*e`6`bY8 ztJ!R#Dr$rNw*Xnf|CVweC-4a4JHZ%n;neugHE+h8chTX@f8?||TEYLFX>;2CA7sWo zBIeC0XC7IoEJlgz-0vie${aGy_{{;BH{l;=L-9asJ^{bI862x4E1?n5Kl>bfmalMU z^KY9n@2*iQx2aU*vWv%Kqzb>s#eZ}$?zU%W8~s3SQ(6D^Fc8B5VUv?7OuX~qus6P0 zN<&B0?+z`-q03PwjZvj{v<~~&p5H2tvG)Y0)$`^q9)7xlB0X%}e^9Kx9={&0t^iQm zu&|W;bmM@lcwOs}jD1@|X133)c;UX$ws;Jkw_ZG!^Gyra)PHMlZdz6w0#TCGWt{8A zMs|3a-jt5TO*eeYC@*QmU~fz$mq$sn!g^943mWWM3;}yj3@o*=#J?@`XiuY?&Bo8S zCcj`co*`~%YyYpjMe3YG32|ag-DLQgzZ5rpcXidbozT0(69b>*Y<)Wz9d~Rq0*;B* z$9-(ug#X%f0&b@_7ZEA&jej!|_9d@J;@@#Y6T4os0E#i&T3=P5&w+CL7B}HZ!c@O# zaX$xE(+EmM9L0edu+NY~yZRw}+q530_Xi2WSg}Yj{uq!|BIGc+ta}qivSncFYO)GE zmulpIHuhmZ#0b6XZDYNel=~kc9$Ig6xC8x?AI>o$KSaofVQ_$txV5Cs<^GQZ2Td%C z88@Qsu5Lfq-84FqSR@C7<+S~Hn7EPa$?&EiBeR*LRQ0Mxw}I*MYBf|TX{oS2CFGejtT(+HQyl1-pt7b(wbm6fIY7?8>P{`9=rKnvx}p8cz1 zUIomXv~-at2j{t|aZhP2TW17iKt`k@sFi{+L*H^zLwNwj#zP|dhH`v2RK%l{bi4y#zEWV0z zdFUIXt%P<=Mmc1@e$W+O$UAEJa86%c%0igcb5gU1!a>paMmW}4+t3B(_Wnu=Qp){l zi}s$q5Xms{2Ri6*xQ7#4dG)duVXZO~rLHYH=YCFuevaK%ugjXcAZiASSJ{77h44?Ju`JN17pj!?A`d z`Zv3*0s!~B=a4vel9qsXn{H{**uo!mVj|Ap!Wa=}cU#;*c~`B|sPMILh&|uI^S<4N zq+|j0A70|nXI)5t0-u6ljSBoYnIJ)a@Zlf54=~f*r`a3Rv#2lSUzAaaln9!-+_JG- zoev#$hy4PxcV4VL8lx5Rehz#`9oy|e9~yhVJMLdjS>XTZ&xop(`|jY`)4k4F?`h`) zwrR2Cx8wdvrPzT?19VJS=*jNqn`xmz=fi&gOsF9&$!}nGPkcy5KCn5+^?ZbbxP~X4 z@yXNv@whYY_vmal0t9MH?Yv%M?!4y&QIS#9;jP}p(*s>@5eiC@G4lp9a4E??l_ zm~M(bn3n8IuMS!|olfMZ`TNdF48zyzQYwo>-|0_5LI5_+Utw{e$FkXg}wIfW$&~%hA*YJ58E6MiI&Oy&M5|0OLzak0s)`&97f$; z1bb>*xv4AyY= z>A-eU6^$au^c{|ZCo_F!l=^YPR&P!hr+P>b=CBSHbjpk?h!SR7a;3Fjl}I(b+$m7V1iOB7h*T_5ilCsYYk zQmnw997t*aH6C$9=;UD^9L9If**@Tncrs;@xG9km? zi|+qFPxpIcxCir!(gaL~bj)v)W#Z8XZi)4R}lna@!0-CFs+-+NEM_9$8? z0kDHN(#~gXtD~hHIv|P!34?(SFeYF6dHlD{`toL?Zi^Y0oL|^{%`eMM-wV1o}puzA3_Vjc|Cs2V8xH*lvI1nXcy5 zkPa}X+UNDHAx&v+HSAjSse0NP(jc^&#rVrym(BZRBZ@3Ni0eteBl`}lRg=*@dr&j; zql2ct95jf|Id=Rg(Yo>rrX2tx6O9X;*N05fg`l#n{pRnSo=q{#99NlJ#{GBFZw$lbz0BvgI2#@U8}S5{V^^N38)n3_Z6XFh# zdy$?%UP?NNUFK{tRqC-SD=fbxP`?y_c-s&mR?d9EB;MTr%ig;HS9YA|fkzbe8q2mE zE4CcJEI!x?&p0`a&2t_a)<-g_|vn4(Q@w2tESL(a-s zI;AACEmyP~WmnG171xGY)321A)mB^^WmnmFD~coAO5!A}l*>`d+0uUB|LD_w?zwo7 zR7v(4w^TH#J&G$Qy|T z1z_ZzDrFAuX+fHnf`i&y(TZmZieB&8;$6F;3CV6J@ zU5q#)-j1UI&rLw<<RPJK;mU3afz~l$?;+e2@f-je17imd)W2hEu8p)dXUw25YC_st|8g$uZigF0AE$qR8LY;ppe%)@vj9w#{7B{A~RX!x%VOsL70pzWe^b-`l z-3E`I!I|lLMC;16>LL`fh*g<1kdMEsM;>#L1e%8RTvRYx9fW8}ih()MqJR%0vCq#GaLG29^K59?+W{{{40dHgg)KO!eumj6gr3El zET+p9)^^i+aM%P9#^$PXgi)Z}P<;g>4lRCLtoodK-IPI+@RO9@m?*E%C8p?ucT}h# z52aD+V~tU@&pj$2x0+s1)>%@;n6_!J0dRmfBFZpy#K3l19NNJP19ECCv^Fl4I&>-h zRZ1#%^U2bLQ-I3=gp0?hyX)i)C`P&x&t!rxE|C^IIa13|0fmwdV3|ggamS)=LnEv+ zACjqt_GCb#1yy*691 zD@Rxn_G05c7X+{bHf(!T6`&zj&0d~gTWFE33(g(SMORf!A2C>*D~M39daT03Rf1P@Cukefu!n~NK>c4M^@n|yG5zC@#v=5g4S3`t(fRAY2! z8DhS0)~Kiudkr;XDFIfAd}Y_|1G#MpfL7zt@ybbKD+!U98&#pvbNZP-kzoOtP*SO^ z0UI2zF%svM!u_$G+VovM5TB!$9)f3~Y0!8nsw$eu~qC<|OF4aikowb$_mA$GBgQvVO$3HzH?gq_F zaTnB^fUyhhp%&K`+vD}v(w5MeBFLv2uc_EVWBt-Thlp$bGFQXxrxE6rYYuz!4Oj(f zHBV3|>_8W}==1C>#^ zKd4P^a&+$h#4s6LRZJ&kih_-;*x#uUsO_27^muF!r%2>AsuDpHP83(fsHp}x2*wnZ z#&wRb)f>%f_!M*X61)<@d;?v5UDc7Oz(gAX)Ix=O=l~DNs2*N$o z4^PPUse})*0^?VpbGf2tvR8@9Q`+egcXKv;!@WvW(W$83<2G2RB{af8o^;qjtY>RUXFiJUmEymBQb`F$(b4Utu#@t7V zuO}~3h9p}FVGs+0PML0B<5qoTh+0lG;m|)P?q=R@XCPq(kD^}#jxbwWz+qeV!hAWf zHexz{HFyAl+PU{^5aDr;<;c)H9V|q455g=eB9iu(wZ<3O!Z%%-uCB1D$4b}`zEARo zq6mRTc|xX0xJtY)_3$2};n{idi`m|a)UTu&b75IiOH*x%w*}dyWqPe2Cz-JmpZqp$ zOX;A)-qAE9QWB7_yNbgVyrCG1$pHk1{HiZ+0q9%w67xy5-I_NLhCNyA3|bH!738et z%Be?`Jh(j;gQon0yhyy`7wcllhf(lPBq7ZkE&gvstII(Mq7$zY8ko>ZB9Re9Cxj-$ zC52Q4hzoBRdC&jIG&aj zESCo=3#k>AqRP-v%`1x@(Fes^&H-cMiXTym?w}^skY#8k<_QpEU`K@|9Nj1!OsIQ^ zQmNf_0gF0iB9+p6NBJ}l>4T13Z0SrP;?4Et9GTrx5jP;GOGN2X4@W*OSKwC^hrjxo zPeOqNTU`MIZYO9JcLwtBi4$0AIKbi(P!8y8l=Ca4O${Anbt=Kbv^Qeg&M9DI>Fti? z5Zq{aE|IVtF%CjdqN-H50%%`1qjJ*YxK$iz$XMKQ!kGtKcFbhHGIEPf#iGdN%9way zX_82@^4Pse0mW$5s=AQ@qr?Z&tYR(gRgj2{m6%2l&>ciH#=Y+NRiL8r@Fn9Kmxi(A zDvb_i&KjZ3bg$=6gkt1~0Qn`K>ng$QmdwBeKr!SBYdktU<~d)`aZ*)bPt8J;M$?A7 zH&mY?2I=Qn*j5_c*R(KkE?T z4$02`3|DhyN?kDzrNrkG?5iky9!OZ4wM!4otOJH-6i+GD-7YeV#dUhkB1Nfn3l+T=dhz&5>w5i>8g zviM0_s?&9aYuAY!PGCeCLgi7MQvw9+hHU0$hwNdp!9T(0S$xwtZXcRyNxAS^%PU?gy%_CSN?2{4z zP?~7@$T<`)02Ml6R`=QR9=j zx;=NK#MeyXuv@LMDZS&k;mkfH%u!U|9YyplJ!vA(RWgb!4ABK(GqwzqwD=p3Glt7>Z_hsMgBU5E7L0%~0J>ve z_b=smT%0OHjL7E%S7FCD0rNcSN^5x%>wS3y(-iPo+FCx(#IAzlBrJbG{A4I{&-WPZ zzZQmpT+E1y;$nAzcmJ~AP~?>qh9l;U+KQVa5DMeb=op@{9c(ntM+@s#*!5Ujt%V^d zrAXaIkaLG-(yz5~`HGK;R`4}B9^f409xgq&;IL+E&AM;S+gozNgc^Y0JFiw7 zm4UZx?LbV#*aRgqv53w{2nNUVm#$tPAGs8cGvQV#J)vtgGElAZhRoWkpQI+S%(b!n@fb4v<*O1W?8C)LuOZ$C^oSzX-{;Su^a$7n?6|I^YrMp_YEHR zYt`4aTXBg=k5DCbB>7G|3Vl|1OtsSo3`1_}>`Icjoiab!B`L5>yDow*kw!w*vu=9W8qSL4YpHHN9La%iuk{$!nm0?Ba6Irv%xE7XZn{0ca#dGV8yEu_g z`g18cz6C&wy3+~QuXW0dkvaVN4Kd|b_6Gze7!0N>+=By4s!~;}B1Jl;Cdx)^EluE9 zSwION8qZ&pOscsl4<~Di1EY{2PT#^e@vsuITl`SiI+mCRbq-<|1RTBk*IGFIII(b+ z)B|bZd+y&kImF#ky>N%@{hnvH8|}xiY0AB0)L$y9W(Z9m!Irq4l=6gY2#{(ryIGnRI-%l*?kRPM%a0PeB`)vF?gcidQg*B&;i>} z0-zc;;~Cl&hB07Ypss*SRwNiWj2%Zqp8+X}4i3^QHap3SS)cMeT}sObz>HN$?NN z9lVS!;~G3v#)Uw;oF=Nt^?JJtsE|3R{hgm-!f#`fi*j3|eSy+dV+qO2j60>wK(8-| z21_IQm$#cZ0Jz`7Dv77`kw;}^lrI$r%4fhvrNf&}?W{?XMK)ENbv`-0; z56zcpEeo8Ux#@_NUrl6%uR>1}#z+lZuAd;djTpb~*Btq1S>@n>=y=SXOZi+7hxg6? zr(FyvN0VT!1bhx52m~wVUJ^k;8Ynw^OC^un?L-{N&4Q`Pww!F2_e*I^P9_#O{y^&m z_&#hwey#b z?cj8s0NQ-pBRa04U^&00jGkW(wP%9}tb+k8r=La1&?a~y8ihu?p!>SWNC!Y5owW*N z>eS9%aT-XYU9Yx{S_qj^wblou(lb=wtqjjx$9+Mxy&-clLL3oKafsmx#RI{Lx8B`4 z^Hjh9tu&;LtCkVsC=q8*o;=rYoj5!;`*7a0>uC_kL|WmzJQo)%W(TjG}r5p|qf2e&#Pi^~W= z6ZmXxffwrjadEsWzn}}bI=|xbacry%No-L5jS8vaW+e+npSwSe;IQ7;qhUt`m;KncyIqfH@%SF&Ttc_>hyH(8T4a zuty^fJ$yDzs!g0?xY99zrPIU@D)wI71gbkEw1jl%!NPGaPTK?x&&3nur>b|Wt+wXz z_-=RWo$C)5PX_pbaVlqx^(`gB>g%iqH6vp@E!RiYc%5z@)KgG-*siAcMCHftmYrAY zQ_QJ#s3*VSO2jt(9&Xk1Yy#%5&P{w6DrZ6BNG9M9mU%g~@*>1$^LrPoc?C;?9jrr-s*D9dQoWye|w+V_sOkD zLeej5!6luwwtE$kAg}D=M!n;O{T6MI6lx)7CWA5AJb}y&7xFTMbig*m-h1n1+ z4fzF@j*c!f$t{+^s*3|@3^A#&1QK~ZgWijKUiPrvUOjPMZ+Rfh2-AyDR=7?BsyU)JI_e$}Y+ez} z>#5-O2%aCm-fnMQ7#L{bEceaDR{v&gx!GtnR@+$6*c@2l?O{}U1~yy5ItI9i*u#N} zV6f-dwCn;*9UL8QHjbLq?o}MkwG~sZ)ak1NrmbfYU+Ssc?tP#y(Zi43?j^Ih-HWhC z5Hk79Mkb^!w(~8=E?TWyr=HUFedQaj;CEE-A3AIHFRavXGYAUKU-w7JPTT{6$Je6z z61lZA`6v(oV59W;gmUei>UET$1NZEFuv30=MVOw+b*h(fNh(ODpo~oncT@-&&=T=% z#MP;nxuS#PgrgW48~1Vyd6arlSe4F1uhCfM!VLF#3tFi$Pk26@bDaw7JwPCVuC;@4 zcYHOyT=FK)l+uDU#u_aX!H$b@J{h;b?r@y~)09HuTig^WyIu3CBxx0=qgCOo`HV=4 z@X-wKSfVZx5_lO@^%z~|Aoa?Vr`V$NUryX*QXjWFq~u+`p~IoH4phO^>*Yq1Qzv^< zW^}Po8U}#Y^UEF8_=lYR1QG~CgO**8C*)lHaE0aNbEi)0QQvu#6=sBnNS9I?637xZ zB8k>Wj)Ff0CRs|TI_N<{Z;rkHfj%b#fk4O|^g^E^45pw(8<>byDZN0n`pX1J*pQK&H{MOrkwUj_m}YT1UKI}?f15J;$e^BqcwOG^ic5?T)7<4V{Q zrY%fj3LsG@7~k$EM#4hfSbCvTGM}*fZ_k0obkI%|_gNjL7sy-eMjEw$KZ?~Kx+7&n zww8FO7cc1ELqY;n3h0G9nBqBnaa>LYtYGqxhV!U96Ik<`XJ#Pl9lL!ye2JwV#E135 zeu%i6jXD|6R;wqTLZdr4s*! z3NI|pODIwYK#=&U4sCXxWDG!1juAQs+NaArIhOKr6csZb`53f-+AWWLb+_!QgrXdrsG?1f(-Y;D z#0d}-($W?t)(=lgAX}rk8!=PJuGkz5I@^*PmZg+p9EwW(=fa2@nYB{`sCURV{s2q4 zXLq*BLohD`JvX=77YE|Fc!=UdTrZ!!IFP&}VS*8QS#<#+IPjO(>{*lj@uxnbKbxM3O9a6%T*3<2qGKU`X+a10M>im103X6 zl9%9io!ECoQYz?^d_^t6rH*TXl$pA(*rTO#euG{~>Lag=opq<8DeFXz{4QZ>CsGGl zPJSSj_nEAmIfRu*c^Baa0grWQdc4`}o+vxR~3xyAS zGPq>}ZX}TMo8664npFi?(-9%IlD;^L$yEv<#g!V@LVAQ*;D|b}vKzZO+A=m1u@j!@ zxYe&T0}MsXfL60zB79ECib*p-l2FloCr&^;uWaIy#MRx(F0PfT2nE7;%6$#6Kdh{1 ze0#C=3`VDg^}I?0*ReKMIqN`aC~&tbOVEq&ifc7awFifTX}MBs_eUfH6j&}Vt+YUc zH*iFEIO3wR#oI7ihyVjTUTNWC&ZvLNF0G!tK$EH~#6alqRUnc|SJ1{o|4fjNV&ggg#uBZ0iEdM+$U^>y-=hRG1%Ct}t^Yh9o5= zb_q-KCMyfWzcD?NK}tI+z;qfE#*S;uP858==u>{`BtQ|l_jy!#T3e5`0y=S^xV)@- zoZUhftU%m_6R1yRViEJ{l3%Nvs8uaNRU?}a84kPi51^16kyMmTicV;zYUX;(qbX=% z+nySSxS1fyFVaXF(|g>Ss3Zt4f{o1T3Y+8#FL{QoaGHZP*~hO?g9-;oBU_vT41JUp z#~pVo2K}{-#?sOH7HPeFIJn;PQzQj?L`g34u?_6S*H=>EDcg!J}P-+ZrcQVIo{=Zalm7d&SmX5 zao7Tzrc^mw#vlQtk3&8JGM&sne4!2FR^PH%AKfm9D=Tq4O*pC;Gpc7~7toR0ReAn8 zHcp50;rTVZ*he4;92j}@GruC(sHseZjL~7s#gEvCZe*=}OIsx8H$Nr9X910Bt2Tib za9&UF&4F*Im4GwygQd39P@7stE?v6ad$e^MdxXkrYKYF#5}8g-F;2DZ;^&@`T8me{ zW1ZNj!s~Jabk7)6FDwG<64ybmfrcMjeU3=IXU@|5pt09_XHLEd7%1@@FtO5uJFX?R z9E)t1L4tRLmFQHYT0(R>=Xh^_f470e5!Jm;12C5P{$-HgBdNUW6iz0v)WqqMWC|wS z&e-1L%5u^PA!aWlAK3dQTCN>=SZxW#?2HA^q@3A-FtUOikmPs|mG7S8ww+N)xilDZ zB&?}&JFimmIHLD>sw4YNy*>5=1x{j+YFBpK6w2L6iAz5e56Mr3^q%IdyK@Q#7P=IJ z*PhXc)3CVsDL-6FK`wl-U#(jvJU>PH=%3x3%f>tZ5J9@yPI!vP0X~+sM4!Y3$_Zfw zcyX)p>axRHUkt<;akq7(_eU8iqDXHf%a4~$l-KT8EqAtIsJ)T5fiGR!AXhFQu>LYm zGCVOmx4k90zglOa2y%k`=zEbsiVCx^MJtyIIj>%1(V_u#V1&zH5IUD-TC(sk&*k1- z(O}})U$~9eLD6dgLk(G~UvL8k z1|=NpVDvFj?V~LQCO9Y8GQZ9vuEnVe2^`||jSOwnx)048_7{&i_OSP1kTHL6-@Z6t zFPOhE(rBw9V_&eHxd1Hm+pow>2!xsw=?QZ=T)sl_GKw#+%~+L9YQa*uBiCB07v`39 zu|L1dA!i&aW==s57eW=i7Akjb(H9A^g__0#GZqETsRh8Y9u>#8E~N!CLa{g_x#6CT zAW@oaU$FuXxUd`mD`W{uO}+6|KzfG6u_G1Z#wGUwNjuBp(c(6iI=E-NPwN5H2=usV zU9LOgpz!1@_;~r{o`f1PbN6$tJP2`iEn*_p+RcuZa$^yj#&yO@OI1HATo1Iubp)BvQj$&)D;t?9xM(+3QLww^ z!c8ms&3Ga>x5zQLC5^WP`WRufEWD13$-V-#6fO{o*9f8Zv92}xpLRhXsmJ>pxgxwhDYDX0p4EyON#Xb$&qO z%`(D)H12{6nPZ)_GuKdSx4w)uRd1bN1ns>jnWoKlyH?tV1Ky=dnrsY@Mu>43L%dhT zX!Z%5rlb*fmthK6XfFtlt<}rg1N+pSYI^l5tJdY#lDk)v<=C?CWe_gEi*@gMOl0Za zknY`v<622q!!5^Qu-SmU?n}Om3x%t&U)I;=#wSN0%2u(A1{)nI8 zT!LH*07(XT@`C35Z*1*Sqvr2gXL+wh%3Po|QG0Bpp0cOJLef-s=^O-!CyddcM62UX zbw||q#)ycCp^S9y7o>T$?BN!bAnR(TV`&kfZ*USGs;|J(JX2f1B7$flgXNXGE!8CH2ULxEqpbqQi$zNN{k^c__eyBYeW{06h(hH z_dFx^8lM|#T@Js7xoyF2;N63B+-aRUxv;!pHEV^&MTa-60A>eXibx}1B8q`BB?2Ft zk;@v>ngXL-fe_pgp9mhFw)eQ*rv_BUN(`(+xfDzQ#8oFJIvS#}9QbX?8i=4*6486S zQCuTIuk=cgM2096H#SQVPIM)2<^J74u~6vzbF z-OVMeyd;n$`K^eJ3S(xu%bDxIj@emWhnKSSXh*er74uy7mG$7A zGh7SqwF{kb-7okA}YCmV>x}1s-gO&Z!?Q0(D;;Dka26O0e^^ zjbq6$+b4{Jc>h={x*$FX8QjZrXd2`6j`ws~q&eUX<>axtWEsmNX-v_RAzJKFTXeq0 z^(yZxXV=3ThpmuueklGJOPD}JM3%70TQ9U1w9VguvhH{hUYGTq?WE9QE6DWB8H^mfvt!M3v^C(P#M{pr6TX}^yJ9UJhvt0l3&@EoL!R0 zaRWzi77bj#^MkSd7{So;cxH)OY~z!oyzk;(<~lZdA7FW)2~}KN&fWTi;ev{!24e$A zu16W-j@af8Gy-GA^@hEYBwANV2~fa%q`301pDo>52@5)ot4{6c0B09}C2Fs|k+Ypr zu5{4}2s*ha6mg}n995<(jn>=ET@mlpkaBqwd534`K^&Xy?Je4`3JHA73q;0EHLIk~ zU@#7gaEc6u8PkUgFj&z;7x6=q3WnGa!6vo60Z%Msl64?izT(1G=?6?9E5z&^C4|#V z+d1k5S-`%Kkf3%ET2q^t=@|RdB-<(L>>Izvc_# zRR^}l^>yHB{4v4M_yd;pQ{#YCi8h5RM%m+nnaH`a4aL`*zX`xZR!Cpu9AC`S2s~vk zpTpv^7siJsN7so#mbKRM)|PzXKRO!PLetOc@YQPfMa}IGO~s~h z;?`_R?~s4(IqjOz)Q0Lo=4P2+E-X~X9(1<6@U`z0zsjeZ~i zL$UGYHQ2?KK0Ty$fRi=4b25eWK*swuH3@RChB&FY>W7DzrzMj{PIr>i511!Qi4Ksu z-2)dj+dF2}b!@<4enEWDX5DVXF2^JzG@u)y=r$lq-BpHk8UYDcE`#@zI@~|dSpmm? zFQ@?_U{H~x8P!-*gj`&ypA5?qpHmW{2iueOMliG{Mi(A{J|j=WRyE1ty;3o_C@Xh_ z$@RYFb*1km>s?cgnPFRW)_Oxns427YPqB6BPAaQL3$?)vv^`>7VL120Cxv9#HMYX~ zI;$<>k4rXToWk}a#YX!3rDg8nQ5p{nAt|oGCBWd4VOl7~)~?B8`EMq-J*(Ffy@o4$ z@bXIUAOn!T7+f?y?WlaD&jTVAER#o>cK*itHFb>|Fd% zv)0&@8h2WlOpX33E?R~aLadD($Ah28WVpiIauBDLYP=NDy&dN>^->M@bol8TWu#u1 zow;urv`pOm?G%?wJ=g&C+K+T9P<5Z>wukJ_@($$&^>qeN)F{WERVqEsgA90f$Q0O$ zoDgHgOtV^4*_Jf5T*T*2JTW}4J8ow%$vV8pA#E;V7e+tY#lt1!?1!%zzx%+Iqun$Z zzI>;qDkC-(6!uPzIyOY_4Tre@NfAQRd|yxS#F(=)xnyglr7ya^L+YU6N=nvovxG?( zyl6V+!w!j~!)z3+^NmtRd6(5wb;hy?-!e>D=H^t6ZgNh5$wx^?4~{c7vr@-@MZ9uo z83OJFEHs5{586&AIz{QIOE;84ti0MQdAdnMC@^B=EI#7mc#WABWUxb=sesICa3ig? zJ_iF;2~#VuW2-wzRe&AMm=3(;tT6IKq))e(I-EnD4sb&VQRf0P?*wC%<;`Zj#Kbx@n!=915qdhI ziq?1?iy5`{@!s3@2l~3za10mTVF*O@X0OzQ9Lngt<*uVQa?xGfCG)vSYV1f9nJE&G zm-FuQ{Dr+3_#Uz+72j^!NUbqvs=R49=2z#owiMq{0?!RRK3gjIdtrR87-pujB(bN(EQV$s_WgW_3B+c zQUTHD|{bM5VwTB8EpiRo6b0@|#J zw~VAjI$sFSkDV&qy~eC)f8^qUf*P&R4ln2~M2N=S7Czz(j@rrsm+Una01}q!u}GS) z)>l{THMo56TjU0TRSqr8-Gr5^^$auJ@IwztaDPuoCTAR2gPuFGyzDarlo*Svl5qIKahxrFL_7<`?uw5tuj+MBK72j}^cJn2qeuI3RS_ED!|E3oc{)+# zS+c_~Dp)u+NRXG^{pKc5;ianB#M+lzEI4X_ozdng#cNbH-FZwguWYpMUVsRRMp(Ny z!>mCl#?z!`qP?17dA%Lw1z0ouV`pc)6r~YB!Z0K@-95{KrBD+KXdp*t88>Lqt;7f4 z>+xYtzK0AxrX|x`GT8T!UX2Yv!cKA^>GeA>K9Gawj(DTYwuh`eGLbcHIQxk4XZz1# z?#T@B)KZt21A*Z9PsvtR?4E$=49ZC97Guhj?n}2Sl_LisU1!kXgODz!dzGZ?47%`V zr0ba~geF|J1%v#bkuCy36i-snKOTMu1 zm2weD(srRUH>I5Yhzs4i;OnYsQ)*J_gB}bWoF{%RZX82POXMPW1UGP9oJ%r?5{t0) zw7LSHHt@6jq&JaQ_7;mM(P)b964BZ+BqEi3G-+@T7SJAm)P+K36l**_x) zIfeMsLJ-1qhB^nr3n^h|nb)MqiKPRW5WGCF6-Tb%6tUvWKHDGiEO?1wmUo+vKEXXh z$;|#LQWS699^@zV`Tw^B$nE_?iZMAHX!35F6P8$Es>#3cJvZ=6N?JK(Z!nhDs%x}# z)*-Y=Md)dRe3A#7oxWN+V&IkrL9RJ3A$@E4%QL zFRyRaFDNSiyjfD(W?7`JGqg0hTji|%b+5b-ztbhYwp=eSXlc{>Tqx-7;+@Wl!wI=L z>}5fL1C^h{#z9g~rh-pr7*u7bG}h^wlCoei-YG}pPg=xnm!dArxLg9N!y0l~?J|~8 zgzs7j5|GQ{KViJOozm`#THbCpLA?5o^%rEmba$CBRC899=FnK|qDUolg~nDn#5hJx zMg*)RruiSRQysRX2g^WB#n32CAqmcegzK>-6a3V&_Da^`P={n38Cc02@dca1$GA0d zRRtWr+g#j&l%L0t%q-7sg0}ZU$-P}T*z!~MUSMZ>x`2E zHnusV%i78>l7#cd{|X@CVTA@?nO>y@0k%;(omHVi?=_>PHaORX`iiw#k8s#DF3;6Q zd5xJaipiGPvGbSj6&5YhS>Cm(G-%U4vRke5jLtg|}g)z6HSuR6EG z*F~Dg`x!~$(NJDcxZC9B64bG$ct4H|nXc1y&JA#QUaq&t^cW6X*#s2Jjm>@_8airO zPcUyk6*#m_{XWECPBPymG4to=eFv*LdfPn@NIB0A?{CR*zNjkL-w%{+MUN-%1fNhs zR}H!mLzVgu4y4=cxt#OY550Q*1l%!qs>|Er&~Y2Pz>9ElEZ!3^yqytxDz0E9GMttCLbnUJSZf7( z)+dQV7TpV%urEy@hp|NTPL-}gxlpufEBUGK<`{rwU*3G$bM)ldn`I9mjPL9HO?V?W zHv3KhgCs!h+^|Hf74)W1VPa2r8>-?egV@pWyz&%F+up~#PrM4%N+fQh^WF=} ztNlQLfI)(uoa`d9>a51^`pO2t33_?#h|dm`f^_;?qce(^sZxbv6%zvz=pAV&kX&+W zvHNAxRLH}QN}(LJ=kr(6SNf1zpBT6Sp(x{Q;XD{@7HqZq_z(Tv)^tM!UlWJ4v&V6d z0V?Xl`o5$x{|CCaM!uKK7x_hDId*XiXO>DToYsoO7uH(aOMN{gFx+c`P+U&V!7qh= zD92HLoJ)}yi_~bqG3;3F))RHx$8_a-_P{{pz6Tz>K0kK;!3XX`9>zVVA-qbA9Qq;> zymnYF(%{=NEHAX(0HPu8`TVnPAJ221Jy-k|Wu=c?iCj5z<>nQhS~Ui$*Vk@Ziyy39 zsbW({CI0H}7?lSHP?X>#_BYVr$c)FK7d7fO$WFUViGp-37$w*WgmNzyI9Fg42Og~8 zUsQ&F9~@v`amxgPxbDZqM+I)Dh!0sF9C)k(Xuzr&mwsI9;uOIt4voo~wi4yY z2lV*>iBULl#cfI9z`^VaS;!o1Y$UEW>8mr-Z0zg`h^cL~<-B!Hj8;GR;l>ISg#^WW z0hcfcodu<>F!UGj1Cq6^MV_@Cexu(MlR5y-%CI1E!~ ze#1i~T6Pf3z-Iy;66yGP1snu-HWK2d)haDRYf+F?J`WiUMV!vd$9hVmUDdmD#z*GF zUrs6^VtC^TW>@8MYSf>1MN8oUbI8kye}LRVu`CBb=B=$%a#oaq4Sy1cQyf-_mid43JrcoJ5B z;ICu5Ouq1<>hnCUW|t~gu8&X5FU-tNk6a&~pIaCipH-^sGDSo^BYLY?kZT%As?M3d zrz~T;nT;gSpwdsF5!e0!SP_j=aH&j&O%<#K?f1&UHj6w*bc1D5xRQwQ)hBtxcu;Q6 zr3yp9)spwr8+Yr`81%Siw6$~s6Q1;_=fEQ{H+_9}cywWQdU`$vy}|%Q%;}uKM66+D zF#-oUA=+TmnFw;D_&q8Jx9DZuYA(|_rhGKkZE^w~gPpN63ipt}WLoTUg(|YVj`<6B zp~Ys6n;8H{*JQ8~CE^0JLl%4Q(Fcz9_4Pk=?C7xp$Yf|k$fNeRye2Dwf}Gc%*Hw9_ za!R>6iYTqC7vhXw_J_SiI zFD0)tm(@i69}-9SN#5}aLGIMqN6wx<^YGb6@>B$?>UN!4Ms6j+-XSU+uxb_D>h4kl9p}B-*S|W@G2$Kr9zaZ zHhSrBV*4E!z|okRIwZ8dyaF3mvPNxP%o0q@Fg_?qP-G#37|N-TjBm8iI`ehpF_Dlo zS88E8rM%h?DJ9qIwsNYh-xGPa=c0`KWvGBI_=$EWbisFAAzw7VWOUPMzy!wXJ4v{D zc+Jc1`qFt7CrHcPOVLjWb!7Llk%|Og#+Y0dng>QAWQEjhwpE)P(9(`oA&~9ez z((Igj#f^z%ka8#%4@)is6BnV3eaMAV{qss&?<##>8wdxWON|dw`95)6%Le_(s|&t zm?2mI;r+(OZRMWh2l}o}PmUfNSgO^b4d_{8LU@zB(Hd%5W|ZlRr!CbfFpgWnIk^xPCb(lh<1on@o(piXnki}iYS1%mTv z9Vs9va~eigPM-0ptlPV|crO4)&Y$+gA}QMj^IRz6m&_IQFfdq2sQ=%P7 z=Q<^HaRBuN-0QX+m1D7*ISrW+#7k0%Bwu2=R^sLZPn76b=Sw@E4g?zQXu!NdbXGW^ z=|RL2YS-O1Q<5&N$)sq7Ylo~*J{!Vthti#-gOZ7|!FvE>X%8b54Z6v9dDHWe7is4; zS#I{R0|ZQxno=iR@sA08%Fhi5vMOi*7Cl1>^m<Wz|lDHjmOl{6Grr7C~l>(gqUagWUhm5AFr4~ zXA)oYeaoBJG9E9Xm_p`$EvA4liJUwns|WD&U_Oi|rp`f>J*^I+p;d=pUJp5^f4E$^ z--=clP^+{PVO^(yTL$vns`NbY-0<|&*!b1!vqSUa(^Jp)ppn?=0vFtXNvd!P$3h*) z_zk;bo_b7*2O(hFc}hyC7AplfFvZ&g_nt0IuoO7)M44WfWWB338b-XoN|p= z+C~ZkDh3Z?2*oD%!ze|H6sZcu*fA@Hoq;-tkb{0d2LkT6;fu9_8kE-dTIhjJFK4|Z zJoT;ZHsvT||Bq42x5?QoL zvCX^$bo7C2AXy`Vr9Lggbv+UN^sp+%J3KOwl35a&@jg4QT3%M#jX^W9X5fS1s-db zi2`Npm@aC+$Wx`ozb5`_E$HB`#z0HfMLOF2TBcG_9NbF^1c3^Ijz8sV2|U3i5*yVzW6LTXwk6>R99!lR#aDg^!7ZwFN~BZ5oGV%S zwIy?JUj0Y*vZr&BcL?BdH`8(0bA0DT!p*H?GREY(`>N&%n=KZTwxEa!ht|Z(&3ML* zO+Aw33>DaOgPuY*@aJMPb9}I&arPu->PTu4TCk|9!dk@;)Wrv|sB~%GSm)YPl`1we zg=I9F!~>s2Mz3xT3+YZw^b1-bM%EP?InL0%rXiPE=L5+r$belJ5|GvW1GS5sh*n%G z8kIFGhrLi(KjK#gRdPd78Jf5;^yJ*a-00Bk@U?~ObEC5hGeh&&=8$M=ngy67$cChM zAUXR)A+WApzq&9!H8wpvDPr$}{z4%=D|Z*0t0(T(+UqA)TDzNKWFc5lF3S{Rgw5me zxdvQtHwAZyALS*eWNejd*A6?sd>5=>bOp;f z@(@L4CdTLH7skdXMv=p*L7btVom9#oh@xh8!h#YxEIUuBNV^NHYm6{73$Eh&vU{`f9B?gCMIOcAff65Pz|{*APtuN zl*DnUWSYes78d~77yumuYxjiuEO$p5j*D} zUclAk$Mcg%^l*{~`d9`i;{4Fm$k6P_!kNghN@=RZO zcgW)uzbh9Ro}PJfVdmP<+$exQJ3a$oD7_&S$Z5{-!o<+j)$2o7nM}tBDc9=Tk3MSr zigQV8!9DQd=HQqPk+_Xc3D+2&nOPXUIXpT;b4||nA{OOE&Cbt{?@@?+j++cRxByfn zp(m(?_n++%ccQ$7daautbS;jWaof{7?7P$rp0@xnEMOew$EU87s6073M^KdXVP9}y z!~mKDz{Bq$q!(1)9gm&E4TZ(_P6Nb~G^3qx0?ug@>cj?Q1l%lwlw5i%B1UY~-jIkND?(Cj#;VP0w(9Bh{WC>~*<&~%Yi zq*Z{REFYM)<(U}2GCMT;WHFOA%BU3D5RXrO2dcw&dTQcHnbF2*kU=;;WkJZNa=@kv z_Wd};lV`3(}+tAQIe-Tr3Wr<`A>Kr6&{w znhm|=nRjwd&tHSOGCVdhbaf6GASt6p!m%F+5E&bMjOfnI%OE*0Fk0HpkH@^_qygHX zcwYgEGT-#F?)0445FdD^61vM?d4tM(!zOrVIsjn3wk}SLcpnMaz#&vdo!!RC<1j}@ zRMPq4!tlb(#B~q@{^L`?rX^(5c5OLg6MG2;#`^(WpCKW&b~kC)=nnQ@-dJqb#kC=7 zmr)ZNGM0AY1|;R66Jk0-zWs`bx}BD_GL1Pk1RUBHYZ)*?5!OMqQKFzOhKgBbm3Orw z5SIj>F!S_IwpUerhp^8w0dij+ENfE=5GKY%leTQv#b168s|Xzz1?);gu(KQ^o*TWw zg%REcT~WF@4E?U){g^l-9ePyI>R{QtLR0f0{U=FMU{6qtr*aycC@m_eUtB$sEy5+8 z;7n8e7BZG}k~|=C!`l;oPzE@3!MlVsK@m%vM5RmLLF*Lz*ht|-I)2)eXxYn7P)yo2 z-oo+WHQf&7p8V)zS<)$85p(iTR(U=Vc%pznBi2`Sm|jOuEg!ekS?jAkK#^Kp1|*Cm zcf0A#1gyVSMhK&m(}@I0#6LNip-fv&CdS}uRc-3=12{&&DShwlb`xKwm&Cyddn*p( z8)y@&_j#+*?pYDw0tKAu%e#p)w3bJo_*yjc$|S|QKflH2v$Vx7XTqM1r3xmxxR9sM z(Vd7G->Au{7*TK@P!Ue^(8=mYa)?M`1kX6TNm$y3Ewdy$^xfCv`0&{+VJpdrcgO~Lu=DxI@kfLU2Q(J?T3=1@_bdIqXZhxQH>JPM83+QC@FMm zaC#8Xygeg5=V6bvFE^&b*jlZ(YK^-33rd}O(#;0P)2z$Nvt+d@x5F-(Q7+Djizq8S zqxCg-;p6x#p1cRoYW@P@?RP&EI;asS(b&oQhp&;zIBS(_%`Hi(n=oxdaX5+LmX{<3 z(r~klXA)V|`g8*l{YUNKmZUfk9?NTMgt%Hl_MF(^R@Morng|6|g}#>o zDukA8P_s&{IK3yCEzt3k>gi&4N1ZnFQN^MgMqE@BIYJWEgto?q8mgBJ5u56VO~;aR zYHjXN){bEB8Ojv_3}Su-d^9pLZS!vP73aa!pOG~Z#s64gP{iU|1{I2=SNU)<(NikA z+o;E5mSl=XO^UYDa5xHp5*rc}56%xrxvEr;1$37p7 zB!eLm5BilvJ-{N=Yow8qO+J7~Lki!5JBAGm))C9cMHd)#lY$be9!et6C^*SfH%C?Z zVMja51_>Pr46XcNJ7lC#pzGZ`%jK~4Hb;P@p27&ZH<(JEvqf$;KpkK9vGJTeo~nRhCa--@QKO zLmE`4B#YCqjK3fakGU>6XG(DwnPP1B{sep>;~`#aZ*)m0#ri@DpSHR^TAPJ`WnS2P zKz{`VJ7+`LWbo8J70OtS;gT=6n6m|A;jvHvJ!2zsLE>5B`yy0a3Xrd*r2t#xBL=}{ zyKR=on=VI60mNG*YZ2Y!%~C zi1B2yQgNz)Ck-39x+KMSBn`U^1$vGIfh8r->5wD!tH1!$gm|w>G~UkY;@MXk0UwPk z$&pwg8UlIQc~0T`7+81k9~YfG|5UOYKTRPUIMS$it5Eu7M(0-=8LGxt+lUbAFY!oD zGne}06_ONv>eA1#<4*czzRWjAn^tBGi87ac4h9b`!IT}2AE#*MQips!3;YZB?1Y#! zNkJ!xPZH;ng`MJd@o}80TA>&Wnf6l8Dc<@Gz9}1vF7@Daw-e`k9_IyvRLd^j@B_>> z0!oUh@7U4 z4l2>{H=(6@8Cb~g?-gkZ6EGCq+gp(prZ_Hq?(fGjEBLMoH^9MLA30YPJ}^*tT?CNu z?+aL=#VvqQ5h(e6Dizb0TXi9R4ltH`9$D)CPzp!B!(NtOn57s6^q>UqK+(%7I>yIv zgAx|n6${9oQSSF-1G)CkdMHM9l#`AZA=Ht?P(jc;bL5p{;fdZ{EO0Yn@-e!y0+Jkt zbP@Xnn&_^|69+j1HO+^cHJh7Q>hY6J*`O{}vsZm-?$=VQDlvATDyxNTriXjNQzgEu z57;AsfgeG9s|2EFrHl#;;T3qWz#Fux%fXHbD{{sl&#vC%788!P)f;I<8Y9EqCxZ+r z$l{9Eor%y<{!Q_WowJ86G4#PWJD_!5zuJa)c_r;|Oa(S8DP=Rz6IRHRtEp08@crl@ z(*{d#Cn)5Q4(af_`GXn@Q-!&be*lPy)2LZAI=}RqoENuXn)w7Plzvl=H^{%BY{~q-3es zuv)8bthDk2u=~N98+WTXUW9PqI%Y&}1D_z>RVLfXP>?4tjviCSNACB^!F90gp@zW3 z{v8rTY(+T4VY!e2MRsKBG2gTe$UFyQfXJadt%)JPc{O^3ph4z#-8?P5F<2FDw`Fmk z=L*1d+1!91hDasb8EYpc~(TjrS? zI`2drTyy9GO_xH%m*h@9&*^D4IrN@UIQdOZ!r9M>7zWGFKkO@}$=Rz(br`fhb4WZE zpT6IMk6j-oAH7I;H@(e0L61unakqnKD>? zo|VYx5YbT<|GU0G7Q5u>(qX&uZmPipk4QibP!((q=+9x+k9`7|D>xPZKCnnT)6&>Q zPN8hj!c9E_X*O6t7|g87QC_xp0aF;qdt&RC6{dIYxeqEidM_%pyYm{)V2P?3uV$(< zlHLf!%Tax#bV9J|#xq(iGxBmjt5&cTQ7l^sZfDs-Z{-q7j04f&)fHFZ1Kz@0JwE4w zOouLB$Gk^OYanW=cSx(B!xco~C5);XEp?%`h3<6et;rNjao7$^i&p!3;MzqWsxY2M zTfJ<~ao6EYzs#HGNCSq|XTPH~m|lH7BaJ%9aX{>oGLJ-Zcwl0C@j?avRL=FE>OWf< zZfx!1vibFPrEmCH<DrCMsJ8_|M_wHR% zOqPr|HVD|fjE)ch)%u-Uvr*>}lM*kSn`s{BKlZn@xPtR;*S49ea{tklO5f4eF?_4z z*{q^{D_Q0R!hxn2_f1xqr&ZkmLTcKMGTM!bq_ywxyWw50%k3&WY}PO(j<&M7jVwFb zx^Q&mLM5x!(F-rpAXa zRJN(Yx1}pOpUI*Kp9=a+EEt*@m&UTrHg;5lJg+hTU}G%1hGz!PO*~KIS;y1H<9~0! zk9+XE5zm|Oycy41@bK>ir2C_H4`v@k_6lz$xI|AF!^B79=%-y-xsc<^rV zHw=4_@7vjOV`Bruf)Jjah;Oy4@*VtpYY|tbGA({HdUG4N$E!Gy&GW4-HkWV+Zk8$k z%`zd@$g{T0Jy{5Gl@KNrLre=?SqTR(mI-g+6>@R^?V*o+9i9`6+ai)MB)>-3T)Ybj zaFJ2`jbbH46WLDb3fn7nB24ORH|pD)OI4z5##4G20x>aRvTVmH!f6Slf>~?3B`3@g ziR-*y2sN%4PA>zw+sF+VV*V3mkUKFmxaFT^?oy;j*z1VgTiM*kxr!zdEZSkzM15#K z=ocp2+mN0*9e34L8mkp-t0q{pg@wEe_1XpnB7tEvoqdij<67IbYGo0sg-^9S??&}b zbwiu6#8bNKC+gKTk-W1&MpSdG=hNVvQuXf1$ zpz{ddhIrEHUqreu;{8|fK7o4H-#2L0X4yJ^{UN^n7d-cFD{p!`LJ*|&M&lWVLI4qp zio=Zf-NZpQ0M#ykZ*Q4ofoUUo5Qen)9lje0#C!mN>!6)LE0)Iv5hE(u>PCHT8*!I` z@G;zVdge%x$_$`gvYLqy5>AgEX&1k1o6oUvj zdk_~GZ8dlwfz*BJ*6>q|AI8Hn_#5+0#)ZEx^~nfv_$0j#?%GCUiTp%NKS<{T&4sDW zElh<5CqMpIVQ`T@1yxW=nkN6m`T!>$YyyKyHN)khu#jv*M#XM%uz}h#Bv7P@@^cQ- zq|&}C^+^pfjwNZ(0&Y7;OHhW%pYof+A-oN@*VZeWSXu&Rn+!l8iZFn~eQ_n5@YF;$ z1>8V2VrdmwotGI6B%cbL7~>wuVcZX4x6o3WjTSC=hme5>^7p{-2|~WIMCBJl#?rIw zd;ijK)}DtvxPy87nZfKs2><@Eq3lJ3Z~NPW+13vaXaD%04Q6eGKl2Sk*+&pR`fGz3 z>CE>v2c-=DL#cs+j|((4$U59k6%ycw%cU>wj%6Rg^HDtfyI1~&Y>V(?i2FDm(h&ZA z62G7P80Zwj5B?nFON2l0GlSX72>&;PABMd6pYT2Ve#o78|1{!n{5Uk>F#9dM z|2@1@p8V&?|2e#W0P+48m;XQGJInn9%3=O>r29PHe;MHy5gxbvKZo~M@c!edcclrs z$olP%?PFU&uQJet3?@*>4y64ewq?s34G|(&H?~^@j#2`^&yg4rSQHLz?J24Uzfz8~FK%{qC0E^l#ylA3zn z#SKb4yP=Gu{in`?jiN0grJz1hyBGottRLc$NV8e?;&Z?;JRigJaXg>ILwwu8c>NN7 zzl7)0cs|45cgC_85dT~F{aHMp!$Usyd4yYt|02Ry@VxMW;p|KNh9`SJa1_s5@w^?+ zyYN)-9LK{y054lY^Er4c7=P}vWxVdTMIRf@z?T?bTO}MQXSOqZ0&m-bPz|28uDT@m znNi5cX?-%7r0AC}fxqFoi020wsyHvS`688B{JN+LgvM1HtNZ~$0~ge8RG=P+x}HlY zz=8F}X5Uc=kN7(ZOv6yttBXxnf$<$|f#;Kp*dmguqcGC@>$prWF4KD%D=jER z@WV0bi;hM}@4P;007`_d1XIn>COiKFW7)rX3Hm6)2me>lIfVBhoIv>f2{E>e!m~9|jLb!$S>6bw-5uQi5gYcgr-3tic@XMgH2tR@N4}=1@s5ucOv`*!t;p#6vAIX_)7?X80lX^`1QX!nEg7!hkgxo4&epV|2c$z z4&iSj{L4uH1%&@M-v0pM-$m$uuv}-0%MImkxG#k9&_?U0E#&)hRZtFmR7e#==zMD# zq$kK%4Wenb3KI#_t~MD9+=BWbVPoyoNxnDMfEPR2CaOSEU^!{sUFfPSczFLT^;L$n zgkuiZ>cTVFq}H?4tSQ%N972pC#tN*N*g)ZcZ0s(qQ)`BEM3|sXu&x?p(`^}k)}KB4 z*Jpor>|gxk<182Q`s439F*5(h_q}b9Z!drOUkobKray@JJSo$@P?2LZ1?iV+BtlI8 zgJao`{W^FK!rtE;lsNwLTX;YEX^aKJ5B$SC{?Ft0zeoCWzcHBoFdqIO}aR`LeVgzNei1!VHM*X7STlQ zT!`d%Oh$n%4&xQZ-A|jGD3nl#7sK>~>`2X*L|+9&6=B$F&Q5Ku%4Ms~#%7-K!jZR0 zr36Ba94KUDn=M>RW8|Q|vF$ zbxRe~>YZ9+8^}W|I%rWxM|4c!(U;fFREQ|6JMIEyKUrFUlN#&n)sMX}FSNxz0OBW5 zS`-tXi{9D7BzPK+q(L5>W;6UA9IwIrkGJae2HE9Pq3b-8k`YgMpT&dE7_a0#3 z+~_=yU&@9jrsp`o46o0QW;lmoX!fcC^XX^jpPqSkGhtMUV7;Z zUzneN$I{XV-kE*xOqPB41CVd@w^|;2k`tTo=^P6vFuZL zUc&S1c=$It{e<8zY0!J{&hX0g^aQ{0_tf}Ab`{4|%seq9ark|8V)}}_;I|wo#r<22)bxBdF*=pu zz?R{m`7C?)DN&pmkFi058BZ^Co(hXHFm7x9eXnZa`tkN-W5-?e`_n5`px5#biX-}u~M z_I`v*pB>CTfbhN0$9@#yFus2n;g2BxBMAQRR(xlffAjx>y$9hRA>RMUWN722 zPP}YJ-%YGa!T2p2Atbjd&aB9|9?d+E-)RocR+xcdZFSYZRd=e(+lYoZDZB(6il%Mw zE^aPkNFfyb$?P~R0Q!wY#(WJ|A)|C~o$Y!JVw>;*%}HH=WL*^h&6-|p22R1&RGj>0 z52AK#SHL<56fU-LaTmP4F3RUFgjdGhWhDM~EyNiiEon3(2Y+Ia@-UQrv-p2VnM#I&@4>TCq9&=Ov|{@}jq~=%LGf zC~$MBLH=YqD)m=w2sA(JC?WRTS%f8OEes0Ur5zMb&-NjK+%Kwq1TwAZq9!C95f1=( zSJ|>YXfhR1Gmq(SQ4{PpSAWoYNn?(}@UGH#>7MZyd_5A*|m!l+7Ug?l%o(Hxd3dgij;;;5B zmJMYeK==o57|MPW;orvh4H6v9@o0;*qN-dni~;-aJ!r*UD4%)bh_dn0%N?i;ZCicT-Etn@8{ z63IN%Oal(Z0bO43`2|@I19bc3NvdqwiH9op^M}J(aiG$tHVF!`I6nx*7Qf%Et-vG- zET@u=vny5HH4=Rn457-% z) zgr_68>mItUKtW52al-HbSU@hMeFhr>Ix1w)u1c>qRzUA?Z8VdmMkZskXtwrsz6?aT z2~NDowz|cqy-u4Q^lq%V!tABCe1v}po9MfK0(Lam;BK}cf8zIXJZJG-#500t0?$02 z+jv&+Y~X3*c|V>H;-QVM{o{DY^FchcNz-0UTOQ*+gy%&(AH}n-2qZpQ(4jdCaoiNVGv{3v`X4?7r^OHl_EC2LrHu!&DWpgu}g##~#%J4%E zRfcEhFI1+A>ny~7!kg1X#SKKDN5cL4rHuTaSAzZL%)J`xp}xKH%Nm{>wmdI*_~p0Q zFAFxaR<@iq;P(SLIP1@_LWuvM+u?#Sh{ZE{A1~ggC+B8HhcjGv(hs>gs|+kxSj4?< zK;z3j-;#~|&7th~=B{Kvh=<{bh5r6L-ue733z;TP&thoT^3eO`p~mO`7W-pgUhouO zmcMVzJoIurETj+lJpaJ5|0)Y#8S--V>pMKr3+emke@)*%Z{zU_!&koi<^MqBK=_U4 z_50JKZ`C5+pzp6#*v`i}#&6BCFMlh3f3tph6U=vCr$4jr#BW1`?5{?@e+<8oP5yp7 z`u!8p?+-=4e=_?0SEAqlZS?z7_vB^%V)Xm%s0C$-uJ|^D)EhtaVXQA8d<7xnuKm=Q z_@I3hA-nlSgx`d4@UOz=i}0fe*}X3!d>7;YyRq!s5PlNjUqG0>IF@}of-QvKf$;MP zzYF2p?BFC)B( za5j55JBj~JXJ3!Doql-NLNYw$&2xm&1n8Tr`)mR`Yo6JNKpO~kVHR#dW=`{=7Em!;i7 zkD90lR&Fbu{m%>@rvMR$$6~f5?P+Iy*)gqd%mXB9?`RmBPh&uNxCm$b{Ls)JgX9cZl;NOO{T4BhsXdydSJ(Z4`t<)(VsKfsW5&+fEs7XCU8&3otRiw5?jkQRp23` z^=<+pI`;gsh;|T%Ip&llj-YMpD2dd8<#DV?8(16T7V$v47MjZM6;NkVK&5r0svJqp zp%~O3#?oPt(`kpn{&ZC5Jn@8mVa-ZmH%oBF>p8OzytdIJmNuEqW#icdeo?C8tmQl) zUD?DhLP_C4n#mlSLIHO1ZAE&{){w`LHgMkDkrMK<_`MsRetBXC3)l z4OFv=kZpIoATE$2lF||rR`IWM1Mit)iNooHz=^n5lkp&}?gK4a##>YPOD7e{;YkeY zgcRv!8@abp3h65GN&n4rj-dPvjMgIRtpit!b1bK){7(3@R6>E+tGqv^p348$q_h)~ zf<3u|@|%eB`JQv^k!%{Z4*|MUNY}}Gh(%M`)zWWR0YL6K{MVV&`@!C`l{H`(aWJo; zB8+3U@O>2RRPJ{l=IxPeM&8!&?;=WouN`t-#3(V1$13c#eAf019#Y79Chi)5D*xlY z16k6QdMDAMWhsTcZ51u7WKSTDSfgBHIC}{HRX}0V9!wzBlC-BOQe2i*A5w$QDDuW3EMEjb)=X z#I<-i_08bs$qs2-N}YwmS|T;;c49- z-MImMOy1|PQ)~;Ko>hrMLGxn1n4=rs)os|5p`BMPt$x~qkMlg%Mxc6UA$=5iW3kpA zYcvaqhhZC_J7y~zrax5SEBmJH$v?0h#wT@HWT6a+9iu;)X`8$Pw0dQGb(M?Q_#@CJ zYB-&-xqAcdMBbQLai2vB0CRY|$qS~efJSr5HwNI_L}QJx4BvR36&|TT4%7&s2``qRGjcj?jmB~>+=s&|#4|Amvzp)hvclhvUw)3;} zOT?Nko?pWAX*^%R^Cdj*{5h=E;<<=t2G0tf590X!3|Ai{Y(>v%qZ=S4jHKGjeU z3buV3=6yj2adf~79IEJ|g*37wQv5n#Kn(l$k;lJnnSo@Sem%|Yt$5Fam~oJHoR@+Q zsHdaX>iJjT&Vkhgln&b-J?fEu2&+ImZVe_OcfHR>u~rG^3}TcQkzIk!P1@#X!eFrr z<;uCBFyA$nQ5>;$7Ki?9H@RkE?#{K1U943u-hscl@85~+Q7t&3h@EbRkWe50m1@04 z_a>>+TYA0zOucc}+=B5f)T5osh)`H8LQh)R_$C?z&~hP>>#=G7`Hs!05roHaj7{_k z7{X46lh=Hi4Mk-0zJ5XP9D#(ajHx?O`FE6qb1BdgtfaxtHa;*-tEKFCVC$49NV^60 zs11-{TL}{w5m&L92g|oIXrWE+<2nnQ=7~lfz?D(yYzO;}@wn1#Jc9;r)v#r0q#@qw zVT}(k1l`e**V+@f!r2Pz*_FjR4Xhb*JlIw)ZN|8Q_A(V9n2+$7Z@Qr`PmN|qF0)7x znUgqd+M!iaywb+8jcg4DORlfM$g19msPARWpU>d=9G)-Wc^|m;iR|6NL%$!=?Fr}x zQ%Ls=xciChJ>cSSCzkm63~Vc#+pcGmAiVS7eK%13{Q5TNdo7!TvnamF>NdVhh%_z! zKsvZn>cThDH|;0$NhI=a!;mlQBi_(EC=Esyd`9;anDqZqHpb9sH# z*Ydb`qfEAS45e{_l@c~(4Xpf1$al8t-PSjTv^NhSTthgAd|L>oAtkaeQ|LR_HbxLn zAxg`qtf*3;)!t?B8odpw6Vs~2_G^0@=##!{ zyHtPE7;j@5Hv|aqZ^pt&gad%r3jW=-`uWEHjzj9@cm79xP-B=%Q(nuiK@#5-d7RoK z|EPmFb z_kYmzU;MUVXc)uU0w6Pi|M;ix!+-;0hJbscN&I~FB>EV~O(N|SV&;*07GJ%LDYSMH z@w3oyhNCj(@y>Ytd&50%y!TCSe#=|mcI4~c{`KGRjqmuTKljb={PW-Pt?&A_zwqtf z@txoG-Ie=#?tkFu_w@E1d+?#-C;A6Yo;rQz?74>@IsfQ|$1YyF{P^I|m0{d+boJW! zyWcZ0IW;}=y|Z)k*Ppm?^U3eKb^EFJE?|ES^1cP$*U@a-B7e(nz>-OC@d@L3ICcwEEm8#EmJ zfBvnu=kI9v!t)kRX*gK1_y72BwBFy)F#Bl>H#K~D$ii>Y@P$A4*INItYWTuWYnVN2 z@z?DAK70SqU(|N|6AfSfguVYI4PRKa@S?@P%fdhWeaScYTNeHu3x7nzm)9&DwfF}# z%-(3>@BE(j>sK@!{26=yAq}%74PU-u?+@7bH)%Nd`@gICe^bNk6B@q!AkS@<{6TwfS@>QH$1Oao z;mhA`?_Z~3_MiTxl%M@83;(f(gMVMcm;aW%f7rhNkcD*%Z`t>28oqqa-tV{X-(=yJ zKd<%tI}Hc_xrQ(N=Ksgumq%08#eX03P-IM#j1fghkuKkxiljoJR5COwluA^}MJb}B z!8}zejc5>E_gorC%9uH0rp)u4cb|Jt&-1*k-?P@c-nHJp-sdcQeD-J0`@8qq=j^-p zCEeeU{64AQBJl!=CrET;Bh7-1*cy^+kh(1CE@&p9R z@-jnxHX^>9cI9U>G|yjlGfZj=@}K0`oAd(K?UYpS5!FC^)a=my*YF&Yg8MHJ+;%yA zZaR4X;_C9XSJcO-%<}Y)Z()bhe#5pjls>Jd;rMX|?7p*HAxRy@&lL784`Ko@=S$HK zYgZt?T<=;<7Hm4{@;Jx_`KLZtKC|Qy82j1>Y}e=MbMnFe%6AE zsD68!JZ3y8g1hIrXNZJ>K9*b@lDM`QY)jkQ+@p|ge&gx(>S8d+4pbP@9;2X7E;Dwx zl|b!alVQn`w?g3IKDX+o5dK2|s z;4Nwo5ys*nhgZj8O%C`Hdl(9;m#92Fm+4 zRD-=ttiZ-ssC*)x!siRCLAF3>L3W@L;;tF?HZ}0e=kx8%m#Fx~ZF%lK zU50c8*@%PRf!n?9wt3ea)V|X~jh)}&QDE0n=SJkecg52Vw_0epb+uk#HV3MobtNx` zYhhesw%V>2R8+o`vD#wYE)kZ=CiZvA#tl{$5tuS9<&rCevSs%QOpBDN1GrUi zT&y)j_9Y^ldx+8qXA)JepPY%(r|pbaIMWFCB~mqhCT~LJS+OAIS{1%6+{|o<+b9!D zZhFQmS%uSrQmb!Fugk**L(C@@{56j$YL zTTD4DiTv~a@QW$It+}(-&h|#}AA2<_xwHh=43(~LbQQ|Pdav^@$SB6O?&~}A-O@qj zi+|d0QjDA3e)UVhQx$~|ai&%l;abcG2IWSme92}dE1Zh(dl!PWKU&D4^g&Nxq7YA9 zD}X(DvINCn6|^I$5KpM{G+Uap7KI-Sl9w*T*EEag9(SUk`fxk{$gTj-+WJBEl%zhY zKYfAo{$FwCRXQVBhyd!e!$)V=VH*L^B`CDxn)j-?^b&redDaeX6OGEE{16OSSVx6&1c~D$j*&P@;s}YuBo2`{Na6sA z{Ur90*h^v$iQOc2k=RLM2Z=vPY$vge#8whpNNgtY2Z>E2Hj>ytVm*m!Fr6iWHkycD%5s8H)7LfRr#4jY~llYm$JQ6>Vm`h>~i62S) zK;nB6-;tP2Vip@Q5qA4CXOYi|BvMJlNMs6=I*C*gF%p?Vq)sB0M2tk{Oj0M2N+L!g za|WrCNF@;?kts;(BvMJlNMs6-I*C*gF%p^lq)sB0M2tixAE}c_B@rW$$xG@aQc1)} zWb%+YiBu9X5}DkjP9l{=j6^0Esgp=05hIbwN$Mn0NyJEGa*#TSR1z@~ndIjdX(Uqr z@j1r-X;fgl(Y#d5<-p28V^}~yP9M3|FIlb)(t}#{K^6jtGwtIT06ME@Zl`}j^C8Us zu1?~5@PDrCfGtAvDUA7f^AS@hQ;ns|IHLI*rZM|dxH+iuKV8~+2F+(Nk$s$A+kj!3 zWT{+-=FezCxz)}ZIwc�(#N>2s^eX!_Wo-OUHi=u2({wDYQyyH+T=$UHZIb6`|LE z6|jT4r|TWHY}X(@+L1YI4-3T7z8z37LA-uE?vNvxX@y8_uwRaNd`xiFUf8b{mgewF z6Y<_s|4sWri{XoNZbkD?EIi=~;{dD~UOq3_4$X%#DtENTA=pLt({kExhIsN++8rmT zU7&e9o`=Y9{M9;%BXEIs_rlk^YKSw_#4b4l{l{4Tb@ymCE_ovQxZA}OO<*feQSvHlT%K9 zJPFPFi(YQIN+tYn%k_4LJ;&gvb|~?D!v8^T)hXx}-QJc2M1ClIKK-X)iN8XKu?>-* z4f9$adw?6?T3Wz6BK+lF=KDQCLu`gpbtsyjVmn@DEj$e$IA6Wj#Uv4zpy*bd205w4 zXZ62qKwS3r%j;(#W8YF^nJh!ZOS4y+o`vF={-5gwi1=@2Gq}CLgR%UwW-DP|FK=yP zcmY1TM`VW@(f;9-dXICUu^=Q??jlkD@oA?u&x4oz`{xrDggq(p&Fwf383RWG4)78A zyRd|D&l{qpG93M^2>S~O)m}ye)nHS*&lFX}DUoF-Y4D9VKiG2yVIRU;DMd8!uqm)! zv}Ot7lP|BT`+%p+k`p@FME&Y29KGTLS^G0S3v#O6@00}p|Ra=>Oeo#MOR&W6lIu`1O zY(w*Z?6Pv{(u<&zG5T;vyaM9djM9LMur{yeNM|;oI~rEBT!i=iPu5ExBkIHOW$jvj zP=2iCIrj*0!Ud;9Tf9Hyzu9gTHIJ~LKj#bi*Ylp9!cEgY?ut)9^Cv9YxWM8RzF+Z* z^Z0yXzEOX*c*q?uJRMtlZE^$RK?h4C+;Oj+#k(n7H9bTT8yT@)Q0` zxej+7!?_>Mnt0exaIeqXuOlv$`;57am_ODltXpvmH=BN3uxvTOd*|*+J&H^C&NN{> zlSABINORm3=a#s$E@dr|KYn=*LsvY&eXighIf55%{~F+ee;;G0seM87VJ!Svc#$)H zVav;o{$s>^&s@oVt}`AwLKDr7K=XU7jiKmp1W$T#`N5WCQN(pUTOJ<9Q+Ek-3#p;` zE~dLRvDXPd9r*10qlHBM=5BJ(b;5UDt9)|Ch?ox+9-K!zgdd>{lu&hv_SkT@Fy|nC zPRFk*v~v;S{L_B34&p8cOEr}q6Z3uStoHT;_^+7PCgM}Xe0GjhW9)wXT)Nr4jE8Fw z*DVQZ*@q99v?V28}S{sX`3AIFDo~Hh&_&;m$1Te^PBc~-c<$!kVYScD^8LH;%irHV zIAVaFCox%tje|D0gc3hbjT(_Z^Q$}kcHl0*Dzq<}5YH1-o&93l@dbHiQ}K6}5%EbM zPO!qGUYbNWPZ9BjtZ~w?#C^8A&3NQaJFuf-6MpARP=%om(Z2D*(!(bBF>U!pSH=kYw_jE!Zj6`c z4GF&6PQ-7Y>!-O6UzW}z9xO-b)XGI>Yw+~+HT|pP37z)Nvyy_ltNvUmzps#vc=lBZ z9h}RP-}pxzt31zx%{1}UXtTuytI_irMicFjP{uFnZv9?ylPC|idak4t{!8_WmTid? z!T-Fd#*!n%drRheHd5EI5nIbf+8UBmNp8SKsy-W;t4Xd$a$Ppkbl8ZkB)K-pmy=wR zjZ_UbGSx}0M)Ds>$=OZ~U8!;X>(zr;@Npg(j(-Q3Vm|`P! zlH}tgA7djn%0}8S$%jZjz(#668=1W%?;&{?$vfGIb+D1vPVzRAx3H1g%tmGt$s0*t z&qi7u8?joF*O0uL`Nb-9mzeDl_He&H?q{WgvhU8Ifq(-ul8BX#rlHXz@?Is(s8zjF@ z@(_~WW!oL|4jZW?(h^Abc+xIOq{gv1Etb?{NS#D#H0d5ix|2waWOG^ssfUv~iPSJs zzfHQ6NWDe6-((~62C0)s4JGyKq#i=*B+{;t?pH~75~;y#P75OSE2K^$HIUQ;NOuyc zmr3_aY-IYAI*HVaq<(?a{YafenlI_@L%NelrLj59o7B&fI*HVCq`MdCP9pUzo72va z`e{-pk?Kk69;AMXjZ_k8?xg!kHew`FPp~=dIH|jlI*HU{r2A3QokXfDo6}rK-I>%$ zq#hyl!=yWjR43B?5F42XNu5OM0aD*j>ibBYMA}}`-H~)Bk?O$aG<#CFBXts~dq~}u zbSIIzn{?mBM&?dZCy{DH>N`k%JE@aMvnJiGNOuycmTXSjM(SHhokXey>26NClSnmV zbJ`YC-%RQxQa6#hDXDK{Bb7v&3F*FpjTni)@Be?!3x0DZhP^ScX^+Bs<%#DVES>|) zHBhg4?YzB(cs?R{k-_2WD@l%g#PbZX2d~^9B4>}urg_Bk3YN@`eK)xKWwFx*CF1!9 zJH$n6FgV!qJi}n@%)3m;bZn2 z2u#cnWt`nia7=(YXy7mMI`Z=b@w|lD3o?fc979IxZb}o+uUPmDY{cLW|H*yrZ&n^hpiOYmPasT>d}B3mZv z!YaR#FvbbNZ>&!GS|}3wj#*4jFn2P%dUg*h{csT)7Z{%^*BQI5Md&|7sodc9%(-FM zWeLGK#p}62?w1ql z!l$HT4G~^dhA9Z4^NcN>p0oHJS^F6fU>f)B%0pIrAD5S%2`%f6rP{lR6aF_T*3X2= z{_~%P%USiMtrRZ=sX}iXcv@KgOO>g@P|#5-bs>gjKcy}8d2Vm4I-+_rNSxt=ASudzC*5E1A(xK=jSiFkj44bSNnfjOc(DosOK@k`7z z7X{wfiFPT53ohXo>HAe406sQ9ue!S_hpi$RYakJTp`R(suC z$P|O@)!vqY4J!$~P)1f9N+U%-_J3|C@+T@s69>2O?Wzh~EPIrfFBFG_!|8&)yIK60 zf`$Zmndt0P7-qHCJH-$Q*poiL#bMn>B0g!QUJ2;UHdv~x$;z*xGF1}9ee`0NY+>2& zgT?WZK+`<)G=s|WFRqHshAZJIeZNv!{pI2k``O@e!%79E9&QfkMPs|4=$kfbqqV8L!Ap=yUzNaS7p_w@T&N^@GEg{ zOJzE%Jx}WO&IM!Jvc{9_bbyoj&*X5Uof(u&tnfY{s{n(_ z(P0e_S?L8)HzCBqW1py8QHARFVG+Fb5m!^KCdMBcr@7}M@H@?rG3r+*xSB=$BG|oZ=KIx>OA*JywiYe|t?X4h zC-<@JhuacUg7Y?}UNS;_g#Qy(8cMMJ{oBm;#jNr^wBDfv16#8@158-?UAW^i(xsWR z1jJeP8)oxH3AipC%vBa6@`s7BRdwY;y@d8-IhxL^X&nCb&T9wb}TG1svP%WfAM0*t=4@**IOm;QhQZmVd8r z%Vy&l4`+^j-M5+0waca@@q^hZ;^lLR_f6Q(vI0q5D_PpDDq54^vE^}+xSPYg9P^~L z1h=R-Dv76KU%MBrUPW-uN|40GTJ5fAJZ9-0l`@jJ_Q6FfTN7F7DOL4I;G09ES0A}d zCH$9E;S#v+BhBeG7YqsRU42ag*L?3j+wijl!BuMPB=F$3&t}+6jM5P=s#zw158SwV zcd3jXp?iLx7RQU!zxnOSRwKA{ZGkvG;1l2T`Wx}S2@9-!E{^lz{iQJ#CInZj^A^Wn zitE|xn-TA0u&;Gy;<$o`-Qmh>tn^*$WyJBs5dXzX*2@z5%!VE@TkPRz8HH~&+xWud#<$YqtgnqJVnHb)%cKqrkMOJzh zO|!)C-Z6#l>8BeA-Q`D}D1KY5Q?TBLRsS5#&qeVR>xiE3uB`gG(d;dXTUd$g89vEM zPqD>J6c-gw<9&CbpQw-278z0e9iywxDW4Ud+S((6n?BO*_FAn)#Gl`ai{R9(#Z%mC zSoLY$c1;Aob-!%oJSmnvq_iCs!N2hSd=hWTYOhu8Ac8Bow~VKxs1xDw_E{o0*LkDK z6M8KF{-yXVJWp7li|4^cLJ#_8Hw(X7yGiJa39EiX%BF>J{Y$0AOnh|`m#(O5NAtZZ8FOJ3AzRS$ScKx&8$ z(M}NOw_KigXV8@3^^Ma4_>Il#DP?z9?fJSXP5`HlrrAw>WVKh|4-mj-Yew%rWYtWR zXGe1#KmM-uPTjFdR{JQmc=O{Ib2h#S^JJB;w?&2@FE4)bao&RkMEtK?k+pi(OD{V# zQ-|QXZQgwNrkI>lwsY1Ke7sGD509`8|333AEBu>woEPVKVaUro#mfKjpLV=>>)wQ+ zmK0WcsvXlj`0W)+{ByEc{ja_wjt6%u5*w>4VucUw1Rh*(-u11|5?SL3wX2RBuelnh zIE#mPpN(~Pd2{2!rlH?CiU1C?2qvL@L}1vSpPH! zzDW7_gPDe`{NC@6CTo1# z-3h-K77tVTg>hE?EW5m?7~d(@>*^<1^|7uSpJZ$Tbx?z$bAsM?=`2{Bt zUv7{01mn%2TjgJ@SoZU<7awO>McnzA8_wz<7y72h7$-w6aZ%e?>F?-=F~*#MTesp1 zSoWzn;62LluMgN;Ho%I%a{wP)(0@{aqnKn z!yoNe1jd&V{Me9p5953d!^%dMRbO+4y}KEu?}|Q0rm@N+GBVxC@Vyu{ee4a(zDGyA zI~XqCbwwsLS@9K)PPa3Z?*!g&abe{@e$2a#5z+D^y0?mD4~NF5TNtM8-8LFb*7znn z;oZ#0k8^up_)~^RKX+ofiShJQyWxTTtp1=eIo-%uG&eOyD~DCS+)3{S#@9rt^^ap& z_qX`*={iQaeWJc_9qWD(KQUdy_&g!HUf)ZQh>tcoUCD5-6W@H9#z%1K)N~mm;@PSn zcbP2v@12@1VQ_pMf4eGcCCmS>=|aX4DT-i=C#yZMU(=r%{eElREVr`8BigU&9LC$x z#)}R6SoI-0J)On4FJ@f9Jjf~^b$U9DVRxudXb}%9KeXxT2aK7*SGR3@#Ts8w7XH3w zzj@(U>ir(Nx9_`s|J`YWy)Un*8k@hu`PMiYtyM3>**@8yOCbJVv3w#)e|9gVeR&h9 z;t)k~pm`WXF6{+d{$1PrLfq&VyjFbCfs7`@lW0Jbg`w5B+CwknUJu9~_w#R-_Ob6==`+XsBZ>MQ)WMbICmAJV*3+6R<lhE-M|<-1qo{`E)tp|YJCu{_h4;5_|f*P0XJXQdw2#{iYx8&eJWc5nMx`!S&<*#o~S_T~lpo%o#$lUS^Q= zyt5zt6uzyWmFGs0Ow$zn#XA5$d^L*?5BgIyD>h9YlN$gp?m1UmqXQ|^v-hR)QwAW> zX5|Gr7BF1&_3fb3T`j=Do`1_A z6wrFO&qQsYY{#U#)*K#$Q-S-6Om9Y03MpBy?)VQvuVcvTyIChFwI$`fs&@y$no%_1 z_&I{2th(}B6+Q@!hMmvD)}EsXeV>X*tr>*no`>nxo#B)ROJh!@jtxT7cAH{>7dPpK z2MX>~%^rg88<|cQPhF$?n#EWz(H?>)loV;k85(_7sBPXoiy????&^s?6ilyDezns0 zDAI38b1Z!uL7B=4D-{bKf>!CsT@AN4gI*NWGX;Z0{&oJyW7+LpB*PilawfBM8 z_+hZxmnyFPJ&KN3&2SfE4#UhD(+8~sj?UU>~<-;AJC$EzAPKN$rjQ-ABe z_S=-8b+2Y?6^z28rgp78-;U8&dPqh*>l+2QzOEk#zQD)21@WF@5Z6fxiwEa^cwwX$f<|)U1gNW63YSdA(u=c9Ll)*wz=c?~GllxT6oG~1yR!Qy;CS>A z$eW6#M{m9OJ!#UEWm=jh&`qzf(&6h(d zuUdI;x?G!tb-QUB=ug}zP9Z%tvT2i0ZPH%!naiCr{?68FM%^SFkMZ5Mx+0kF6~MPq zjc*E~&zxSXLyxBS2MFg+1#?DD0A`+b_-M4N)P zx~?l(3$N2JYE^T!Bu#XZon0O_IR(2+=a_v- zyiQk17?Lnj`USTo&iE+54xOV%#;5XN~sFD zcjx)NUvOh_w3I@0EWPjGRkaNTzaZ$T=*xi{(RAgV4&C#|e?ju=d;WttF_f^3BRWeK zO~cx&LU>}r4GML=>#kGg(-4?Yq1I3mL6KlYKb<-|4FW583&q~X&?`b>eX<@*!@?Vh z1KT!SqV&04TasNe4Xvm5-#*#lNMYaqb=ey!G2OMl&m;Ve|NHy^-N)IN_ixlZ7})Av zVFh?RcRBx&C*ZLqn~PM@_N!Aaq1s&jsEt>rpW&^=>z>YUGtzk{o(<)6d@{Go}Lz>j;vfeGkosw zttnu;)dLs%i(1%}gZNefbYea_O>!&x^|3HyyZ`>noSt5heMs(3)vQm`WDLE)&yCJ; zLBAOHO!n_@-{^Cj-kiVv8&KJG)kEg+cNxN&Z13SzHvX;)He>(&y)W#y);mqQlX1!Z z(=+iu4gDaEhWs%RG7+*aT3)}28~W|DH@qW+euHfWMzdvH6FErqL6yP!z8bdozwKd1 z!uO{9@gEdSvssWXpA1sUL;jPiKTxoD&y~JACPT#1g+~rfc2h99IUA?%B*S~({g#^D z4HPU!^R#?ZGK`Cy37V8@rC`z`Z;TXDz&dz#T((0e1yjrwbl#f+LU(VJdZyM>uwqyD z(7P$H_uHuz9di+Ho;#V|lmZ%IQ**b>L!41}r%@pl1hTT^`4uZDm}9xi%zdd4bnv;w z?2jltuQ?7n_fny)J>wdEehme4cuU*+BNc=+W^AyFNAK-LDhoptUx2NZv@FcZrC_>O ztnhs=!0X((fxGaHg01Kj>$&#=&M%4Lujxne&5;t4ZGHhN8V1h|pMIub50p1-UGx$j z&t>|UD7~Oyd-QTI?tcjea(P$9^4cia3Fqq>i7#RQySOlD8v3v@azwMdC;xw2iVN403Sd8*(B~jq)@)n#d1-jrmvM($GYpr%Y zr$br5{8HX%)L)ca@4f4x!(n$h?Kihj{%1GmscB?FnbgCPJ3Yw0EF|+Ij%5N*MzcR2 z*hs;iy?Vlzk_neMUx%Nq=|J|!hbrAv*l5HI(4tC*8Koj z0LW!#qJ4;zTOYGde*ohr4M!I?^&c51yXa@zWpS z{sN7}(A!xQEIvQ~IW-3oB!pTurBV8g%ZK0lJIa)Q2cT!>+7>~Ajag@!^?wc zd@8E@!oi&jV`Vu4X-!2GjCnXgWqmG~N&I^GBmCG*qV|2dLtx`4kmEBq(^)l1!G0~nEv|fmq1Dpu zJMN+KY_x5R&;JD0di(WC`}&b?@oiWr4|=w{Jgl)ttW5Vf4*}@ z^*0c9K5ei7j;ed6)<`0K(Q@8>^9v!ye^fm4ET)fX%RU|5R0yJD>LPIHu+J|mNA_~pZq9S*Vvuq8F7RwWpFVaA zH?p@c2C-%1b7vnQ(o^72x?2plR%~;z)JFcd``^V{iec-kGt$p{(kYm1QDE=l5@@WI zNC|zjz4TIeu;|*Z!BzbFSfl9k{^?RU zvQ?zSC>Y(p3sA|^#0l0su|xBn6%ViNyz~un69SI#s3QM?1*6M9e}iDvC7w=; z(fupMxJ_wR8RX?07j!<&t&i#b$XULv49sC^6CdoO8Kz~0SYP9HU%rH@&U6`b&>0A16CX;&1{c*OnS z#yNTgtZn6TemITnyCD5VB1a|kUAwKub*~!T@2(|v8dbulTS1Q-&Z6)?0NIDX&KDtjOANhcWLpLFm|cdKBP_Du=qdNkhzA0vxc`9rUt)5^q|Ggqd!pnyLR5srPP9^nxM(B zXAK2A5Gjy(p%!>dp1!tyfbx6sqgi!+Ep&aZ@?3ZsmH&l;jG|;6Tygb26HtNr+f5bO zaN9a~;2Y6$=mWBMT=|#I{W>^pdx$x276R?9gwxYelX#8_L$2&d09{3_& zDONp3`s!yzq11XXTX!}6y2^ZgEN-e-e}6sbStQ%8+l|Jfro1dHupW$DH+)%RgXRwl zz=jvlBTiYyJ>Y=mu#aTvQADZ88!lN}Bpf5Dds!$r;AI0}IY&qKi z>2J&SmTIE%9GHqb^05J&%wGC)s7UBzi7#A+MH?Y_q2Q<9t;pV-ch&dXHp0Su#ShU= zBKnxevKRXvH3DN)`psHDl)uQupUg%Y!QGdx`QtmmFD`qtt_gU=WHaU}N$X>2mt^mR zHi6TxwVE?*QGV7t@rpGx!H)a-5(i5VH@|s)rp6Dj;alC_*Np04TGsZq_YYVYcsI#q z3bo(7nB%+oJ*m)z>ccZ&4fRkf3|*Z6wy+lU_nj^#NsLzD={suPK}C9U zmCaAdHfZ!tDZshV_`}C>VWw*vxPK`OhZm?m7KI*}o6`n6*Kbx+-h=WZvd(foIxk0W zXLg>e{ybDaLmJXv?Z8)PRy+9s?E{;7^-{H>9bC?t@D-m$@q3T%H(&7+mW-dW@#NyA zo}#YEo+G0rL z2(?ev5AQ?!r?1Y)-uVXYpI$7oAYB0Ac9SonHfSHU#N3zr?xTIw6=$EN^r3y!Ui^`x zdI+U@x2^I;`>AWE-kium`>8p@5+4X649*xTHAVZXf#c#tAlg^$9HTxz7wxMq-}z-Z z7s6|?tJhP|{_4B~+ssa&{nhU`c3V6|`>R9GG;gX#`>VNeQR+;DXHT0hS&jBtzq@-_ z_z>D>-BV!H9*Xu^t1V7@`xfo9zT+Btu@UXFrcOOG;zJnsq4Adr+HYNA&h>0#K@Y6O zo3}Zk{np;OdwMUT{nnCi4gKQLerqSM{WH_ierx~Bkyi@Pe(RCGm)H-q-};urr6a>= zzxCePKR)v!+|e8@Cz;a&x~u%0<WCqKJ9_NM_Rs0k5PVx$5s_S>;c2M z{mCEAaOyJ0r>KM;c;dVHuwq;fw8k&bx)+V&JwgAxAhHK$I<{X53hM#u zxR(P1H+z78k*LtNP{fmd7A0Nlfx6p^sG~v1ojcCTAg~9t9xWB|y3_-OHPJ397mz>N z>m#*3J;2Mcil56H*{?|rRmQ6aveM1Pm!Ix|papQw;8YLXj@{t3<^;0$_q@WCV<z+faGo+(K>h9%!tnwl3O)>_Q}y^MFYYxQlF)>0giR5zk3-H|zn; zmz20(gC2-k{CU!GHEQ23KRjQj2QJ$l{h_Aa1H$IPhx|2Aea!r}p=@anMCh&Xl~(Bi zlgb;SyBDGMUHWz4x?B(ZOn$Qnm+pZr!ZP=o=JtRh|NA4;l085>>Tp_2tOryj9)4ON z+yl|Cz-FOf4}2N;s3^sY`W2Kc=Hu*v?$5mr-BaB#y6W_cHzVD!N>XUm)}C%y6xzBq zx1}45p5JYnSBqBm)2UxfzIDUlIkJ70=zO*pTwEHIcikYP#eaA@y&JNQC7q8=?gpu) z>6_+0=mu)5UFy-8ZqP}X8=P>X8{(8-q(2Mjh8xz#5@Fum(7KM|xzoKH>~~X{qldde z#n$+vn|(LT*mP;{2di#)w$c6Az{YM^Ym@zc0=l6C+o4md-3{|wEMp^8y1~>?pg;qi z2dA`dASPa{8=S4)*0%F@L)v|f#^2KgV|v^dY8tzMS{RK7lyyN|_a@(2 zd0n8;w?EP)vkPR@jK1AX?t+1JmeVPByWs2g@uslbU7#wIu-Pi03-ZrA;`w>D3ugUr zF12#)0znHkpK$vwFqoWMoU*kGtZ%IgN?6wgN4H0xIJmM4tom9vaxLzHIR<#i$+=yi zm4D&uTfr{y(X<;cpXdZdRV~R+o#=i+3#-2Ry%V;)E$3JH+6j@Njn-j|PHd6gJgEbrP0l9cE&Kif|5s*7)Xy$OX|dE@ol z)t#_X^+V@%)lSIt`nF;fx=-bCCF-T{b;1UJ(>r5B9dJTtrkL1|4(R)AqBdUC0ltc7 zZl&NI(6ws!)wa7GaJ9_9OZ|KY@HDDjf3&>=LiX%FVWZIjdRvd()#2{|mjuT%YwCW2 z@wm3~iB~_tr1B~!?SkxSFN9rYxeM_JI_xjbIHn3FK>t0zczmy ze9#Voms(x3J=($4^r3R7Q9JkyIM3T7f$ryz6Xj(a+Mq8iDW?dXd*@$uZl$wF8%Rq} z>$OqQId?icbqxgCK=ERx>;ZJn-F{^i=k2#!;mkU!u)wZXxcOl8u-n2`C^N@4`q#IB zQF>A?C7}iKEtEF~*tdXZW$Avqg)N{eyQUqqiNu)eN2<^TV%iX@(;|3Kge$ znnBQS+p?s$KOpn&X-}`CKfpY8rR+YrA0WE(^l_7-CU`w@=drkV6I`1rNqwN&1cH`T zIjtp)&{s1c_X3@7R3TG%ws~PA#5~iuY|U(d%evAgm3tcCY1{KPtH8@ zZ8X+NQK|=qvM7(cYaOJd_^L=;ss)$f`%$NFe}{3d^eB2%4Ll0=D#^W74NK)2trmV& z(6KCBINPxj%&nJC$fD;MKE}3y?(Q-WI@Z|LcIz8h`#5AglP`sgS(eTVt`&p1TgTxA z#f8BCYetbx#upGhCenLv=_iN~bCxd)cn5oJUIYw9zDDmmC6+7QzfNI)Px=_i6X@jk zrZd@k0;vm7+3(MPuV1j(?{$ygh~CQowOT=XN2koNt`GQf*7X8UbE$sqQhgt9zjH-B zqwO}3?vUfQ0R6-P=^xBG-*4%Be;r8wpm-deGXL-rL`D!-5U|%C$n;Rc>;!**-sAWB zf(XW5r}3wvkAr7wIr7gW{h>94e_p*hZ1&&!z<=ul|E&-Fw?6RS`oMqd1OKfL{I@>v z-}=CR>jVF-5B#@2@Zb8tf9nJPtq=USKJefAz<=ul|E&-Fw?6RS`oMqd1OI=m50G!m zY5#a@{`1H!U;y z@dyr?N^napc!o3P1?)?ZNd#TZZ&uP$5Ai#97Hp6+4ujAAmleJ$Cg6#0>NUJX4nU@9 z%2sPvf4swz$iUy}pEr>RA=007FpQP{tPO^DSEKZWQ}FKhD1H9mS!dg$p20`YQNOat z7dQte_wTI#1AB2c`MpjeOJ(!lSV5LIodYH`)Gde@xVBF98Nm8YLQWM zieS-^hS;1aL(CiPG8sbu|Ni~W&mni>_W_B7-{1aQkQ=)l{%`h@ka_supcgb~ zPvA>ms?hICr4!|;pQCI*oJ)YNUYKx_;NX3F1D|oh`-u6rA3tlEIuxMZt z>wB(b5Az-ROV{d|j5zI^2kd>0tbItb+TX8`>DR*!55=4L%d=pOl;jZ_o%Ow8vJD@6 z|I#l7wM4!+@dWNB2&){3Vts#Ey~gk=GX8ql5*^~bbYl#jl1b}J)n$Ee8ZD*%)L-G< zcdp;>j=m4WC|IlDNC@lu*JN9^pZqJl+US#0n(_CbO6{ZdnmE??xYeiFUn2EF_;%1n zSVsB@p1gacAk2~VeE_mjj_!Zy=F4nm?N5lpl@=A<4Mv~CL6t@wK6jZ+pGok4T)*x4 zZznQ5{7ZP8OdtLweVoic{xA8*$@0U$lpiPS5B{b8aI*gKf2n_*Y(M;8+7Bn&AODy3 z$4UEvf3Y8&v_JS4`@>25g@3VMoV0)V7yHM_{saHge{iz@!N2rBob12wFZ~z)yQTl9 z|9bxnPxvSLKWSgRf2L39MF0Ow|0Dl|PW1o3^gqhaiXZj=zw|%qkI;$!Pe+KZ-aqS~ zrKA4;m;Oilv2@h`|I+_xf0mB=Kbe00pZ3GjQUCu-|HJ+Wo#_8${Plm@FQF6t|F7_W z*gq?N)c^kq|408}#gFqyG{*(f`TxnFRmG_4~f|x6>cv$DiT< z9Dn{yAO0A>2%Q+ee&-MV82?z||55%w$4^54XZ^t+<1Z`zf7Jh<<2Nh*f3zR`G5)jk zf3!dRasOcH|FEAw?>{X4ANB`-+`m}*KkOI&xc?D4asMORrx*UXe-ipX`_G^EUqb(9 z|NHa)&5HjY{rC6hLCY@dJatYy)^ zKM%uJHOThWp#1V(vfo&8JM}lBy4?Nt83Jb5l4le497~1ze%9&q?9H?3U>njC4j)*yo~fV8QS1 zuUm68kitF(g%i8^@L|)`)!XE`C!AP+t4#P3V;XtR2`8rcSiq%W#P@%(_a<;XZEyeg zPDK$ilMsatMUMGM>#NCBgpj#WMbl{@P6(Nf%pr3eLkGuv%%^YWnGPNE5aMJG4Q9{h zT5Ii8bpF5l|9kG&eLv6Zw|iZ^*Sgl)YhQa!d#%0qcWqm-u1P6nXIZ20No{A^igip% zDZ4jqbB%JHEY~g3Rl2>oG^G8lIdYv6UFExRHP6O3Uu}DuuS=q<%p5v1;=79*Z1ea! zB)UqlOOp*yGFu`tw&bbl#h6`yF3rL{F)_`LO3R9|v2pu85v8MtAk~yJth> zIwE??wyk3ioS!>Nt{bAK7&ft7c6M@-?RvgWh@MjB?~RAH8LqRP$kzqYS7u-NSX00C z0$Z^Th`wUzZhG&o&s@3ghrZG})wk)x?-S%YANoq!`0OQdm6L4a__`jYmC`pOlxf*Z zY%TaY9;KBZ`zDolowdaFEMK>yv@*5x`KkMwuCr~%*Xby&Y$_jWHs9%7@l2?3topRbwlZH= z!$7&CbFymHdJAoz@^v%}lwFUr4>jL2ORk$?pg0bDJz#R)Pqt#63-g++vOGJHAduIc1Cf z!tKo}t+sXM>r#|c&fOYT_R8l-Td@vBIc2BOR+}v$5w>F82_t3m0K-4$WW>pJCXAGT zTJwXfM*7QjC5)7vOYXEW3|nCP6JJN7yt1K8$MsWwiL@2#MwC}J&p-5++rZhjVx5Tc z_!?_B(})9!wnO>45apFi(|_>P+Z|~u)`6&?H0o+Nzj5e1+iQH?hYE^;*~U%>kIc3e z>pWCYo=^F+#jn0U%XJ+pDl5*-bL}u=w5?djp`vW}YXA9vp3jT^weWnA`X(FO)4u)n zNQQfP^ACX!^AV0=0b{~K@fq`BV{9NTAR)@IpiqBr9C=uXUv#9`@JM{7zF-sSSCsS8 zib_R(mt&5-N?da&%7g;86YD|P_@7wN|8+B5N5m%5`SM@Y3##Xk)i$O@h8de zvZ*wXewFj(cez!{?{ll)=~m?TxeI8|=Y{z6kK|SP9d4E3(yy|D^sC$~zQe7kl;7u8 zRLbvjD=HJkceoXm_R_CXexF-Wxm13STc!GaZk6xk_qJ84-{)58EyGo+-{)2-zt63x zl;7u8R65J=ajPsP{VEUhdA*{joFx4!?WJF3Y3WyaT7G9+rTTqtmHlM6%DU38@}c~` zw#seNuX3#Pt8|urmFoAoRm$&kD=K%$?`^A`B>gITOTS9>``jw!_qi37>i4--u9V-| zRv9V%D!WO)N;BzK`9^*}TV>3@{V3#fS_cXe+aaEs{-nQ0ZPPi<9-O?B zex)~#@7Alj;kFa(m(mX}#d_m+fEvA|rEaI#@z$<`{OhugN}6%M8F~0!eN(;tPX@I` zdP~pb&pOLO>dwx!e-3-hMvk4<70 z(wL)B;KKwP*qeA}i|Oz`*~}4Tj1*IA{0{QP`-$IOWJ^B$I&SRN8c6T%!F!J`vcq*| zRW`E0`FAeTs*&0b!9&AV+_ z&pLb6(K=~=u}eeW#c$Fy?x?(Rv8XxaGOP7+=9}i>mK_y?n@UaFD=c-Cd;Sfxmau0n zoBjyDn||fs|8^_R*Jj4gmlChC{>|%_8J>sy@zk}{>voMfj*Q=S@Nd{3pv^IVd5zW6 zysBn41M5Y{-q`eE$#qugmubV6XH~-Q-mAtecD%uEV6B%{BTybS8?QX~_y+5>Y}$+z=yJGZtE7-qu`N_?9SkQ?I#^vp*qr6V7o76Ca z#otZr5oFb%qjFO-xI#F7Prx98?)Sj& zEDxq!-DY!-{WQEwNa+vQ-}Y49m80*mG0R#2)8$FJH{UDJB|=$o&`zS0_WYNibuJ5Zco@xyX| zX4z!j7*;#KmvuXR^bgAI-7EUfWcoJmKN*fh{>Lu5y(KJx)ww&*DEdQtgxk$-c6})` z2@An@=h1#s2jpM1M1e;fOU5+Fq1z} z-<1=k=Gtv!Q!k~R?0>O7`hz3ZEx-Sb)pNY|=Ehj;pL98_lI1ow%5AIV%-8UHzIt-MY~uyH*z0xP=Qba04tpOBjnLoCR*hS2 ze&GoEr=)JPMvvUX#)c$S&eFy2zEd-Ml)Jo_O>*gR_JM-)(O5Lt)p9>;-QT}e*h}gk zHn{Izcz~VgG0I|S1^n(f<58(bj}NjlXPQm=?oX@>lN5AlmwgI*{1Myr zXKC4n(RTWSn9qrb?=F}2A*%sTm)ls}P+`tCNSVpzfv<{NG@hCM=k zOWM^Z^1Gv~RriC(49`=3e0pbb`Y7ua@X+4D4C8Yf!^%|Q z=X#G$F*A!-K^0n}erR-Ot_?ZOCJbu2Z}SuM-`Ty7?NQFKh>gB3fjKx{*xQ(g)6cN* zr053bpOIghX8Z4#KFebM=zFf(8Opz)^cC~Yvhd9h6P$NrorSqoyxvzj$I30*GVew! z^k>E&Hw{>Ijzt7rIcWY9?DHJoyJPk9EOSA#SAi>VJxD5b(`xB?Htpraxw%geJ~v^M zN4*P7nOo!Wldb5F!fw=yU2=g1EP9ncE&}~UnxWI6vT1BW!#?JNMk2qn-nWhIlE#i# z-nVbT2-L^q`u$o=OJiGJYsX#pf_-~O)cNsv8pAr6v${RP`6gdHckOc;i(T#Ec;YG$&na!2K=L+X!-7ulG$ zD>nzWM}Hq%Zq%W_E;6k_#gyBRe?WaWuwkdsCHDQ*-L;m?M|=KO|5d2}<6S~8*&j+~ zXI_UHlO=WDT@T)XyBK#^-Gtcw*Dm_;T zze;7G@T;6#TliJlTt3J5SDCt4_*J?M5Pp>zCc>}M^Zr@Bzsl^j!mlzcSol>c&4piO zZ2lR(ze;>kocjw%neol-Kkpy>=luiz{=$FWKlt|hE5)CGQ@bks{>y*fKM?O9sqar{ zkw1l>pB8`rg2D?we=VMW6#k$0557FV|IhmeMc@D8_2)nD9~8X*^Z)tx576F=Z10S% zrsw!BVnvjNJRJ`|?0dh_B>Db?7^fEsRx-v_tI#nxn2Gl+4t=N{fvTT?`kI8IdLAdkm5ZlL*%L@FJq6S^}`E3eYR?hsUN7Ozxz3rJs z234lBDP2E|I@RPd+i!4S^puz=RzJ#Ocinc|nNf`f0~Yp(V%A-pr%j%4nAr@{uQczz zFEjHQZfE%L7k2;pqw;rjd|2tzb?R+N-pbZk4~jp~W*=LZKfQ)((AiHWc_vIh)LQp}T^;S!Z?m+UV2xuE?;Rerp!9 zcOKt6^ty1B8D5+7`~B&2*kjYtI_}#Kv)Rr2z3x9}Iy2v3nOd>!36}kKkC{tOEPE8- zP=Di{-OQ)v49nfS#<1)I)-`6I+08s!^v%ePI>oAHu>qHIn-$pWcI>vUuKn5qJ5+n^ z`cz$TZ1#ZyeOjB1hB=pZ7x+Im-gVQh{dRU>+t{l^TmHq?WtBZYY3C-^s72K=o%Y{i z>6T50z3jb?1)IDYu&G)aTR84;gSbaYY~%4&cZ2$;F}Lrm<|N_^8CI`$rg|qFV|&+5 zu4%eEfoY~rW_v3fVK;pSw>B7YiFI6ZzK&~H@Ihf$HuKpuspo?%^{1EmK3 zI5>rwS+$wE$KX19*4nYV{iS_uo~CQ}KPp{gDIUXrKau%68})I`g88YJu)ba6>&8oV zu%>r(Tc*tTQ@&qyo%vUd-`#v>SV8`_<;(43z2jn=&I8@E6<>aNkHsl#tz(DXw+iB| z`YoXC+VC6fv}sz$eIFmNN|trj?+Cxf%Khfl?=99x+O)Gu+Ar}}Sk<@XwDVtPGMm5q zxO(D#zcP0wYq=zo88qK}ugUl6taWmm0e7+P&@$8G+u!fJ$n>XnezDT;E_>R_pw=Ff zyX`Oe@w~>e4>jM9pdV!ytEq=}Z z@M-mE+Oy{@$+$w99l5XB+PPP9jE(Rep4mN;Ox$zWrL!a2;TuHk*6=0`>i?d@#*hBB zK@EdfETmgf?>RMcSqo+3t+w@Fv6%GEjyf^9tj>hxbt^4+#pb@<(`2vL1NL&U+W2?D)a3PFL|9p|Gvw9_YI1gvC$;jjA8A!R}GVMdb?o17B@OxYxF#=8Ttj_qeyY ztn$?r1GjCgq^YSKiCmZaj@{3)nlSTI70o;M*`HcPd}On{%KICwuCDP2+vi$2L($xF zykjwJc`eQPka2+_imqnT%drk+{54d)NoYf2B=rax#~HI3+hJ8MM$TU)04Re!4n&Lh+#-{|pPvx0J{G^1hK>-}Z5k5*-x#}C}3ZPuWVb+z(&cD)j+*WVGo znjLg)c;NR-cT;9}czffm33I# zSDP7q?M}Vj33eW1M_nmh(*ehM+$8J3DXrII^U;CkKm4KE`+3pmJN6I7eX_##o-gxx zU$m)tfxVM{?6oc$k7OziAsI+B8%1-~mc0Z-{)Y{q2 zSsL}nRJ*l%Qr2#-+mR`$7rr~+^_bRUK{aj5pV}AL8}z*1wCv$u+uLtl;y91?CI$BD z+?>ARuwI!0dz&_ijOeESEM=NgoJW-=%e7;E4(QwHmn(LcU$%UngX_Tb#GA?OXJ}LQ z{+_&8D(#+?W|K^ z?@6^tm!lRrP;+)jPP8xJ&{##oO`N>T`UfJl&wnqHq6VqjKiphWm59X&b?ZGUM)mwfDv3u7%>p}As-^1QE zmnU@Ir#*P_@Zq;Zs}$I4zWw8?;lHcnsW0`{_l(KWS&sSz_J(%ccH`pM_jYPMvR-7r za^04{QeM0n(=gF|ruJIhpDjC8J!yArR;_J6EM3g{+`H5KQp~!P(3ZpdZ;bKM9&t4E zyxe1<-J6XWYn_jKv*kPHjQr((WQw7l%gx?zTWbC8e_r-f$H&fb<*gE9OcR4AleNO8RmL9hzzrfz5hbo2JnI28C z`zvH&&i%34@Cs4SYtKDmm+O!|uS)(R7F2h58^e$VDH9ijx@9%!rDf^CXYgSf0@+VTG6nkgB>zP48lSD8{Fzv-M2c?I^WdX*CzIbPPcJZvZ47hY6QZ`5$_c^OA{ z1kbYbucmWwL_e>u!Ur!LQ?JMM{&wYFv|2FwkEUORtLcdT{cC$={ubC*LjKyx{ADHN zubs?aRzm*T$^2y{MxV^ zx1c_hRDWgvP+)Hf{ezwCA6NDhCG`)o|1Ge$g#On~ z_P?xz{?|_SzpRA**G~4otc3oT$^N&%-je!Xd3`Iew}k7PoxHxW60UD{^7_U~xW3uR z>l-WK`o`q-t-#)ru5WVuP+)Hf;|C_k53GdogPj~duoA`(c5?i{N*F&dIesXxx1{ld z9G?}~Tf+E^$?+L0VSHvM$7igB@tK_*pRp3gXH1UI{!sg~lE!Cp{F|!AD`EVrhNqS= z{!NwR-_#Puzo~Nkn_9y7SF0XhQd7)d;Fa@>R8>*J{32D(FSI4hFH+_FLR-T8B2~^W zQcIX$XyyDu&NoY*Ue!TFUif78nOo3@1cn^w->v?a{nv~vEYEn)tqmGifP`DRJ; zH#t8pu(yQyu~yEHwI$4twQ_!}En$AFmGfh53G-vEoFB{l&1NOdkLCS`0((oi|Dcuk zAG9Uhf6&VN584v$KWOFs2W<)WAGGrRLqYzQbpJu#-zl)Sg!?;Md4ES+!u=hsyuYI@ z;r@)5`m2+7j-cY32PhZ3*|! zwDSI0LH?F>|4iPWEU>qP`;%IEe^Oh*{YkC7KdCL@{-jpkpVXFce^M*&PZro)()~&G z_iZXgzZNZjPk`TbX&m|Q+Z07o`efU*J>wOUYQlFTax${v@?NOpx)EW$gyBEA$)zZLqC#9lL)@eM`uq912ShoC2ln9!bn^;v_Vaf23hxva?&>wBTU1E5 zsGv?!{(bzydiq562p#3<=R@hb55$_(N*~2Z>8-R>zE_$lj*1KZXmj<8i170Fvk#5# zuXOf{?Bo*}ILfboV2E$%X!5&wMM#}nXjl}Cfv#_aUwBXN5q>_AT|0`f^m7gj@}pz>DbWzxnkhawPo5=kd*gh0ws3tY&jB7y zDIFU-HFjdGq8RH=jh#9vwyM6qlatewDO2?EcBDyF z9ebKt#!ipLUW#XcXFx!JfkBz*u&~i#Vfe@=;u|Ps!2fK>0kg0EEEV#(miT@>Sg z{EJh*{wuo2MCHq$j`FYl6;5H2^5w6T@~{1=`)#@>fddU;9({|N4CXs~UrUtpDn9O6I?K!ZMn%QZ)X*cD7$)87p7h!lk1_ zjUaa7q{(qprcRqa*}HH5frEdfXj2azK63Qf@e?Ocoj!B+-1!S>e_p(l{@3LzSFc^a zar4&gI~jNH-OqgR@b9cgkDok!_WVWm%U7>+a`W=vynXln!^cmb6`fMLditfy7#Nl< zXH>pIMdM19t5mI4y++MiCbdoL)U9V$zk#_$!$yrQziZOe%DP!|n-<@sL|15#ts@hWau!@lEZ(B88?2y|MdC) zPapr^&i_vK4vtRFox61H*4@RmM^Cq2z5BTL?bqL9z(A4zC5Qj7$p3hs&KM*0z*q?{ z9u;yQE#%IU?oQNTd^;nyi$d!U7{g8b(qPj9?tw}d{3(i+ba#cj72FQc_JP~6P@I55 z`niST?~;0V#J>r*daO5at9F|D2{+l(O1f!e=Puo3PmpwzopYs|>`9jHZg8KJZVVNb z4Cy92jfRW#$p(#dcZS?yL3Cj9V6Y8S4q-M_2H;=yTY9* z-DI2I2$2q*m!))5o^+9JDwiPXrgNMw-IV{^q*rrgUryxoIBar-Unk7^8bBAqW+7lfpVP#0XLRl_125Py)Ug$NHObq7Xk4 zseAE#)%3j)Hwftk$|H&+dtrv4YVZpp9EXlX3@l&>_Kbk2dev`@SfSj;k&4ov#<-hQj)99Z#GUSsV$WW79dJt5Bk#p%kq#(V3)RK{}az7AJQv-p0br_%Q3)`=1+9w$(qb8-C$Zm}2krc}N@ zW(cgIt@wTs$REm;0L2XZ4d=%fDSRE5@--0W8i<^v8eyi4fkYw&nm77`U+)v8jQr+t zMb3RIegwClEFoLJU0Mpa^h>zN(ZaQ~=>9=CN1E{l!%{lma9CSBjR<8l&MbnLsDf~6 zWx=o|1hvHr`-s+}wt}MJRBOr7NNzL5^y4{IJe_abQ8cZhc2IeXmi=`rruzM*^;*Ni zFJ};f^9+aI4`-s<{I!niYA8yG&XZb*g0n5YpEY!2q&|X|rf7MQ$P<5vX!Vp5^(A9* zG|CONKZ!s)CAWGGRKw|fLts1AC912%e=4RH?N3J-;?RNuulrntRSdbBScLD=+l z8lse`*CZ>y9sXtiF(t$y8=^`GqnZ|u+z2fwHR^AQ+e-N$>ZKQI?bqR>kP2OO#v&G# zgUC~ACv?4_E2uTU{)F(b5NIilFYQY0&(!{lNNKnPb>{c?I>9{{?#>wD&=?^IZW?dW z_#&FS2jaS~C~<}2BouOQgPX>fG_Fa7yNlADj}g+jejvYN__UBa2W~ZegWxZ2Q@Gt= z0gYu^!A)b3?r=N8JrE;38jpCw-Cd5?q6_I03h6h)?T$Yhf6e#qrW#q6szsQw2PDl2uY zVU3MI6M)XJ+PeW(@mF`O&v_?3!3Q-5C$Dx)i zu66%(O)VTo7K$^YI;LKS>1tE-I$Suc@IGH(j~n4Uie8VSa7L8pRKF=t=nVbgR`2Z= z4Hvy4#ipL56Ql;G1L~53Iv{9dWMWjtxQ(8Xfwi8lo{_P!K^+4fgh$78#(h8B!vJ-` z>7X7s7t{xnz|vqcSO(k%8i0F2LogLA3!VhafoY%-coi%UW`GsIEU+S&4bsZMZ@@~R zGOn{y8885=fW}}|uqIdyGy|)HmS9cL2CN0P15H2;SR3pDnu2a%9nb@;3wna}zyQz; z3<89c&2B1sj2hU}G=|{0>Y8n}FNErr=)C3QPs9!INNfFb!-2 zUIpockr`k+FbixCW`pzs&l?aANEBs!XGH@TfK-OYAkCR-g4C~=fzHgnGWip0>p!*z(i0N zTnp-fnd_VteQ+E6rNM(>8So@%0H%Y6pypR+r5u<6zY+KpEDyc`D}Z_vP#-{J&=@oY zD}kvUlnWxRs%I)b zGz0g7^}(ZH127FV2XBHFU>4XA%mEvLN-XLFXas%-)&!e?7GP7*2DAcgL2J+*Yz}&Y z--AJ5D=->t4aR|Oz`0;sa5>lxOa|M7yTBj7RImeh7PJGef(*2FbVYw9Y|GB4?K%Oc3w8hxf_C6Z&;d*bM}kj59duM5Kt0f4GU_o{6*L6Rz_Oqf zXau$cO+ZJm12`BQ2?l^V=+L4;JunU|1I`5v!R26CFc~xgcY!8gD%b%$3+kZ5y9w%n zSzsA32Q;LPFb?~JMxYT`6Ep!Wzz(1-sDqBt1vCUbz_Oq(Xat6VCSVNM0ZagO&{3`h z4Z&?-S@0le0-gjrfEl2U8R`{i2z~&|f(BC%4ps$qEMNy{2-<*UL0bw3yHL0#?4fYb zm%_m?(p$kE(u31U4}gFZ zfjS!G7x_U8q9g1ic7dHl7uZR3LpufaV$e=O6EF;nYd(zuw!!ZN z(luWXJP1FX=}B-JNMitfFdcqmAs-9S*dPP`HQ-Zl9{2{NYm?se&dN^E7~BNXHHNNj zrtr@MX$(;Yw1S_mVKkO70NcU;3+M=*1Kq&Q;9&3+7y!-(BfIJ9NY`PA4p>h8hadt zKM16;NFy)}{s1D*r!06Aeix9&8qJ9CM}RaYD+lJlp8zU9cUIN`M&K5(Cg=)UfV)8( z@E}NI1{$l_!atq#@Y9&43;Z-@pfR2i=ng-PJ!ou0V<1oXBS8)P=z5eO__u=5AdNj} zOjsU_gTE(8V<}5;F8m|FX!z@a%i;F|Y3x`5Ooks#mX9GTg1g{f2&RI)Kn435183o< z`CJD4*5Fn6!$Af1w*WHXj|RKI{{xr}|0K`=@hgEeMq=Ox=x2aNGigqSaOl4WYr;PU zOhWvIpauN?U?Th$pbh-PL0fP(NMpjLU>Epnu8^o^;wu9e;2!9RG5&i)njYVxi zH~6Q3gTW1805}Sa1eb$duwN}O7XCrtbod*9@$mbCiC_wt3?2dZg4@88;9l@5cm~V@ zPk?UN-voRE|6q{%_WPjTtj@||&=mXwYz3|dUBD%v2llHC`og~y38bZYKFV;q zZ$%s3%MqlrC%=eC&k|^(88vMIGJb%(&qVipXrnnYZFC=0?`&G2jJ$2gXxdd$y1?MB|qC17O1PbjA6JyJX}SgZVJ+_~ zQN6In2&yyUP~EWRPYy+Wp!zY8?@M||eto8TL2f7Lscu+f&)#rTeXvH&>&o49kB6R= zih5xUtqXTYLEnYvevpr^VD;M~Tf+jXGgQyXP4$ZEI=Lx* zs_&F<#m|N6zgl+Q++Z2^DoweGwC>1JQ1wE5EOM9L}RF5f-L>UD0Jng~ri27zq zmBvG&T~U8b^@eN`c2IvyDbux@`dX@Il#im#i1?z-2z?0OSCoxtC-hbT*-ZQPK}{}N zheR6??Y3||bKw1k=u1R-i*_Q)+n(oF;j%0oUp;>(J~LAL8z){D#C3=6zN_cliQ6a6 zxf5@@;+*LX2KAgBcwH`h4uy{=>=Wmp9@~xQl6q_xUQ*)NeYhPWzg^_9?zBABJ!;(KUGZbylS7$j_;8T1Cr% zo=~a1shU<#)LgYbcav_a&;7W)5wiCXC8zE)K%R>z6MMct?c*SiFIsHTeMCzXHrw-Z zQ1gP?ifCuu`2HdeJ$WI!DBVuH97G)PzLR?HG@cf5L`@cPy7O_ih~p;PtFVu*O(IUG zLgmqy*FB-PFBIRckiM5JOG-!d8sfP0bV;>AJnIsA@s5yi)6=NJ<2gDr(cX!opC1dV zwt3;4RJS+!c-8I8Mb~MCC-h^a&2;S$y_8R({z+)We|qw$ULVDE*_)>+#yMgXEUqu& zS(#{a^qz$^di*c@im?#ousRl^Cu!<6!k?cnjg_s@bB}~xv~}8Fb&F?iLQnryJw4z4 z;-*oa7@Jw6HHZjk~R=7nSQr$Fq74?rwNOjX22CCZ!`T50NSpOwHJug$^(9``dZpwF& zzCX`X;SRvnP1OhU`vO8w&*D_Q7(oktm^^oCSM=oZOB||aB953r2zL~(E5c1rv?&ga zOZ=pp#vc@_j@#%h5^~cxNz91E_({wHL?0|>9>OikOLbG*6K?viy6IgO)g8?9M2tnm zam5%&)Loi$S>xE0&tj~m>giffby^(`RRAM5^MdAZ9@{H==x@x+2Ca)Z$flFfU^UfmOSLlnzr#Y6I z59C(Qo#r1SWj;_k6hb|Km|v@Pg`UrUanpQI+*J_sDN){Ze}VK=u42w2%0tWt>AfPF z<4~M198sO`h>>#Pa}XNwzwkMTIp8Q>U&Y)_99N7hsV!3Pqh9aHPc`_9pX#}Y8^Dh* z+`+tV3wH!>Yr-ALui;`|D2{{O_xAm`ime!J-*9{PXB{bR+&!gRth)x~Ze^Bd+Vptg z|5oa^k=~-_^F6HJ0?)|lHTW%PsP>XRasOXhmU!~}ke|Ht?cm3jO8TtlmyfWib;{3l zJ^MzcbCmR5e%g0}mHVkxrK{m}Wqke}U$-lB&#(r04UF{{RF!(Zj@X;1K^NGZ8%-{k zKUqrZ(#I3+2miPYaE zeZqLZ43=#D>zi7`YfC-JH+wK7yIqzhyO&43K=ig{tMFM_p*;z%SP)fIhWIsdcK}dvXb$oMZQoi{(veauYa*7SIM^?RX@<@$C#b)T@s;WaYr zH#U^<`TA_T=7m3DdGXKM{nXu5#^?H{VYi>K_6zq1eQIwZeRr|UAZ2ypF?8@{iuKl9be8{Ev zR&P!}V=ZRYY8Pr^CGFvQ$6FT9nW4`g&G(m8>q8#J*OB{ZDW#K$oO0zb1m&T zGrfGRZhM2u(*AskA7*C$g2i|3G;3y5M;V{%)5=A>VAG?OYr878lJVd0_}f!oFhAen z+D4~jJBsCct16|lnReRn1KZYjlJVbC`~Yo}Y&Pm;nf+;bYW{P*g@2E1*4t_D-U?~m zW&C#(|7evTvsvSzcMbmR>Ll&qddKtgv)Qs$jl5PK?jqyAr}!?bc4o7gjk<@|>E$fr zbG_A^OWEv}dGoc6v()zcf#P3X_bQtW@i^yG>a%Q*v0Se_F@DJm($cq#J6}(x|B>Rq z@oV;yz35uoxBo(UUa?%CUZ?v@_Upyl70%C9+w~`k?@`PDB}@3dRg(oeYJ9HO8#d`B zn>6U$iK#l>Wcr^ee$CwFFIkJJyB^lHk>?f5^>gp;e#y=S_?W*M(pkn=G>C82B<&@; z)+;Nz?BDWyHC!K-p7oOXjhxtfOS2v_AGm(=1LYMve<|hBwh)cbr*eI^cgA*n_W(?Sp3?lant8Xdt$l%>HI;j*zA^7oi6?) z&o`Cp-Cu;gVhMeQcU{v*teUI}~UNQSN-@R+GShfod*S~4G^cAzaK5=czA8n=m zTyJ2pp zzGjo!oO=}DE&D?a*DIf_U$gTgt!f;uB+o0B>pc!&oyYAqxtk{COFL7!es8^=ui3II zv0XhbsP$Q&(oeM>_L^NiR3p87Sy^5huFw8E-V~if6Wp*zC1SX zlk8VhxjtybtkQkv3tX}kP2TazEXHNTx~j~cFz+qdyG^EqSQZ$U+S znLk`_^Lp=V*7kaGv&RE!3VkZqyFEM(`|AvPVi49y=#?^*{?&x^*DRn_yzBEfvOZ|I zezS7#HFLTgb}UFy?dSUCOaVNxIrU7*P60j~V1J z>m{uEfn(B64cEK1tCqtOEQ3;GuE~6h<@%(s`Z+A>#DhOI_hq|F<$6t+bq?E8sbA%% zZ)HEJ7*hIc+qKVOs~=2Af4Qr*I4=#?+n#aEVfnkO9FM)C)=#bv_|7ed8T9saJl#{a zi&Ub+4AGED7F4@{RiVMIc(ISo>qb0>h;Bl(m%OwPY$d7sgcXM z`3^FFxPEWtLpf~8pW|xYfA1pnv0R_(aXN>&gszz$zekQwQiVSH63Q{X_JyP+>iDEQ zrJodWGl#Wt?&?^ZNxg>a+jV=8!{V20O7V|S)8~5MkI!;grD};uV~pkaFqP}GX65BD zyH$-2*tC~+DitXGgb$x`m|oe0_qNB>`pNZ{-OA)L${xqlgobXUnTLWfgI1oa(%m-19Mr~ zlan_5xT%A%Gl}arr+MYF9(D4XPZ=cZe=65oZVt$0)n{~EYZ@%`EraWALPK-eye`IV zTi0_H=_r-R{w^k?a+%wnuFw1p>(wsp{CCB5H)Q%6u8(apIhQ>%Zxwgo z`>rCsC)X#;osr8zYtQ*BGgP+gSfS6DlgqO6Z&?2tC(}>j`Ye-0xhy^4mxP8!^17MI z^?I7+xvXyQP0>RqIEwT$xZcfeO)i^S{kRhGvs%8D$^KR@$*_OUu=-_FRQtK!xYd?i zwmEb159YIFIcm7x#b8%1`_X^)4~-0@ot|9Z<Xj;F|G_bT<+4p5re%KmRjofR0xs{LGl_NN!Q%=+!*OHD)7{O5X$2|2ke zI$^<^%6f9Vk-_yoiFOO}Vw8wd{I*wJU zk^PZBndh<3>#l6ovr_Y)>uv5g&SNv27Mcy2DBGik>#gjp^4R>Rx2Ljd@_OXS^-1wI zdF;T}z^*@5RIjI8fAoB-JoYfa>-B-()b`8uC-vIrvA452&5b`N>v<~IXEy7Y$Ih

-(_yX1!z zSC7d4O~dt1eO&X{x>pUW^{QM!m6pU24lHiPS}wt42UP7PhhOuTL?j;qul z`%Qy=^H}!#=j&E}Qtjt@W3zxfb|~yzy-PW=ztV8M&8d-jY~=F_rt`+hcIV0U%L795 z*oxm1_6~0=>uoI8$9@XWV~NY&lp6Oz>XW!$nK>$tHJy@MK5&s*f4E*#VQe0I;{DRT zaYtF-GPvGz`nWtc(BM?&xlFbHtV#Aye;b>}T5QRlk+?wW&A9%iS6m+JeQ$iVSywVe zd1<&l^3b$A)~koR;l`)3zIk$ebj_K0>~jB>dOnZT>nYb8`OnT{51Lx8{CQ2DNGFNw z9e!aQ9=r1hS>KdeWWQC< zWqIt3S#HXvkFwm&xZW{pWgdI6=v83I5E);?^~)Em&SMP|W-mC@QuZ63T<^AVZ5|ss z`=E1!elj0oxjuH^hCDW9+_IK7Z5nxA z8m_mE*@yb@_+-8OIBBOR*Cz}(n8#wqEw0o-tImtL-pWRs$JX19GJ1bcUPqF+-al-?;1PJhsVq z+NFeka$IM|_3od~=CS?1t+n~iMIE>8nvdyutpD$~Q!Xj|%wugsH$P~;N#>hkO7?%~btjK)=IT@Jl4T9D9`Sr zOedD>is7BZ4KAk{$`lZ z?5oTSsT-x%AFhwTRxY1?+>!1%)vBq;w^*+C)v1urmhO5`PXD-!pTza)7RLE3NN4-y zT4rVmp?=d+tm+X%xCw9w(9=p8JWI@>ys9m#2v))5uM)bWqduWOLc+&rCHZ)mFapE7-meAY3s zSGiti)c(^qw{S0jERW!3+^2n8maL2pIeys>Cvf&qO>sY+QShi zOLXS!#r5=-2E9+x7jIZtK;H%Of4gO*;+@?M^8ME*I*L-SloC6r)Hjp+gw+cv;H)J1 zhq!c7euU7bhNl+_C;bVJqTy-9!jJSX8eV+=9RvS4yvmS&4mThA&*9yM{d4#MuYV4Y z_x|Vb-+cZ#{G)Ht@Z$NS7gaR;WU>646jL<(XtD4!u|>nL77I6-R5ZMJ{+mxO8eX{k zseDVviEwhKA#O?az+y_#bc!FR&D4Jm|NiHH4lf`7&*8U=g*z6@hr9EN?q9qddMqd! zUOXS_FDe>dyd2UKiiW2aJHGX{qT$8MY4Dvdc^gp3T|8c5#+P^-3b|v79Zxs&pN|(- z)t-v9pgfD`$JrYH9G+I|pTlp|{^#&64T^>ruP;ioe-2M>Q8c``{pKx;h8Hi-dL2bL z<-sl6FYc$(t=9A6`R~}KXu5^%-agn~ISQfeTl?=DVy~Qq6b>)EKj~)?^#}*AQ-d}}JDjHtcezJQ~T+#4d#q9T= zW?ztZk%ipF&%@;BqUjVrk45n!oQ^pX?r)XHz0J@=Xj8+Bmq*scqT$8QXLYggZN<)K z=oPVlrqo9k3qSX&X!^y^v-{hk;l=a&Kv@TFKmB`V=umJT#qF$&urAxn6vXv}d)Gu= zC1L9q-R2!|$LcDYLh;T+w;A!iwT1j%M(MGyj4}f8CqZUI)okmklB!xkj;?8kaWmn zh_VaEgP1_9Aa;-*kN`+DWHuxTvJH{~xeUpIeoGc(eK-?jdAoC%+AZH<2kkY$x z9*{PWZjgQuUr0D41~M122C@Tk6mlMN9r6(J24b|QjA9D0fjC13Lq<9DEb`I>Lfd}iC2ek09@uIr4xwW0 z<<3!ozKRkWp6cTP8#XOkeAkfR`hh?@8=gDfmNWLqB|+R6z&@88x`c&#Vf>@l2TTXj@E|m8vA*)b8N-%UXk3z$pv6gmJxStx1kJZ~x9Bs2+@5rFOA)}G)N`0i{6YA^d z78*z?DxVZAA704cOUEfXP+j0#8U7_#MJYRB0db|a(kYxT^xY{eFrrhBK3$bRbzP%^ zA_MKmMEbdh_7C*+a|rMXSI+3GPWRA)aIDYJ+t16F3Q585ad|xuX)E<~9HYX5aDtJt zPT_aWqGro{L5bWWi`@N!!`!?g1K__*eo-dwfhh9`N|;KytRbyTp2`Xjg?WmRzS1bgj{H2;F=2j+zK)|`kYD6iS)L;H@(PcP3Zn+oU%8{- zAH~JlNh}KE};2oG-8*;^XHK8pTVcEX9|$O5j0y}7 z4GBh->4Q2L;va#OR`7cLt)P&iEsU36Jxa$h9818v@4 z%39i|lY<);aG(@>I=Pa2Cr`+A_ar>m=j%8JMMVTqM%qUWAC9gR4_FEvn^O2d zJvPOV&C@AbQGV6$?-dw{o~jQDJIGJ0$*)-Px`3w2Z4&G8uP9|@y{3crS5E7?qJJefnf^|;sNU z)?r7*?Kb@Rkm^oKYHcUwWD_SPthtjC5aFaGk9NZP3r>pXG$*{Bg79@tisj#MKXX#{ zzC^q{C#B|RCuOdSvtl{GSuq&wtjzVHyMAA`9uGAKw%5|4u81uv5$}RS)J+GD>X1@W zXhcqKJxEQ6KBPRPG{g`>qdX&sfh6h3Ulu}P zAf3x(2!+Q%$ewwS?;tCse>F((RO6#pn=ifxom;V8v|`Ee-Df42YRz<(_VGFTYCwL_c1F|p2#9vFZ9|11;a=7IQg z5x*p9CJplP3B}0b%jOk<&psF0ydu4d_=`nXf1NQNK-61CE~|$JQ4Y_ z6-!Y52`ebLteWkOa+Mu>|m8vPKUu8-9lEVM1 z@zr!xs{8+k|9%b>{ff%hzEZ_DUAd`8ZmK%|Z5o%jiKNX+-2UxD|8Gj5)Tvy!S9(SVq?_2S_&7NX>z~fqZ}{>zx%n zhylb1QWatfv4Ge>Y$1*iH;6mL12P!m3GsylK!PA)kVr^0BnA=-nGQ*SEQe71wUEt_ zU66y2laMsXO-L5xDI^>60b;ztxnQhs3bzHs3epN<3vq_tP|P9q7t|`ODSq6l2#g%>tkeN_fhJ%QSQ|_P>w#%t70?%~4`zZD zKu3_~j%MBD+hsQ;8T$1lY>E;V@8A3KutrjHr{6V$>_Nt z%}KjJPjh)Ukk5Ugr_-DcHUK@KHwOoU7N95C5cCBbfkB|y)N%708nR-0A;-}Llh;@o zG1l0zpnPI!0YR_CNUS-; z6%qtd_vwJJfskm(V#x21bjWK+CB(CV^ne6IV#9<@*)PTT->r{3V*TH5ZjS%n{stGZ zE-Zx3MYW=^Th&p$9*cUKfP&$3e?6QBJ`qdzO{gwtAfM5<#)1=(0F^u4k1>N#Vex%H zbyJ^Dt9~V^l%~LaQvZwl9slhV57I0*D$u(fsaW8)$f>NlV;3b>SKVd@Q)&sfp7vDj z*{#UE`q|m@wGUtFp)t!|`FmK?J=J;tg5*vIr;O~|xc#jBu#6b{ zgSTeie!JrRlkppSIjnv->i*eR-!+e$*vQe+xW>jeTa51?JNnM?$%gdT`A_b48@up= zli%4B^D3<0l2q&dFV5xtY8XXLoEkOiR2Nqd`^*gs+IMKZcgM3nt{WZ<>3D2NhnXwq z5Bxl2efDazRQGl9ZN2{dICJ34s||g{>7 zVHD0+Id}HlS+TumKXxp8<(Dw}_l6Dh?Qo~(ti>Uf@mrT8m#qA1-Seoi=g*fct)h8c z=h~e&AKUPpY1Z!!4BD#yN871SSj&zF?Z-a&D?xktZ+7_O$Marsl}4Z4l*yXA47%%j zG$mo-!&7Yj(LNob9M4rvS-O?o_TQh--s!ix+b<+A3-jam4({*J!fWPO=GAw=q4C@M z#8f^oj9q!&d{*g{`BSRBcVrf9-J|yF8b6+p+=PwWygKusZ;W%=t-f|XKb1af?ml|* z=Xn!5&U^Pmt+7509a^Mrwyi(oPb@dSc(3yDN{5@%Jq&7Z&$>T#y>5r&aZdJSb}xx) zd!yM8Eq^q>n5#V^ZWPr0(Ml|(0i5RfIY@m4ZCJqw`tY0 z&zAEe8_pS^w>I_m;c@S?j@MoJ_w<4N=hfKs@_5X!gJCzV^I=r7mn~CM#f`9aNSo9y?(9?ST^e&|*TG=>?RBW?l(ZNAxqr{`? zj*VM1vuSMRN6$R*_@rawzV6P}?HZd!;4X!)R}h|pbZk7vFQPHCD__>uDVhw!2VNOj1lT zOpcu>bIff~8}{PKVO^?*FQH8_JCI(g3I?6~aU-(IiVXAQb*&W!1k@9q^hds@GV z!6!F+y{6orJ!^8WxW0cJ$L;%{pT+I_cWZL{{&8`5yW98oxZ#I?*o%MpV-$EAzlb5d z$y~C8yiL9$rr{X@-W`AMg?48BrxZ6M1Hs)xk=#9<_?C_113dxOI&*<#M@Cc2`o(Q%|V_>QnVs zW!uClbl!EwxTeSS4o&>wJ{A-Qe#XlvGMdcCx;`M^lAlNmBg$~GwlT)-#$sc&u>tEl zYsAz3bRZpx^-Z8t>D}~xtgx7t(l_WX`e)ie+p&0NvZ+|X=KkxS%uxmS)<)6`nE4N<+$o?#!czqNm{ z`#bZo)6Y6@Ie&4!aGJYUx>c_B(!IIfaW7U6&}n*t&e0F(HOPWb^w+wD-_y7KOn*%< zA9A46@DfFakx|B1*z<98A}wW4iFd?4aY}qI23SL_d#%Tf}00 zoORA_=L4s$OWc+221M;XZ?K-Irv=wv7Q*GgokkCq!|&%C_)*?oEHamxCFZAATiHRr zF5i{)c6aAh=U8yf75E}EUS^Od@hIO^`T#vf!`U!4l4Y`)Y(94JIkuVIhdMrr+#Djd zih!A54mNYlb>@fWr{*~`+PcO{vr4VMS^corWpajEuAWepszxQ+H`x>HHTE|9bGx(+tU@U?M`qXa+kZS-DlnP?yGLOyUqQRd&vFF{mzZ>x_Y;I8QyfS z*n7h};a&9F=-$YRX?nTdtXukR{Hy&Devbc)zdLx{RhMe33t?m^nM4*K%i9^k7-BRT zJ!lm@&i=+bi3i2=qFi)0`Qp-0++*(D-g)nU{>TqBUecyvWFmQ#JZg-g%V=-5f+v`ZtwJkU zYyIVoaL9B;kdSO5+0@XCN=) zTX;wDlK4iB0C8(?!U9BI1yFpd`RD!j)u-WV4agvCIzam@JM*vv2T zGoqP!+`P?NC&Sb(wO1Xs>+K{b%$=e~`A_)4#v>m8zZJQTq>?OSgX}V^TnQW$*L$QZxX1bYSW}1`CEbL~EnQP`@ zPxH-%W&tw3&|GO2nZ?Nb^=7HL$t*L=&8_ARGufJMJ&&uBYJwW%OmOaWW?^?%I7Ln! zaOkhjwQe8erh}Y(-|g&m^O!fud&Ilo{o*y#t@P!(y}n9!*4=aub?3){f!P}hK25Zn-dnAJ1?Xw z=xcNbJwd;wtywhV%xB}-9c&4ElD)|)SRFgV+VGjEj@A4*?pdkURBNvFy!Dy2TD~Z^ zfSq1d=hPK8wddMT*jFJsHsX`!)Hz(Z}`oI@&My-}7t0_vif3-FouFLY&_QdMb$wM!t_B4*;E3kd@>OvH<-5 zlko}Vi0NeZ2$Vn%Zt@rTW?scV;(NtN^RQ`KIo31QLCXXRoRX`cG>V-rZln97*GXTa zXX*{QT6gwu^KlQd!m!Y@6WWtf5^GEd9owNvbTGY*wnaqVVMp0lY%TwuJ0e;1Gmn{9 zS^vWtVU4%)tU7Cy93$_L&&$u`WOa{v2WspS^_`-q`hE5pyRFm7xeifz$oU%h-NNnV zdhSi`?LdNu+&%8Kp5+bl(!BZJQg5yInz!BC6N=U`?=!E9P6v-~*M{%-6a43c`&JTm zNjs2UgafM!$P=W5yiUF&V~u-^g{b%%<8!FYX3%jj(Jk}s$sNsFkdGA|?tPib^p^QJZzOcSV{hkFU{RCzWlPzRxdAV#a z$Eo-2kKF!xkN&If<@bfwd&=MCAM-#e}@u&1F`YpYI-N%+AsvUR&r(AN)lc7al;T}+Sfw{swVxBb5 zm_M5>EX&HYW?2iYJ=QVnlogPzu&U0o8#r^I9EA+MUCxsa%W}D2c2u6aP2H}hsK?bS zP=1x_kUF6*x37er?q&D42VzH7+ZA@TebPRQ{K|4JLIYk644Uh{=I!#1dY^hxdN%am z!@5N8LYAMY|EuZg#dh2OZ*$hN?RQ#Ga?0 z&|m3({;mG)ey;zR|Fpl&KY$}N3w;*mWpaj`Cru=U-bBmj7&Zawbp`vCMRSKQw&uvw zs!??WemZtPyVhx`FVk)H6{!EKbr&#IwBH2|0ry?MKX!i%&XD8J_4ED5{bFSLPXB%X zsDHu_-E}Og89rN$&(vUVt|YOf2k?LpNj!1`83I(eg^Ys|m_}w|-{z5p$e9)7X=s?| z$wu-jd6Vn}OIDI9@*(+zd`{}o{hTKkNtn^vXlq0pu|_xSa$kdkY5GHP4l`~xMjM$% zmT{*s%ec?THy*~`KLH*39D0sTsIYCuE@MA1{V+KDr11qZ;yYx1fVQA*Xa^cYyU?EW zI`jpKYV-y}=?HXF`4`Ed|c(XGhox z)&f{IoX_L)p+dLw-O!L<@F-Djo> zK$eopMzQgh5k|K{d9P(}fCtXANZx~cJd@}0r+EqA$Pe*Typ4EHY=kbX6km%Q&1L3` z<`(k=IJCVr%(?^hw^6z zF1ntcrx)pEOrld*&8pbvY!(>d1E|+uM1<)DM#B4GbS@WR?A4YYJx67Q7?i9Dk zUFVj$+o7O8aL0R-y?eX@?>(>Hdrrq9Hw*pH-7%Azg`T(5+u-Pcqxg8JkmjO|aK!+T zDjr27wulPx9+bve@i);Sgr+&r$8D|d)--E2_UlP&FDie8{6)5d9v!GQsLiTcT>!#H zLhsMBKe0PI-JP}01}NT6?qYW<5OamM4p{lFcgpLf$NEc9AI}E&Dz#bYnOd94Nz%M+pZ7uCuhjGY^4--4VPhu&n0eTRLQeYZUa z%>RJ>nEivD4J~vWYI%m6hpxo*9B&vpkO#d*9@Un1^(bAe*XftRxN-iqzCb@Z2?$(_ z-NijdGn<7xibxWywv|SS@q)3@c-7cyw4qVJmoz#Sj>L4DgZeo}PtjkYR`S?u>}@zD zt&o>CPk-kC}flYs~XzmL=dZ+=;&JpnOS{qdWXWt+6-QuRGhF7u~b& z&u({b9@wZv_w|SPGyOnLv(P+by}8j0kRdg!?v)=`~_al_n=D8^C^gU0IJeAhoB->nJ<_>AP26qlC5>X{s)nFwtc~gal5+x z+`HYw=xn}plf4n%{fJa4b~;xt1!|X}bKj>2`8WGx{b~MJ@IXR$R4lx-@6CaQCVYwY zSkEEjV}nu~zS4SVx#RQ;8UZ|cnC)YS*jX0FFXtWkV*Vsw!`q6^qL-K`9v7?8&3z)Jco>Jsd_0#-J+(d995uJLUnFZ zr_}f0jW9a~PETL^Mtc}^ax$EWADkt~fDOpsN+`guy&uqm zhkKHiGz;C`_F=M|JVn;RS2<1E0F51B?JBwn=yMWRa~B*j6ACmRe%NbJ^M4T^o7Y;0 z)qcCie#_mBuD7|D>pkohd987tPUtUuJrFqZCe+1;`co~Tp6>7;0e`LYUx$XdgeCXF z4O>hegO1rkcA<){hZ{D`xXrj1zRqc2dOO+`IBp};u4Fx+-iNa^HVs^V1Z+Q==kR=f z9R0^nyoKl>s2Bz{KTZrW-!=cyV&n%Nqfm^bdf5L~0G%-fpF76d8#AfjU zI-y@gbM!%-&2H!r1v>k|=B?-gA2MGvx0_?-ba@uNhOKT?)3N7s)I(~iTBlxy9yth1 zs8Jj3a{IXbm7RtjS-A=D(gs1zYp4a>bGNQp=$Um-A}9A_7f;aB^k*7i?N~?FlO-~W zD@bBDhGIGz%3(UYo6Thla1F13ZFb-q4x&@K8j5NN>iRwYF<%6?VMi#sRlw95IHJ9* zpih+w2PNIgfVP@sWreV9hgAWb?Jb2&mAA?anF$0~Bwv-E%6i!VKk(ciJ=AcOrqWf0 z%2f9`UEO6~fgif-YGbp|^T|38i`+#XhYtLeBtReJ0ht=0tTW-Nr+%xV6e&`OTf##uSd-Z^Fxd*wog^WPA zF$>E5eW3Rz^hbI<6y!*LJL+{Ee;wU!j7Wqlaa25R9)yy-#=6!L))TmzpJWTwO-;hS zRHF{Bum{*f>{0N%K19dyiyi0mLTCPnv&X4%PC2dJD0i59i+i`f5QtmqzwIA{whGG#ie1KIc^{sHbu5LBO@@+9gI8eV9D%szArChL3g&9;<1=t@ zcCdB4im11v!f}5A~cWBn{s9yQGWJ+sH9? z8*S-3bTWLPr=gqQLq`$`lo}`20mVDvtdpQ9+E`tzu~7J};imd>fSfOP%WITU1Jn>` zj1#p^=!fT!Z)z+22#&^QQ8zshzyTs2}TZ*|0uHe_! zW9k~5X(u|pLv|fFaDlVa8Q{L)Z-I~8Jmj-p$NDnHEasrMz5#VR1eH4qwHtH-Ch&$3 zXI%^5VlHrbg}DaKAUaF=mOL+8sk3S?dg*}sj{6Z=8@5Q?z~s-i@%*QI(BII~=D z1#ecsfvMC7p(3l%UDoP4eM;Bs27LkhjV}-R5q_i}20W~~{XYp*XK`uO!d8n5{z7if_ zF;9XMnj(h+gVJRBZzz-nX3degVApxz)va=etdiBT20U@lLQ|* z#Tg3!D$PlEGT^;WLQjwlCo9*kh(NILOcAOpmha;Eb4aLOie;?C- zCxaXHxqr&yKw--eU)7LWQb+!0HJ^e~{k@`_(4U6W2pUO)x}X7blqMFASVkkqPD4&lLO$mrmlq;dfs&S?p6Bmz;$I&*ibkXRjiqr=#|bo%63VHhE?l%^nu0z! z6)HX*bI(ll!dWyMT0WQlz89&#@$f(AasgCFFrQJIbuB}_&|q}NBl#^n5QXoj)miK!b|W{z}5v|wbQB%DN`mZMP3 zF{tM_RCEGrnxLvB>N*LPor2m Date: Thu, 6 May 2021 15:58:21 +0300 Subject: [PATCH 122/652] remove contribs --- contrib/density/.gitignore | 2 - contrib/density/.gitmodules | 6 - contrib/density/.travis.yml | 272 --- contrib/density/CHANGELOG.md | 168 -- contrib/density/LICENSE.md | 28 - contrib/density/README.md | 261 --- contrib/density/appveyor.yml | 37 - contrib/density/benchmark/src/benchmark.c | 345 ---- contrib/density/benchmark/src/benchmark.h | 129 -- contrib/density/msvc/Density.sln | 21 - contrib/density/msvc/benchmark.vcxproj | 78 - .../density/msvc/benchmark.vcxproj.filters | 63 - contrib/density/msvc/density.vcxproj | 93 - contrib/density/msvc/density.vcxproj.filters | 135 -- contrib/density/src/algorithms/algorithms.c | 43 - contrib/density/src/algorithms/algorithms.h | 78 - .../src/algorithms/chameleon/chameleon.h | 70 - .../chameleon/core/chameleon_decode.c | 254 --- .../chameleon/core/chameleon_decode.h | 53 - .../chameleon/core/chameleon_encode.c | 179 -- .../chameleon/core/chameleon_encode.h | 53 - .../dictionary/chameleon_dictionary.h | 63 - .../density/src/algorithms/cheetah/cheetah.h | 73 - .../algorithms/cheetah/core/cheetah_decode.c | 266 --- .../algorithms/cheetah/core/cheetah_decode.h | 54 - .../algorithms/cheetah/core/cheetah_encode.c | 202 --- .../algorithms/cheetah/core/cheetah_encode.h | 54 - .../cheetah/dictionary/cheetah_dictionary.h | 70 - contrib/density/src/algorithms/dictionaries.c | 48 - contrib/density/src/algorithms/dictionaries.h | 45 - .../src/algorithms/lion/core/lion_decode.c | 327 ---- .../src/algorithms/lion/core/lion_decode.h | 54 - .../src/algorithms/lion/core/lion_encode.c | 298 ---- .../src/algorithms/lion/core/lion_encode.h | 54 - .../lion/dictionary/lion_dictionary.h | 73 - .../algorithms/lion/forms/lion_form_model.c | 153 -- .../algorithms/lion/forms/lion_form_model.h | 96 - contrib/density/src/algorithms/lion/lion.h | 94 - contrib/density/src/buffers/buffer.c | 220 --- contrib/density/src/buffers/buffer.h | 58 - contrib/density/src/density_api.h | 220 --- contrib/density/src/globals.c | 47 - contrib/density/src/globals.h | 232 --- contrib/density/src/structure/header.c | 57 - contrib/density/src/structure/header.h | 58 - contrib/lizard/.gitattributes | 21 - contrib/lizard/.gitignore | 37 - contrib/lizard/.travis.yml | 268 --- contrib/lizard/LICENSE | 15 - contrib/lizard/NEWS | 41 - contrib/lizard/README.md | 95 - contrib/lizard/appveyor.yml | 143 -- contrib/lizard/contrib/djgpp/LICENSE | 24 - contrib/lizard/contrib/djgpp/README.MD | 21 - contrib/lizard/doc/lizard_Block_format.md | 181 -- contrib/lizard/doc/lizard_Frame_format.md | 312 ---- contrib/lizard/examples/.gitignore | 8 - .../lizard/examples/HCStreaming_ringBuffer.c | 241 --- contrib/lizard/examples/README.md | 8 - .../examples/blockStreaming_doubleBuffer.c | 203 --- .../examples/blockStreaming_doubleBuffer.md | 100 -- .../examples/blockStreaming_lineByLine.c | 210 --- .../examples/blockStreaming_lineByLine.md | 122 -- .../examples/blockStreaming_ringBuffer.c | 202 --- contrib/lizard/examples/compress_functions.c | 303 ---- contrib/lizard/examples/frameCompress.c | 169 -- contrib/lizard/examples/printVersion.c | 13 - contrib/lizard/examples/simple_buffer.c | 91 - .../lizard/examples/streaming_api_basics.md | 87 - contrib/lizard/lib/.gitignore | 3 - contrib/lizard/lib/LICENSE | 25 - contrib/lizard/lib/README.md | 76 - contrib/lizard/lib/dll/liblizard.def | 19 - contrib/lizard/lib/entropy/README.md | 38 - contrib/lizard/lib/entropy/bitstream.h | 414 ----- contrib/lizard/lib/entropy/entropy_common.c | 231 --- contrib/lizard/lib/entropy/error_private.h | 115 -- contrib/lizard/lib/entropy/error_public.h | 64 - contrib/lizard/lib/entropy/fse.h | 694 -------- contrib/lizard/lib/entropy/fse_compress.c | 848 --------- contrib/lizard/lib/entropy/fse_decompress.c | 329 ---- contrib/lizard/lib/entropy/huf.h | 250 --- contrib/lizard/lib/entropy/huf_compress.c | 612 ------- contrib/lizard/lib/entropy/huf_decompress.c | 885 ---------- contrib/lizard/lib/entropy/mem.h | 372 ---- contrib/lizard/lib/liblizard.pc.in | 15 - contrib/lizard/lib/lizard_common.h | 504 ------ contrib/lizard/lib/lizard_compress.c | 630 ------- contrib/lizard/lib/lizard_compress.h | 208 --- contrib/lizard/lib/lizard_compress_liz.h | 301 ---- contrib/lizard/lib/lizard_compress_lz4.h | 162 -- contrib/lizard/lib/lizard_decompress.c | 372 ---- contrib/lizard/lib/lizard_decompress.h | 152 -- contrib/lizard/lib/lizard_decompress_liz.h | 220 --- contrib/lizard/lib/lizard_decompress_lz4.h | 163 -- contrib/lizard/lib/lizard_frame.c | 1362 -------------- contrib/lizard/lib/lizard_frame.h | 303 ---- contrib/lizard/lib/lizard_frame_static.h | 81 - contrib/lizard/lib/lizard_parser_fast.h | 196 --- contrib/lizard/lib/lizard_parser_fastbig.h | 175 -- contrib/lizard/lib/lizard_parser_fastsmall.h | 189 -- contrib/lizard/lib/lizard_parser_hashchain.h | 369 ---- .../lizard/lib/lizard_parser_lowestprice.h | 376 ---- contrib/lizard/lib/lizard_parser_nochain.h | 318 ---- contrib/lizard/lib/lizard_parser_optimal.h | 679 ------- contrib/lizard/lib/lizard_parser_pricefast.h | 250 --- contrib/lizard/lib/xxhash/xxhash.c | 888 ---------- contrib/lizard/lib/xxhash/xxhash.h | 293 --- contrib/lizard/programs/.gitignore | 15 - contrib/lizard/programs/COPYING | 339 ---- contrib/lizard/programs/README.md | 73 - contrib/lizard/programs/bench.c | 502 ------ contrib/lizard/programs/bench.h | 37 - contrib/lizard/programs/datagen.c | 188 -- contrib/lizard/programs/datagen.h | 39 - contrib/lizard/programs/lizard.1 | 229 --- contrib/lizard/programs/lizardcli.c | 581 ------ contrib/lizard/programs/lizardio.c | 896 ---------- contrib/lizard/programs/lizardio.h | 101 -- contrib/lizard/programs/platform.h | 145 -- contrib/lizard/programs/util.h | 497 ------ contrib/lizard/tests/.gitignore | 13 - contrib/lizard/tests/COPYING | 339 ---- contrib/lizard/tests/README.md | 15 - contrib/lizard/tests/datagencli.c | 192 -- contrib/lizard/tests/frametest.c | 866 --------- contrib/lizard/tests/fullbench.c | 732 -------- contrib/lizard/tests/fuzzer.c | 1086 ------------ contrib/lizard/visual/.gitignore | 11 - contrib/lizard/visual/README.md | 53 - .../visual/VS2010/datagen/datagen.vcxproj | 165 -- .../visual/VS2010/frametest/frametest.vcxproj | 177 -- .../visual/VS2010/fullbench/fullbench.vcxproj | 175 -- .../visual/VS2010/fuzzer/fuzzer.vcxproj | 174 -- .../VS2010/liblizard-dll/liblizard-dll.rc | 51 - .../liblizard-dll/liblizard-dll.vcxproj | 179 -- .../visual/VS2010/liblizard/liblizard.vcxproj | 176 -- contrib/lizard/visual/VS2010/lizard.sln | 86 - contrib/lizard/visual/VS2010/lizard/lizard.rc | 51 - .../visual/VS2010/lizard/lizard.vcxproj | 190 -- contrib/lzsse/.gitignore | 4 - contrib/lzsse/.travis.yml | 185 -- contrib/lzsse/LICENSE | 23 - contrib/lzsse/README.md | 15 - contrib/lzsse/example/main.cpp | 389 ---- contrib/lzsse/lzsse2/lzsse2.cpp | 1080 ------------ contrib/lzsse/lzsse2/lzsse2.h | 90 - contrib/lzsse/lzsse2/lzsse2_platform.h | 73 - contrib/lzsse/lzsse4/lzsse4.cpp | 1499 ---------------- contrib/lzsse/lzsse4/lzsse4.h | 117 -- contrib/lzsse/lzsse4/lzsse4_platform.h | 73 - contrib/lzsse/lzsse8/lzsse8.cpp | 1568 ----------------- contrib/lzsse/lzsse8/lzsse8.h | 117 -- contrib/lzsse/lzsse8/lzsse8_platform.h | 73 - contrib/lzsse/premake4.exe | Bin 518144 -> 0 bytes contrib/lzsse/premake4.lua | 26 - 156 files changed, 35085 deletions(-) delete mode 100644 contrib/density/.gitignore delete mode 100644 contrib/density/.gitmodules delete mode 100644 contrib/density/.travis.yml delete mode 100644 contrib/density/CHANGELOG.md delete mode 100644 contrib/density/LICENSE.md delete mode 100644 contrib/density/README.md delete mode 100644 contrib/density/appveyor.yml delete mode 100644 contrib/density/benchmark/src/benchmark.c delete mode 100644 contrib/density/benchmark/src/benchmark.h delete mode 100644 contrib/density/msvc/Density.sln delete mode 100644 contrib/density/msvc/benchmark.vcxproj delete mode 100644 contrib/density/msvc/benchmark.vcxproj.filters delete mode 100644 contrib/density/msvc/density.vcxproj delete mode 100644 contrib/density/msvc/density.vcxproj.filters delete mode 100644 contrib/density/src/algorithms/algorithms.c delete mode 100644 contrib/density/src/algorithms/algorithms.h delete mode 100644 contrib/density/src/algorithms/chameleon/chameleon.h delete mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_decode.c delete mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_decode.h delete mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_encode.c delete mode 100644 contrib/density/src/algorithms/chameleon/core/chameleon_encode.h delete mode 100644 contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h delete mode 100644 contrib/density/src/algorithms/cheetah/cheetah.h delete mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_decode.c delete mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_decode.h delete mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_encode.c delete mode 100644 contrib/density/src/algorithms/cheetah/core/cheetah_encode.h delete mode 100644 contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h delete mode 100644 contrib/density/src/algorithms/dictionaries.c delete mode 100644 contrib/density/src/algorithms/dictionaries.h delete mode 100644 contrib/density/src/algorithms/lion/core/lion_decode.c delete mode 100644 contrib/density/src/algorithms/lion/core/lion_decode.h delete mode 100644 contrib/density/src/algorithms/lion/core/lion_encode.c delete mode 100644 contrib/density/src/algorithms/lion/core/lion_encode.h delete mode 100644 contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h delete mode 100644 contrib/density/src/algorithms/lion/forms/lion_form_model.c delete mode 100644 contrib/density/src/algorithms/lion/forms/lion_form_model.h delete mode 100644 contrib/density/src/algorithms/lion/lion.h delete mode 100644 contrib/density/src/buffers/buffer.c delete mode 100644 contrib/density/src/buffers/buffer.h delete mode 100644 contrib/density/src/density_api.h delete mode 100644 contrib/density/src/globals.c delete mode 100644 contrib/density/src/globals.h delete mode 100644 contrib/density/src/structure/header.c delete mode 100644 contrib/density/src/structure/header.h delete mode 100644 contrib/lizard/.gitattributes delete mode 100644 contrib/lizard/.gitignore delete mode 100644 contrib/lizard/.travis.yml delete mode 100644 contrib/lizard/LICENSE delete mode 100644 contrib/lizard/NEWS delete mode 100644 contrib/lizard/README.md delete mode 100644 contrib/lizard/appveyor.yml delete mode 100644 contrib/lizard/contrib/djgpp/LICENSE delete mode 100644 contrib/lizard/contrib/djgpp/README.MD delete mode 100644 contrib/lizard/doc/lizard_Block_format.md delete mode 100644 contrib/lizard/doc/lizard_Frame_format.md delete mode 100644 contrib/lizard/examples/.gitignore delete mode 100644 contrib/lizard/examples/HCStreaming_ringBuffer.c delete mode 100644 contrib/lizard/examples/README.md delete mode 100644 contrib/lizard/examples/blockStreaming_doubleBuffer.c delete mode 100644 contrib/lizard/examples/blockStreaming_doubleBuffer.md delete mode 100644 contrib/lizard/examples/blockStreaming_lineByLine.c delete mode 100644 contrib/lizard/examples/blockStreaming_lineByLine.md delete mode 100644 contrib/lizard/examples/blockStreaming_ringBuffer.c delete mode 100644 contrib/lizard/examples/compress_functions.c delete mode 100644 contrib/lizard/examples/frameCompress.c delete mode 100644 contrib/lizard/examples/printVersion.c delete mode 100644 contrib/lizard/examples/simple_buffer.c delete mode 100644 contrib/lizard/examples/streaming_api_basics.md delete mode 100644 contrib/lizard/lib/.gitignore delete mode 100644 contrib/lizard/lib/LICENSE delete mode 100644 contrib/lizard/lib/README.md delete mode 100644 contrib/lizard/lib/dll/liblizard.def delete mode 100644 contrib/lizard/lib/entropy/README.md delete mode 100644 contrib/lizard/lib/entropy/bitstream.h delete mode 100644 contrib/lizard/lib/entropy/entropy_common.c delete mode 100644 contrib/lizard/lib/entropy/error_private.h delete mode 100644 contrib/lizard/lib/entropy/error_public.h delete mode 100644 contrib/lizard/lib/entropy/fse.h delete mode 100644 contrib/lizard/lib/entropy/fse_compress.c delete mode 100644 contrib/lizard/lib/entropy/fse_decompress.c delete mode 100644 contrib/lizard/lib/entropy/huf.h delete mode 100644 contrib/lizard/lib/entropy/huf_compress.c delete mode 100644 contrib/lizard/lib/entropy/huf_decompress.c delete mode 100644 contrib/lizard/lib/entropy/mem.h delete mode 100644 contrib/lizard/lib/liblizard.pc.in delete mode 100644 contrib/lizard/lib/lizard_common.h delete mode 100644 contrib/lizard/lib/lizard_compress.c delete mode 100644 contrib/lizard/lib/lizard_compress.h delete mode 100644 contrib/lizard/lib/lizard_compress_liz.h delete mode 100644 contrib/lizard/lib/lizard_compress_lz4.h delete mode 100644 contrib/lizard/lib/lizard_decompress.c delete mode 100644 contrib/lizard/lib/lizard_decompress.h delete mode 100644 contrib/lizard/lib/lizard_decompress_liz.h delete mode 100644 contrib/lizard/lib/lizard_decompress_lz4.h delete mode 100644 contrib/lizard/lib/lizard_frame.c delete mode 100644 contrib/lizard/lib/lizard_frame.h delete mode 100644 contrib/lizard/lib/lizard_frame_static.h delete mode 100644 contrib/lizard/lib/lizard_parser_fast.h delete mode 100644 contrib/lizard/lib/lizard_parser_fastbig.h delete mode 100644 contrib/lizard/lib/lizard_parser_fastsmall.h delete mode 100644 contrib/lizard/lib/lizard_parser_hashchain.h delete mode 100644 contrib/lizard/lib/lizard_parser_lowestprice.h delete mode 100644 contrib/lizard/lib/lizard_parser_nochain.h delete mode 100644 contrib/lizard/lib/lizard_parser_optimal.h delete mode 100644 contrib/lizard/lib/lizard_parser_pricefast.h delete mode 100644 contrib/lizard/lib/xxhash/xxhash.c delete mode 100644 contrib/lizard/lib/xxhash/xxhash.h delete mode 100644 contrib/lizard/programs/.gitignore delete mode 100644 contrib/lizard/programs/COPYING delete mode 100644 contrib/lizard/programs/README.md delete mode 100644 contrib/lizard/programs/bench.c delete mode 100644 contrib/lizard/programs/bench.h delete mode 100644 contrib/lizard/programs/datagen.c delete mode 100644 contrib/lizard/programs/datagen.h delete mode 100644 contrib/lizard/programs/lizard.1 delete mode 100644 contrib/lizard/programs/lizardcli.c delete mode 100644 contrib/lizard/programs/lizardio.c delete mode 100644 contrib/lizard/programs/lizardio.h delete mode 100644 contrib/lizard/programs/platform.h delete mode 100644 contrib/lizard/programs/util.h delete mode 100644 contrib/lizard/tests/.gitignore delete mode 100644 contrib/lizard/tests/COPYING delete mode 100644 contrib/lizard/tests/README.md delete mode 100644 contrib/lizard/tests/datagencli.c delete mode 100644 contrib/lizard/tests/frametest.c delete mode 100644 contrib/lizard/tests/fullbench.c delete mode 100644 contrib/lizard/tests/fuzzer.c delete mode 100644 contrib/lizard/visual/.gitignore delete mode 100644 contrib/lizard/visual/README.md delete mode 100644 contrib/lizard/visual/VS2010/datagen/datagen.vcxproj delete mode 100644 contrib/lizard/visual/VS2010/frametest/frametest.vcxproj delete mode 100644 contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj delete mode 100644 contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj delete mode 100644 contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc delete mode 100644 contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj delete mode 100644 contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj delete mode 100644 contrib/lizard/visual/VS2010/lizard.sln delete mode 100644 contrib/lizard/visual/VS2010/lizard/lizard.rc delete mode 100644 contrib/lizard/visual/VS2010/lizard/lizard.vcxproj delete mode 100644 contrib/lzsse/.gitignore delete mode 100644 contrib/lzsse/.travis.yml delete mode 100644 contrib/lzsse/LICENSE delete mode 100644 contrib/lzsse/README.md delete mode 100644 contrib/lzsse/example/main.cpp delete mode 100644 contrib/lzsse/lzsse2/lzsse2.cpp delete mode 100644 contrib/lzsse/lzsse2/lzsse2.h delete mode 100644 contrib/lzsse/lzsse2/lzsse2_platform.h delete mode 100644 contrib/lzsse/lzsse4/lzsse4.cpp delete mode 100644 contrib/lzsse/lzsse4/lzsse4.h delete mode 100644 contrib/lzsse/lzsse4/lzsse4_platform.h delete mode 100644 contrib/lzsse/lzsse8/lzsse8.cpp delete mode 100644 contrib/lzsse/lzsse8/lzsse8.h delete mode 100644 contrib/lzsse/lzsse8/lzsse8_platform.h delete mode 100644 contrib/lzsse/premake4.exe delete mode 100644 contrib/lzsse/premake4.lua diff --git a/contrib/density/.gitignore b/contrib/density/.gitignore deleted file mode 100644 index 5192545dd3f..00000000000 --- a/contrib/density/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -build/ -benchmark/build diff --git a/contrib/density/.gitmodules b/contrib/density/.gitmodules deleted file mode 100644 index 91eee619841..00000000000 --- a/contrib/density/.gitmodules +++ /dev/null @@ -1,6 +0,0 @@ -[submodule "benchmark/libs/cputime"] - path = benchmark/libs/cputime - url = https://github.com/k0dai/cputime.git -[submodule "benchmark/libs/spookyhash"] - path = benchmark/libs/spookyhash - url = https://github.com/k0dai/spookyhash.git diff --git a/contrib/density/.travis.yml b/contrib/density/.travis.yml deleted file mode 100644 index 62e7655a4cb..00000000000 --- a/contrib/density/.travis.yml +++ /dev/null @@ -1,272 +0,0 @@ -language: c -matrix: - fast_finish: true - include: - - os: osx - env: - - C_COMPILER=clang - - - os: osx - env: - - C_COMPILER=gcc - - - dist: trusty - sudo: required - addons: - apt: - packages: - - qemu-system-arm - - qemu-user-static - - gcc-aarch64-linux-gnu - - g++-aarch64-linux-gnu - env: - - C_COMPILER=aarch64-linux-gnu-gcc MAKE_OPTIONS="AR=gcc-ar NATIVE=off" QEMU_INTERPRETER=qemu-aarch64-static PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/aarch64-linux-gnu/" - - - dist: trusty - sudo: required - addons: - apt: - packages: - - qemu-system-arm - - qemu-user-static - - gcc-arm-linux-gnueabi - - g++-arm-linux-gnueabi - env: - - C_COMPILER=arm-linux-gnueabi-gcc MAKE_OPTIONS="AR=gcc-ar NATIVE=off" QEMU_INTERPRETER=qemu-arm-static PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/arm-linux-gnueabi/" - - - dist: trusty - sudo: required - addons: - apt: - packages: - - qemu-system-ppc - - qemu-user-static - - gcc-powerpc-linux-gnu - - g++-powerpc-linux-gnu - env: - - C_COMPILER=powerpc-linux-gnu-gcc MAKE_OPTIONS="AR=gcc-ar ARCH=64" QEMU_INTERPRETER="qemu-ppc64-static" PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/powerpc-linux-gnu/" - - - dist: trusty - sudo: required - addons: - apt: - packages: - - qemu-system-ppc - - qemu-user-static - - gcc-powerpc-linux-gnu - - g++-powerpc-linux-gnu - env: - - C_COMPILER=powerpc-linux-gnu-gcc MAKE_OPTIONS="AR=gcc-ar NATIVE=off" QEMU_INTERPRETER="qemu-ppc-static" PRE_SCRIPT="export QEMU_LD_PREFIX=/usr/powerpc-linux-gnu/" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-4.8 - env: - - C_COMPILER=gcc-4.8 MAKE_OPTIONS="AR=gcc-ar" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-4.8 - - gcc-4.8-multilib - env: - - C_COMPILER=gcc-4.8 MAKE_OPTIONS="AR=gcc-ar ARCH=32" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-4.9 - env: - - C_COMPILER=gcc-4.9 MAKE_OPTIONS="AR=gcc-ar" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-4.9 - - gcc-4.9-multilib - env: - - C_COMPILER=gcc-4.9 MAKE_OPTIONS="AR=gcc-ar ARCH=32" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-5 - env: - - C_COMPILER=gcc-5 MAKE_OPTIONS="AR=gcc-ar-5" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-5 - - gcc-5-multilib - env: - - C_COMPILER=gcc-5 MAKE_OPTIONS="AR=gcc-ar-5 ARCH=32" - -# - dist: trusty -# sudo: required -# addons: -# apt: -# sources: -# - ubuntu-toolchain-r-test -# packages: -# - gcc-6 -# env: -# - C_COMPILER=gcc-6 MAKE_OPTIONS="AR=gcc-ar-6" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-6 - - gcc-6-multilib - env: - - C_COMPILER=gcc-6 MAKE_OPTIONS="AR=gcc-ar-6 ARCH=32" - -# - dist: trusty -# sudo: required -# addons: -# apt: -# sources: -# - ubuntu-toolchain-r-test -# packages: -# - gcc-7 -# env: -# - C_COMPILER=gcc-7 MAKE_OPTIONS="AR=gcc-ar-7" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-7 - - gcc-7-multilib - env: - - C_COMPILER=gcc-7 MAKE_OPTIONS="AR=gcc-ar-7 ARCH=32" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-trusty-3.5 - packages: - - clang-3.5 - - llvm-3.5-dev - env: - - C_COMPILER=clang-3.5 MAKE_OPTIONS="AR=llvm-ar-3.5" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-trusty-3.6 - packages: - - clang-3.6 - - llvm-3.6-dev - env: - - C_COMPILER=clang-3.6 MAKE_OPTIONS="AR=llvm-ar-3.6" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - -# - dist: precise -# sudo: required -# addons: -# apt: -# sources: -# - ubuntu-toolchain-r-test -# - llvm-toolchain-precise-3.7 -# packages: -# - clang-3.7 -# - llvm-3.7-dev -# env: -# - C_COMPILER=clang-3.7 MAKE_OPTIONS="AR=llvm-ar-3.7" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - -# - dist: precise -# sudo: required -# addons: -# apt: -# sources: -# - ubuntu-toolchain-r-test -# - llvm-toolchain-precise-3.8 -# packages: -# - clang-3.8 -# - llvm-3.8-dev -# env: -# - C_COMPILER=clang-3.8 MAKE_OPTIONS="AR=llvm-ar-3.8" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-trusty-3.9 - packages: - - clang-3.9 - - llvm-3.9-dev - env: - - C_COMPILER=clang-3.9 MAKE_OPTIONS="AR=llvm-ar-3.9" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - - - dist: trusty - sudo: required - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-trusty-4.0 - packages: - - clang-4.0 - - llvm-4.0-dev - env: - - C_COMPILER=clang-4.0 MAKE_OPTIONS="AR=llvm-ar-4.0" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - -# - dist: trusty -# sudo: required -# addons: -# apt: -# sources: -# - ubuntu-toolchain-r-test -# - llvm-toolchain-trusty-5.0 -# packages: -# - clang-5.0 -# - llvm-5.0-dev -# env: -# - C_COMPILER=clang-5.0 MAKE_OPTIONS="AR=llvm-ar-5.0" PRE_SCRIPT="sudo ln -sf ld.gold /usr/bin/ld" - -script: - - ${PRE_SCRIPT} - - $C_COMPILER -v - - make CC=$C_COMPILER $MAKE_OPTIONS - - file ./build/* - - $QEMU_INTERPRETER ./build/benchmark -h -f diff --git a/contrib/density/CHANGELOG.md b/contrib/density/CHANGELOG.md deleted file mode 100644 index 6d5535bcc7c..00000000000 --- a/contrib/density/CHANGELOG.md +++ /dev/null @@ -1,168 +0,0 @@ -0.14.2 ------- -*February 12, 2018* -* Improved chameleon decode speed -* Added data hash checks and display option in benchmark -* Now using makefiles as build system -* Big endian support correctly implemented and tested -* Improved continuous integration tests - -0.14.1 ------- -*January 20, 2018* -* Added MSVC support -* Added continuous integration on travis and appveyor -* Premake script improvement -* Various codebase improvements - -0.14.0 ------- -*January 16, 2018* -* First stable version of DENSITY -* Complete project reorganization and API rewrite -* Many stability fixes and improvements -* Fast revert to conditional copy for incompressible input -* Custom dictionaries in API -* Improvements in compression ratio and speed - -0.12.5 beta ------------ -*June 20, 2015* - -* Added conditional main footer read/write -* Improved teleport staging buffer management -* Regression - a minimum buffer output size has to be ensured to avoid signature loss -* Modified the minimum lookahead and the resulting minimum buffer size in the API -* Lion : corrected a signature interception problem due to an increase in process unit size -* Lion : corrected chunk count conditions for new block / mode marker detection -* Lion : modified end of stream marker conditions -* Stability fixes and improvements - -0.12.4 beta ------------ -*May 25, 2015* - -* Removed remaining undefined behavior potential occurences -* Implemented parallelizable decompressible output block header reads/writes (disabled by default) - -0.12.3 beta ------------ -*May 20, 2015* - -* New lion algorithm, faster and more efficient -* Compiler specific optimizations -* Switched to premake 5 to benefit from link time optimizations -* Various fixes and improvements - -0.12.2 beta ------------ -*May 4, 2015* - -* Added an integrated in-memory benchmark -* Better Windows compatibility -* Fixed misaligned load/stores -* Switched to the premake build system -* Performance optimizations (pointers, branches, loops ...) -* Various fixes and improvements - -0.12.1 beta ------------ -*April 3, 2015* - -* Better unrolling readability and efficiency -* Improved read speed of dictionary/predictions entries -* Implemented case generators in cheetah to speed up decoding by using less branches -* Added signatures interception in lion to cancel the need for large output buffers -* Improved lion decode speed with specific form data access and use of ctz in form read -* Enabled decompression to exact-sized buffer for all algorithms -* Various fixes and improvements - -0.12.0 beta ------------ -*March 24, 2015* - -* Added new lion kernel -* Renamed kernel mandala to cheetah -* Kernel chameleon and cheetah improvements in encoding/decoding speeds -* Generic function macros to avoid code rewrite -* Improved memory teleport IO flexibility and speed, bytes issued by memory teleport can now be partially read -* Various fixes and improvements - -0.11.3 beta ------------ -*February 5, 2015* - -* Added integrity check system -* Corrected pointer usage and update on footer read/writes -* Now freeing kernel state memory only when compression mode is not copy -* Updated Makefiles -* Improved memory teleport -* Fixed sequencing problem after kernels request a new block - -0.11.2 beta ------------ -*February 3, 2015* - -* Added an algorithms overview in README -* Removed ssc references -* Now initializing last hash to zero on mandala kernel inits -* Reimplemented the buffer API -* Various corrections and improvements - -0.11.1 beta ------------ -*January 19, 2015* - -* Added a sharc benchmark in README -* Stateless memory teleport -* Improved event management and dispatching -* Improved compression/decompression finishes -* Improved streams API -* Various bug fixes, robustness improvements - -0.10.2 beta ------------ -*January 7, 2015* - -* Improved organization of compile-time switches and run-time options in the API -* Removed method density_stream_decompress_utilities_get_header from the API, header info is now returned in the density_stream_decompress_init function -* Corrected readme to reflect API changes - -0.10.1 beta ------------ -*January 5, 2015* - -* Re-added mandala kernel -* Corrected available bytes adjustment problem -* Added missing restrict keywords -* Cleaned unnecessary defines - -0.10.0 beta ------------ -*January 2, 2015* - -* Complete stream API redesign to greatly improve flexibility -* Only one supported algorithm for now : Chameleon - -0.9.12 beta ------------ -*December 2, 2013* - -* Mandala kernel addition, replacing dual pass chameleon -* Simplified, faster hash function -* Fixed memory freeing issue during main encoding/decoding finish -* Implemented no footer encode output type -* Namespace migration, kernel structure reorganization -* Corrected copy mode problem -* Implemented efficiency checks and mode reversions -* Corrected lack of main header parameters retrieval -* Fixed stream not being properly ended when mode reversion occurred -* Updated metadata computations - -0.9.11 beta ------------ -*November 2, 2013* - -* First beta release of DENSITY, including all the compression code from SHARC in a standalone, BSD licensed library -* Added copy mode (useful for enhancing data security via the density block checksums for example) -* Makefile produces static and dynamic libraries diff --git a/contrib/density/LICENSE.md b/contrib/density/LICENSE.md deleted file mode 100644 index c2b699c9b83..00000000000 --- a/contrib/density/LICENSE.md +++ /dev/null @@ -1,28 +0,0 @@ -Copyright (c) 2013, Guillaume Voirin - -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -2. Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -3. Neither the name of the copyright holder nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE -FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER -CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, -OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/density/README.md b/contrib/density/README.md deleted file mode 100644 index 6c660dea1f5..00000000000 --- a/contrib/density/README.md +++ /dev/null @@ -1,261 +0,0 @@ -DENSITY -======== -Superfast compression library - -DENSITY is a free C99, open-source, BSD licensed compression library. - -It is focused on high-speed compression, at the best ratio possible. **All three** of DENSITY's algorithms are currently at the **pareto frontier** of compression speed vs ratio (cf. [here](https://github.com/inikep/lzbench/blob/master/lzbench18_sorted.md) for an independent benchmark). - -DENSITY features a simple API to enable quick integration in any project. - -Branch|Linux & MacOS|Windows ---- | --- | --- -master|[![Build Status](https://travis-ci.org/k0dai/density.svg?branch=master)](https://travis-ci.org/k0dai/density)|[![Build status](https://ci.appveyor.com/api/projects/status/rf7x3x829il72cii/branch/master?svg=true)](https://ci.appveyor.com/project/gpnuma/density/branch/master) -dev|[![Build Status](https://travis-ci.org/k0dai/density.svg?branch=dev)](https://travis-ci.org/k0dai/density)|[![Build status](https://ci.appveyor.com/api/projects/status/rf7x3x829il72cii/branch/dev?svg=true)](https://ci.appveyor.com/project/gpnuma/density/branch/dev) - -Why is it so fast ? -------------------- - -One of the biggest assets of DENSITY is that its work unit is **not a byte** like other libraries, but **a group of 4 bytes**. - -When other libraries consume one byte of data and then apply an algorithmic processing to it, DENSITY consumes 4 bytes and then applies its algorithmic processing. - -That's why DENSITY's algorithms were designed from scratch. They have to alleviate for 4-byte work units and still provide interesting compression ratios. - -**Speed pedigree traits** - -* 4-byte work units -* heavy use of registers as opposed to memory for processing -* avoidance of or use of minimal branching when possible -* use of low memory data structures to favor processor cache Lx accesses -* library wide inlining -* specific unrollings -* prefetching and branching hints -* restricted pointers to maximize compiler optimizations - -A "blowup protection" is provided, dramatically increasing the processing speed of incompressible input data. Also, the output, compressed data size will **never exceed** the original uncompressed data size by more than 1% in case of incompressible, reasonably-sized inputs. - -Benchmarks ----------- - -**Quick benchmark** - -DENSITY features an **integrated in-memory benchmark**. After building the project (see [build](#build)), a *benchmark* executable will be present in the build directory. If run without arguments, usage help will be displayed. - -File used : enwik8 (100 MB) - -Platform : MacBook Pro, MacOS 10.13.3, 2.3 GHz Intel Core i7, 8Gb 1600 MHz DDR, SSD, compiling with Clang/LLVM 9.0.0 - -Timing : using the *time* function, and taking the best *user* output after multiple runs. In the case of density, the in-memory integrated benchmark's best value (which uses the same usermode CPU timing) is used. - -Library|Algorithm|Compress|Decompress|Size|Ratio|Round trip ----|---|---|---|---|---|--- -**density** 0.14.2|Chameleon|0.092s (1085 MB/s)|0.059s (1684 MB/s)|61 524 084|61,52%|0.151s -lz4 r129|-1|0.468s (214 MB/s)|0.115s (870 MB/s)|57 285 990|57,29%|0.583s -lzo 2.08|-1|0.367s (272 MB/s)|0.309s (324 MB/s)|56 709 096|56,71%|0.676s -**density** 0.14.2|Cheetah|0.170s (587 MB/s)|0.126s (796 MB/s)|53 156 668|53,16%|0.296s -**density** 0.14.2|Lion|0.303s (330 MB/s)|0.288s (347 MB/s)|47 817 692|47,82%|0.591s -lz4 r129|-3|1.685s (59 MB/s)|0.118s (847 MB/s)|44 539 940|44,54%|1.803s -lzo 2.08|-7|9.562s (10 MB/s)|0.319s (313 MB/s)|41 720 721|41,72%|9.881s - -**Other benchmarks** - -Here are a few other benchmarks featuring DENSITY (non exhaustive list) : - -* [**squash**](https://github.com/quixdb/squash) is an abstraction layer for compression algorithms, and has an extremely exhaustive set of benchmark results, including density's, [available here](https://quixdb.github.io/squash-benchmark/?dataset=dickens&machine=s-desktop). - -* [**lzbench**](https://github.com/inikep/lzbench) is an in-memory benchmark of open-source LZ77/LZSS/LZMA compressors. - -* [**fsbench**](https://github.com/gpnuma/fsbench-density) is a command line utility that enables real-time testing of compression algorithms, but also hashes and much more. A fork with density releases is [available here](https://github.com/gpnuma/fsbench-density) for easy access. -The original author's repository [can be found here](https://chiselapp.com/user/Justin_be_my_guide/repository/fsbench/). - -Build ------ -DENSITY can be built on a number of platforms, via the provided makefiles. - -It was developed and optimized against Clang/LLVM which makes it the preferred compiler, but GCC and MSVC are also supported. Please use the latest compiler versions for best performance. - -**MacOS** - -On MacOS, Clang/LLVM is the default compiler, which makes things simpler. - -1) Get the source code : - -``` - git clone https://github.com/k0dai/density.git - cd density -``` - -2) Build and test : - -``` - make - build/benchmark -f -``` - -Alternatively, thanks to the [Homebrew project](https://brew.sh), DENSITY can also be installed with a single command on MacOS: - -``` - brew install density -``` - -**Linux** - -On Linux, Clang/LLVM is not always available by default, but can be easily added thanks to the provided package managers. -The following example assumes a Debian or Ubuntu distribution with *apt-get*. - -1) From the command line, install Clang/LLVM (*optional*, GCC is also supported if Clang/LLVM can't be used) and other prerequisites. - -``` - sudo apt-get install clang git -``` - -2) Get the source code : - -``` - git clone https://github.com/k0dai/density.git - cd density -``` - -3) Build and test : - -``` - make -``` -or -``` - make CC=gcc-... AR=gcc-ar-... -``` -or -``` - make CC=clang-... AR=llvm-ar-... -``` -to choose alternative compilers. For a quick test of resulting binaries, run -``` - build/benchmark -f -``` - -**Windows** - -Please install [git for Windows](https://git-scm.com/download/win) to begin with. - -On Windows, density can be built in different ways. -The **first method** is to use mingw's gcc compiler; for that it is necessary to download and install [mingw-w64](https://sourceforge.net/projects/mingw-w64/). - -1) Once mingw-w64 is installed, get the source : - -``` - git clone https://github.com/k0dai/density.git - cd density -``` - -2) Build and test : - -``` - mingw32-make.exe - build/benchmark.exe -f -``` - -As an alternative, [MSYS2](http://www.msys2.org/) also offers a linux-like environment for Windows. - -The **second method** is to download and install Microsoft's [Visual Studio IDE community edition](https://www.visualstudio.com/thank-you-downloading-visual-studio/?sku=Community). It comes with Microsoft's own compilers and is free. - -1) Once Visual Studio is installed, open a [developer command prompt](https://docs.microsoft.com/en-us/dotnet/framework/tools/developer-command-prompt-for-vs) and type : - -``` - git clone https://github.com/k0dai/density.git - cd density\msvc -``` - -2) Build and test : - -``` - msbuild Density.sln - bin\Release\benchmark.exe -f -``` - -An extra **recommended step** would be to install *Clang/LLVM* for Windows. It is downloadable from [this link](http://releases.llvm.org/5.0.1/LLVM-5.0.1-win64.exe). Once installed, open the Visual Studio IDE by double-clicking on *Density.sln*, then right-click on project names and change the platform toolsets to *LLVM*. Rebuild the solution to generate binaries with Clang/LLVM. - -Output format -------------- -DENSITY outputs compressed data in a simple format, which enables file storage and optional parallelization for both compression and decompression. - -A very short header holding vital informations (like DENSITY version and algorithm used) precedes the binary compressed data. - -APIs ----- -DENSITY features a straightforward *API*, simple yet powerful enough to keep users' creativity unleashed. - -For advanced developers, it allows use of custom dictionaries and exportation of generated dictionaries after a compression session. Although using the default, blank dictionary is perfectly fine in most cases, setting up your own, tailored dictionaries could somewhat improve compression ratio especially for low sized input datum. - -Please see the [*quick start*](#quick-start-a-simple-example-using-the-api) at the bottom of this page. - -About the algorithms --------------------- - -**Chameleon** ( *DENSITY_ALGORITHM_CHAMELEON* ) - -Chameleon is a dictionary lookup based compression algorithm. It is designed for absolute speed and usually reaches a 60% compression ratio on compressible data. -Decompression is just as fast. This algorithm is a great choice when main concern is speed. - -**Cheetah** ( *DENSITY_ALGORITHM_CHEETAH* ) - -Cheetah was developed with inputs from [Piotr Tarsa](https://github.com/tarsa). -It is derived from chameleon and uses swapped double dictionary lookups and predictions. It can be extremely good with highly compressible data (ratio reaching 10% or less). -On typical compressible data compression ratio is about 50% or less. It is still extremely fast for both compression and decompression and is a great, efficient all-rounder algorithm. - -**Lion** ( *DENSITY_ALGORITHM_LION* ) - -Lion is a multiform compression algorithm derived from cheetah. It goes further in the areas of dynamic adaptation and fine-grained analysis. -It uses multiple swapped dictionary lookups and predictions, and forms rank entropy coding. -Lion provides the best compression ratio of all three algorithms under any circumstance, and is still very fast. - -Quick start (a simple example using the API) --------------------------------------------- -Using DENSITY in your application couldn't be any simpler. - -First you need to include this file in your project : - -* density_api.h - -When this is done you can start using the **DENSITY API** : - -```C - #include - #include "density_api.h" - - char* text = "This is a simple example on how to use the simple Density API. This is a simple example on how to use the simple Density API."; - uint64_t text_length = (uint64_t)strlen(text); - - // Determine safe buffer sizes - uint_fast64_t compress_safe_size = density_compress_safe_size(text_length); - uint_fast64_t decompress_safe_size = density_decompress_safe_size(text_length); - - // Allocate required memory - uint8_t *outCompressed = malloc(compress_safe_size * sizeof(char)); - uint8_t *outDecompressed = malloc(decompress_safe_size * sizeof(char)); - density_processing_result result; - - // Compress - result = density_compress(text, text_length, outCompressed, compress_safe_size, DENSITY_COMPRESSION_MODE_CHAMELEON_ALGORITHM); - if(!result.state) - printf("Compressed %llu bytes to %llu bytes\n", result.bytesRead, result.bytesWritten); - - // Decompress - result = density_decompress(outCompressed, result.bytesWritten, outDecompressed, decompress_safe_size); - if(!result.state) - printf("Decompressed %llu bytes to %llu bytes\n", result.bytesRead, result.bytesWritten); - - // Free memory_allocated - free(outCompressed); - free(outDecompressed); -``` - -And that's it ! We've done a compression/decompression round trip with a few lines ! - -Related projects ----------------- - -* **SHARC** (archiver using density algorithms) [https://github.com/gpnuma/sharc](https://github.com/gpnuma/sharc) -* **fsbench-density** (in-memory transformations benchmark) [https://github.com/gpnuma/fsbench-density](https://github.com/gpnuma/fsbench-density) -* **densityxx** (c++ port of density) [https://github.com/charlesw1234/densityxx](https://github.com/charlesw1234/densityxx) diff --git a/contrib/density/appveyor.yml b/contrib/density/appveyor.yml deleted file mode 100644 index 0aabedebc8b..00000000000 --- a/contrib/density/appveyor.yml +++ /dev/null @@ -1,37 +0,0 @@ -version: 0.14.2.{build} -image: Visual Studio 2015 -init: -- cmd: -environment: - matrix: - - EXTRA_PATH: C:\mingw-w64\i686-5.3.0-posix-dwarf-rt_v4-rev0\mingw32\bin\ - LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar - LAUNCH_PATH: build\ - - EXTRA_PATH: C:\mingw-w64\i686-6.3.0-posix-dwarf-rt_v5-rev1\mingw32\bin\ - LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar - LAUNCH_PATH: build\ - - EXTRA_PATH: C:\mingw-w64\x86_64-6.3.0-posix-seh-rt_v5-rev1\mingw64\bin\ - LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar - LAUNCH_PATH: build\ - - EXTRA_PATH: C:\mingw-w64\x86_64-7.2.0-posix-seh-rt_v5-rev1\mingw64\bin\ - LAUNCH_BUILD: mingw32-make CC=gcc AR=gcc-ar - LAUNCH_PATH: build\ - - EXTRA_PATH: '%programfiles(x86)%\Microsoft Visual Studio\2017\Community\MSBuild\15.0\Bin\amd64\' - LAUNCH_BUILD: cd msvc && msbuild.exe Density.sln - EXTRA_GIT: git submodule update --init --recursive - LAUNCH_PATH: bin\Release\ -install: -- cmd: -build_script: -- cmd: >- - %EXTRA_GIT% - - set PATH=%EXTRA_PATH%;%PATH% - - %LAUNCH_BUILD% - - file %LAUNCH_PATH:\=/%* - - %LAUNCH_PATH%benchmark.exe -h -f -test_script: -- cmd: diff --git a/contrib/density/benchmark/src/benchmark.c b/contrib/density/benchmark/src/benchmark.c deleted file mode 100644 index 9781ef361c6..00000000000 --- a/contrib/density/benchmark/src/benchmark.c +++ /dev/null @@ -1,345 +0,0 @@ -/* - * Density benchmark - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program. If not, see . - * - * 5/04/15 19:30 - */ - -#include "benchmark.h" - -void density_benchmark_version() { - printf("\nSingle threaded "); - DENSITY_BENCHMARK_BOLD(printf("in-memory benchmark")); - printf(" powered by "); - DENSITY_BENCHMARK_BOLD(printf("Density %i.%i.%i\n", density_version_major(), density_version_minor(), density_version_revision())); - printf("Copyright (C) 2015 Guillaume Voirin\n"); - printf("Built for %s (%s endian system, %u bits) using " DENSITY_BENCHMARK_COMPILER ", %s %s\n", DENSITY_BENCHMARK_PLATFORM_STRING, DENSITY_BENCHMARK_ENDIAN_STRING, (unsigned int) (8 * sizeof(void *)), DENSITY_BENCHMARK_COMPILER_VERSION, __DATE__, __TIME__); -} - -void density_benchmark_client_usage() { - printf("\n"); - DENSITY_BENCHMARK_BOLD(printf("Usage :\n")); - printf(" benchmark [OPTIONS ?]... [FILE ?]\n\n"); - DENSITY_BENCHMARK_BOLD(printf("Available options :\n")); - printf(" -[LEVEL] Test file using only the specified compression LEVEL\n"); - printf(" If unspecified, all algorithms are tested (default).\n"); - printf(" LEVEL can have the following values (as values become higher,\n"); - printf(" compression ratio increases and speed diminishes) :\n"); - printf(" 0 = Copy (no compression)\n"); - printf(" 1 = Chameleon algorithm\n"); - printf(" 2 = Cheetah algorithm\n"); - printf(" 3 = Lion algorithm\n"); - printf(" -c Compress only\n"); - printf(" -f Activate fuzzer mode (pseudorandom generated data)\n"); - printf(" -h Print data hashing informations\n\n"); - exit(EXIT_SUCCESS); -} - -void density_benchmark_format_decimal(uint64_t number) { - if (number < 1000) { - printf("%"PRIu64, number); - return; - } - density_benchmark_format_decimal(number / 1000); - printf(",%03"PRIu64, number % 1000); -} - -const char *density_benchmark_convert_state_to_text(DENSITY_STATE state) { - switch (state) { - case DENSITY_STATE_ERROR_DURING_PROCESSING: - return "Error during processing"; - case DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL: - return "Input buffer is too small"; - case DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL: - return "Output buffer is too small"; - case DENSITY_STATE_ERROR_INVALID_CONTEXT: - return "Invalid context"; - case DENSITY_STATE_ERROR_INVALID_ALGORITHM: - return "Invalid algorithm"; - default: - return "Unknown error"; - } -} - -int main(int argc, char *argv[]) { - density_benchmark_version(); - DENSITY_ALGORITHM start_mode = DENSITY_ALGORITHM_CHAMELEON; - DENSITY_ALGORITHM end_mode = DENSITY_ALGORITHM_LION; - bool compression_only = false; - bool fuzzer = false; - bool hash_info = false; - char *file_path = NULL; - - if (argc <= 1) - density_benchmark_client_usage(); - for (int count = 1; count < argc; count++) { - if (argv[count][0] == '-') { - switch (argv[count][1]) { - case '1': - start_mode = DENSITY_ALGORITHM_CHAMELEON; - end_mode = DENSITY_ALGORITHM_CHAMELEON; - break; - case '2': - start_mode = DENSITY_ALGORITHM_CHEETAH; - end_mode = DENSITY_ALGORITHM_CHEETAH; - break; - case '3': - start_mode = DENSITY_ALGORITHM_LION; - end_mode = DENSITY_ALGORITHM_LION; - break; - case 'c': - compression_only = true; - break; - case 'f': - fuzzer = true; - break; - case 'h': - hash_info = true; - break; - default: - density_benchmark_client_usage(); - } - } else - file_path = argv[argc - 1]; - } - - uint8_t *in; - uint8_t *out; - uint_fast64_t uncompressed_size; - uint_fast64_t memory_allocated; - if (fuzzer) { - srand((unsigned int) (time(NULL) * 14521937821257379531llu)); - uncompressed_size = (uint_fast64_t) (((uint64_t) (rand() * 100000000llu)) / RAND_MAX); - memory_allocated = density_compress_safe_size(uncompressed_size); - in = malloc(memory_allocated * sizeof(uint8_t)); - uint8_t value = (uint8_t) rand(); - for (unsigned int count = 0; count < uncompressed_size; count++) { - if (!(rand() & 0xf)) - value += (uint8_t)rand(); - in[count] = value; - } - out = malloc(memory_allocated * sizeof(uint8_t)); - } else { - // Open file and get infos - FILE *file = fopen(file_path, "rb"); - if (file == NULL) { - DENSITY_BENCHMARK_ERROR(printf("Error opening file %s.", file_path), false); - } - struct stat file_attributes; - stat(file_path, &file_attributes); - - // Allocate memory and copy file to memory - uncompressed_size = (uint_fast64_t) file_attributes.st_size; - memory_allocated = density_compress_safe_size(uncompressed_size); - in = malloc(memory_allocated * sizeof(uint8_t)); - size_t read = fread(in, sizeof(uint8_t), uncompressed_size, file); - if(uncompressed_size != read) { - DENSITY_BENCHMARK_ERROR(printf("Error reading file %s.", file_path), false); - } - fclose(file); - out = malloc(memory_allocated * sizeof(uint8_t)); - } - - printf("Allocated "); - density_benchmark_format_decimal(2 * memory_allocated); - printf(" bytes of in-memory work space\n"); - - uint64_t original_hash_1 = DENSITY_BENCHMARK_HASH_SEED_1; - uint64_t original_hash_2 = DENSITY_BENCHMARK_HASH_SEED_2; - spookyhash_128(in, uncompressed_size, &original_hash_1, &original_hash_2); - - printf("\n"); - for (DENSITY_ALGORITHM compression_mode = start_mode; compression_mode <= end_mode; compression_mode++) { - // Print algorithm info - switch (compression_mode) { - case DENSITY_ALGORITHM_CHAMELEON: - DENSITY_BENCHMARK_BLUE(DENSITY_BENCHMARK_BOLD(printf("Chameleon algorithm"))); - DENSITY_BENCHMARK_UNDERLINE(19); - break; - case DENSITY_ALGORITHM_CHEETAH: - DENSITY_BENCHMARK_BLUE(DENSITY_BENCHMARK_BOLD(printf("Cheetah algorithm"))); - DENSITY_BENCHMARK_UNDERLINE(17); - break; - case DENSITY_ALGORITHM_LION: - DENSITY_BENCHMARK_BLUE(DENSITY_BENCHMARK_BOLD(printf("Lion algorithm"))); - DENSITY_BENCHMARK_UNDERLINE(14); - break; - } - fflush(stdout); - - // Pre-heat - printf("\nUsing "); - if (fuzzer) { - DENSITY_BENCHMARK_BOLD(printf("generated data")); - } else { - printf("file "); - DENSITY_BENCHMARK_BOLD(printf("%s", file_path)); - } - printf(" copied in memory\n"); - if(hash_info) { - printf("Uncompressed data hash is "); - DENSITY_BENCHMARK_BOLD(printf("0x%" PRIx64 "%" PRIx64, original_hash_1, original_hash_2)); - printf("\n"); - } - - density_processing_result result = density_compress(in, uncompressed_size, out, memory_allocated, compression_mode); - if (result.state) { - DENSITY_BENCHMARK_ERROR(printf("During compress API returned error %i (%s).", result.state, density_benchmark_convert_state_to_text(result.state)), true); - } - const uint_fast64_t compressed_size = result.bytesWritten; - - uint64_t hash_1 = DENSITY_BENCHMARK_HASH_SEED_1; - uint64_t hash_2 = DENSITY_BENCHMARK_HASH_SEED_2; - if(hash_info) { - spookyhash_128(out, compressed_size, &hash_1, &hash_2); - printf("Compressed data hash is "); - DENSITY_BENCHMARK_BOLD(printf("0x%" PRIx64 "%" PRIx64, hash_1, hash_2)); - printf("\n"); - } - - if (!compression_only) { - memset(in, 0, memory_allocated); - result = density_decompress(out, compressed_size, in, memory_allocated); - if (result.state) { - DENSITY_BENCHMARK_ERROR(printf("During decompress API returned error %i (%s).", result.state, density_benchmark_convert_state_to_text(result.state)), true); - } - if (result.bytesWritten != uncompressed_size) { - DENSITY_BENCHMARK_ERROR(printf("Round-trip size differs from original size ("); - density_benchmark_format_decimal(result.bytesWritten); - printf(" bytes against "); - density_benchmark_format_decimal(uncompressed_size); - printf(" bytes).");, true); - } - - hash_1 = DENSITY_BENCHMARK_HASH_SEED_1; - hash_2 = DENSITY_BENCHMARK_HASH_SEED_2; - spookyhash_128(in, uncompressed_size, &hash_1, &hash_2); - - if(hash_info) { - printf("Round-trip data hash is "); - DENSITY_BENCHMARK_BOLD(printf("0x%" PRIx64 "%" PRIx64, hash_1, hash_2)); - printf("\n"); - } - - if(hash_1 != original_hash_1 || hash_2 != original_hash_2) { - DENSITY_BENCHMARK_ERROR(printf("Uncompressed and round-trip data hashes do not match ("); - printf("0x%" PRIx64 "%" PRIx64, hash_1, hash_2); - printf(" vs. "); - printf("0x%" PRIx64 "%" PRIx64, original_hash_1, original_hash_2); - printf(").");, true); - } - - printf("Uncompressed and round-trip data hashes match. "); - } - printf("Starting main bench.\n"); - if (compression_only) - printf("Compressing "); - else - printf("Round-tripping "); - density_benchmark_format_decimal(uncompressed_size); - printf(" bytes to "); - density_benchmark_format_decimal(compressed_size); - printf(" bytes (compression ratio "); - DENSITY_BENCHMARK_BOLD(printf("%.2lf%%", (100.0 * compressed_size) / uncompressed_size)); - printf(" or "); - DENSITY_BENCHMARK_BOLD(printf("%.3fx", (1.0 * uncompressed_size) / compressed_size)); - if (compression_only) - printf(")\n"); - else - printf(") and back\n"); - fflush(stdout); - - // Main benchmark - unsigned int iterations = 0; - double compress_time_high = 0.0; - double compress_time_low = 60.0; - double decompress_time_high = 0.0; - double decompress_time_low = 60.0; - double total_compress_time = 0.0; - double total_decompress_time = 0.0; - double total_time = 0.0; - double decompress_speed = 0.0; - double decompress_speed_low = 0.0; - double decompress_speed_high = 0.0; - double compress_time_elapsed = 0.0; - double decompress_time_elapsed = 0.0; - cputime_chronometer chrono; - - while (total_time <= 10.0) { - ++iterations; - - cputime_chronometer_start(&chrono); - density_compress(in, uncompressed_size, out, memory_allocated, compression_mode); - compress_time_elapsed = cputime_chronometer_stop(&chrono); - - if (!compression_only) { - cputime_chronometer_start(&chrono); - density_decompress(out, compressed_size, in, memory_allocated); - decompress_time_elapsed = cputime_chronometer_stop(&chrono); - } - - total_compress_time += compress_time_elapsed; - - if (compress_time_elapsed < compress_time_low) - compress_time_low = compress_time_elapsed; - if (compress_time_elapsed > compress_time_high) - compress_time_high = compress_time_elapsed; - - double compress_speed = ((1.0 * uncompressed_size * iterations) / (total_compress_time * 1000.0 * 1000.0)); - double compress_speed_low = ((1.0 * uncompressed_size) / (compress_time_high * 1000.0 * 1000.0)); - double compress_speed_high = ((1.0 * uncompressed_size) / (compress_time_low * 1000.0 * 1000.0)); - - total_time += compress_time_elapsed; - - if (!compression_only) { - total_decompress_time += decompress_time_elapsed; - - if (decompress_time_elapsed < decompress_time_low) - decompress_time_low = decompress_time_elapsed; - if (decompress_time_elapsed > decompress_time_high) - decompress_time_high = decompress_time_elapsed; - - decompress_speed = ((1.0 * uncompressed_size * iterations) / (total_decompress_time * 1000.0 * 1000.0)); - decompress_speed_low = ((1.0 * uncompressed_size) / (decompress_time_high * 1000.0 * 1000.0)); - decompress_speed_high = ((1.0 * uncompressed_size) / (decompress_time_low * 1000.0 * 1000.0)); - - total_time += decompress_time_elapsed; - } - - DENSITY_BENCHMARK_BLUE(printf("\rCompress speed "); - DENSITY_BENCHMARK_BOLD(printf("%.0lf MB/s", compress_speed))); - printf(" (min %.0lf MB/s, max %.0lf MB/s, best %.4lfs) ", compress_speed_low, compress_speed_high, compress_time_low); - - if (!compression_only) { - printf("<=> "); - DENSITY_BENCHMARK_BLUE(printf("Decompress speed "); - DENSITY_BENCHMARK_BOLD(printf("%.0lf MB/s", decompress_speed))); - printf(" (min %.0lf MB/s, max %.0lf MB/s, best %.4lfs) ", decompress_speed_low, decompress_speed_high, decompress_time_low); - } - fflush(stdout); - } - printf("\nRun time %.3lfs (%i iterations)\n\n", total_time, iterations); - } - - free(in); - free(out); - - printf("Allocated memory released.\n\n"); - - return EXIT_SUCCESS; -} diff --git a/contrib/density/benchmark/src/benchmark.h b/contrib/density/benchmark/src/benchmark.h deleted file mode 100644 index 749e1ccb1d1..00000000000 --- a/contrib/density/benchmark/src/benchmark.h +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Density benchmark - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program. If not, see . - * - * 6/04/15 0:11 - */ - -#ifndef DENSITY_BENCHMARK_H -#define DENSITY_BENCHMARK_H - -#define _CRT_SECURE_NO_DEPRECATE - -#include -#include -#include -#include -#include -#include -#include -#include -#include "../../src/density_api.h" -#include "../libs/cputime/src/cputime_api.h" -#include "../libs/spookyhash/src/spookyhash_api.h" - -#if defined(_WIN64) || defined(_WIN32) -#else -#define DENSITY_BENCHMARK_ALLOW_ANSI_ESCAPE_SEQUENCES -#endif - -#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ -#define DENSITY_BENCHMARK_ENDIAN_STRING "Little" -#elif __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ -#define DENSITY_BENCHMARK_ENDIAN_STRING "Big" -#endif - -#if defined(__clang__) -#define DENSITY_BENCHMARK_COMPILER "Clang %d.%d.%d" -#define DENSITY_BENCHMARK_COMPILER_VERSION __clang_major__, __clang_minor__, __clang_patchlevel__ -#elif defined(__GNUC__) -#define DENSITY_BENCHMARK_COMPILER "GCC %d.%d.%d" -#define DENSITY_BENCHMARK_COMPILER_VERSION __GNUC__, __GNUC_MINOR__, __GNUC_PATCHLEVEL__ -#elif defined(_MSC_VER) -#define DENSITY_BENCHMARK_COMPILER "MSVC" -#define DENSITY_BENCHMARK_COMPILER_VERSION "" -#elif defined(__INTEL_COMPILER) -#define DENSITY_BENCHMARK_COMPILER "ICC" -#define DENSITY_BENCHMARK_COMPILER_VERSION "" -#else -#define DENSITY_BENCHMARK_COMPILER "an unknown compiler" -#define DENSITY_BENCHMARK_COMPILER_VERSION "" -#endif - -#if defined(_WIN64) || defined(_WIN32) -#define DENSITY_BENCHMARK_PLATFORM_STRING "Microsoft Windows" -#elif defined(__APPLE__) -#include "TargetConditionals.h" -#if TARGET_IPHONE_SIMULATOR -#define DENSITY_BENCHMARK_PLATFORM_STRING "iOS Simulator" -#elif TARGET_OS_IPHONE -#define DENSITY_BENCHMARK_PLATFORM_STRING "iOS" -#elif TARGET_OS_MAC -#define DENSITY_BENCHMARK_PLATFORM_STRING "MacOS" -#else -#define DENSITY_BENCHMARK_PLATFORM_STRING "an unknown Apple platform" -#endif -#elif defined(__FreeBSD__) -#define DENSITY_BENCHMARK_PLATFORM_STRING "FreeBSD" -#elif defined(__linux__) -#define DENSITY_BENCHMARK_PLATFORM_STRING "GNU/Linux" -#elif defined(__unix__) -#define DENSITY_BENCHMARK_PLATFORM_STRING "Unix" -#elif defined(__posix__) -#define DENSITY_BENCHMARK_PLATFORM_STRING "Posix" -#else -#define DENSITY_BENCHMARK_PLATFORM_STRING "an unknown platform" -#endif - -#define DENSITY_ESCAPE_CHARACTER ((char)27) - -#ifdef DENSITY_BENCHMARK_ALLOW_ANSI_ESCAPE_SEQUENCES -#define DENSITY_BENCHMARK_BOLD(op) printf("%c[1m", DENSITY_ESCAPE_CHARACTER);\ - op;\ - printf("%c[0m", DENSITY_ESCAPE_CHARACTER); - -#define DENSITY_BENCHMARK_BLUE(op) printf("%c[0;34m", DENSITY_ESCAPE_CHARACTER);\ - op;\ - printf("%c[0m", DENSITY_ESCAPE_CHARACTER); - -#define DENSITY_BENCHMARK_RED(op) printf("%c[0;31m", DENSITY_ESCAPE_CHARACTER);\ - op;\ - printf("%c[0m", DENSITY_ESCAPE_CHARACTER); -#else -#define DENSITY_BENCHMARK_BOLD(op) op; -#define DENSITY_BENCHMARK_BLUE(op) op; -#define DENSITY_BENCHMARK_RED(op) op; -#endif - -#define DENSITY_BENCHMARK_UNDERLINE(n) printf("\n");\ - for(int i = 0; i < n; i++) printf("="); - -#define DENSITY_BENCHMARK_ERROR(op, issue) DENSITY_BENCHMARK_RED(DENSITY_BENCHMARK_BOLD(printf("\nAn error has occured !\n")));\ - op;\ - printf("\n");\ - if(issue) {\ - printf("Please open an issue at , with your platform information and any relevant file.\n");\ - DENSITY_BENCHMARK_BOLD(printf("Thank you !\n"));\ - }\ - fflush(stdout);\ - exit(EXIT_FAILURE); - -#endif - -#define DENSITY_BENCHMARK_HASH_SEED_1 0x0123456789abcdefllu -#define DENSITY_BENCHMARK_HASH_SEED_2 0xfedcba9876543210llu diff --git a/contrib/density/msvc/Density.sln b/contrib/density/msvc/Density.sln deleted file mode 100644 index 70c25ab3413..00000000000 --- a/contrib/density/msvc/Density.sln +++ /dev/null @@ -1,21 +0,0 @@ - -Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio 14 -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "density", "density.vcxproj", "{65C51F09-D1A4-9EA4-DABC-297B461B0506}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "benchmark", "benchmark.vcxproj", "{7000C5C1-DC6A-7938-25A9-2ADE9152578D}" -EndProject -Global - GlobalSection(SolutionConfigurationPlatforms) = preSolution - Release|x64 = Release|x64 - EndGlobalSection - GlobalSection(ProjectConfigurationPlatforms) = postSolution - {65C51F09-D1A4-9EA4-DABC-297B461B0506}.Release|x64.ActiveCfg = Release|x64 - {65C51F09-D1A4-9EA4-DABC-297B461B0506}.Release|x64.Build.0 = Release|x64 - {7000C5C1-DC6A-7938-25A9-2ADE9152578D}.Release|x64.ActiveCfg = Release|x64 - {7000C5C1-DC6A-7938-25A9-2ADE9152578D}.Release|x64.Build.0 = Release|x64 - EndGlobalSection - GlobalSection(SolutionProperties) = preSolution - HideSolutionNode = FALSE - EndGlobalSection -EndGlobal diff --git a/contrib/density/msvc/benchmark.vcxproj b/contrib/density/msvc/benchmark.vcxproj deleted file mode 100644 index ef32bbb5f3c..00000000000 --- a/contrib/density/msvc/benchmark.vcxproj +++ /dev/null @@ -1,78 +0,0 @@ - - - - - Release - x64 - - - - {7000C5C1-DC6A-7938-25A9-2ADE9152578D} - true - Win32Proj - benchmark - - - - Application - false - Unicode - v140 - true - - - - - - - - - - false - bin\Release\ - obj\Release\benchmark\ - benchmark - .exe - - - - NotUsing - Level4 - MaxSpeed - true - true - false - true - true - - - Console - true - true - - - - - - - - - - - - - - - - - - - - - {65C51F09-D1A4-9EA4-DABC-297B461B0506} - - - - - - \ No newline at end of file diff --git a/contrib/density/msvc/benchmark.vcxproj.filters b/contrib/density/msvc/benchmark.vcxproj.filters deleted file mode 100644 index 26379e28724..00000000000 --- a/contrib/density/msvc/benchmark.vcxproj.filters +++ /dev/null @@ -1,63 +0,0 @@ - - - - - {2F149A7C-1B4B-9B0D-C437-8110B04D170F} - - - {95C9EAC4-812C-7A69-2AB2-B21F16F445EC} - - - {2C4ACB69-1843-EABB-4175-CF402DCDC9C7} - - - {C78AF53E-3316-6303-3C27-E6F7A831BF03} - - - {5E54D780-CAB5-B48E-5323-FB40BF83EB4D} - - - {2DAB880B-99B4-887C-2230-9F7C8E38947C} - - - - - libs\cputime\src - - - libs\cputime\src - - - libs\spookyhash\src - - - libs\spookyhash\src - - - libs\spookyhash\src - - - libs\spookyhash\src - - - src - - - - - libs\cputime\src - - - libs\spookyhash\src - - - libs\spookyhash\src - - - libs\spookyhash\src - - - src - - - \ No newline at end of file diff --git a/contrib/density/msvc/density.vcxproj b/contrib/density/msvc/density.vcxproj deleted file mode 100644 index f1ad77fbc1b..00000000000 --- a/contrib/density/msvc/density.vcxproj +++ /dev/null @@ -1,93 +0,0 @@ - - - - - Release - x64 - - - - {65C51F09-D1A4-9EA4-DABC-297B461B0506} - true - Win32Proj - density - - - - DynamicLibrary - false - Unicode - v140 - true - - - - - - - - - - false - bin\Release\ - obj\Release\density\ - density - .dll - - - - NotUsing - Level4 - MaxSpeed - true - true - false - true - true - - - Windows - true - true - bin\Release\density.lib - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/contrib/density/msvc/density.vcxproj.filters b/contrib/density/msvc/density.vcxproj.filters deleted file mode 100644 index 202f2522bbe..00000000000 --- a/contrib/density/msvc/density.vcxproj.filters +++ /dev/null @@ -1,135 +0,0 @@ - - - - - {DF10C562-CBC8-06B0-34D0-DF0B20A7A1A0} - - - {5AB95D49-4648-E712-EF66-FB0DDBD4F7B8} - - - {1252540B-7E14-8895-C7C4-162233C66073} - - - {3F6054BC-AB43-63FC-B446-913820A9294D} - - - {C09C0CB6-AC80-CD0B-15E1-C75E01E4B78D} - - - {3845B705-A47C-FF9C-AD86-63C0193D2696} - - - {E597066D-51B0-ED96-1A5D-7D3086348230} - - - {40FA2C44-AC85-9A08-B596-1DFD21A1F608} - - - {B8122E82-A4A1-B74B-4DC0-CB46392EC8F1} - - - {6585C1C9-5195-6D9A-BA4E-D2B8A67D5C33} - - - {B6B426C8-2221-E2C2-EB14-7A205740042B} - - - {B2DFE593-1EBF-642F-27D7-EF059335CB90} - - - {367A73B3-A2E4-272A-EB22-D9CF57CC057F} - - - - - algorithms - - - algorithms\chameleon - - - algorithms\chameleon\core - - - algorithms\chameleon\core - - - algorithms\chameleon\dictionary - - - algorithms\cheetah - - - algorithms\cheetah\core - - - algorithms\cheetah\core - - - algorithms\cheetah\dictionary - - - algorithms - - - algorithms\lion\core - - - algorithms\lion\core - - - algorithms\lion\dictionary - - - algorithms\lion\forms - - - algorithms\lion - - - buffers - - - - - structure - - - - - algorithms - - - algorithms\chameleon\core - - - algorithms\chameleon\core - - - algorithms\cheetah\core - - - algorithms\cheetah\core - - - algorithms - - - algorithms\lion\core - - - algorithms\lion\core - - - algorithms\lion\forms - - - buffers - - - - structure - - - \ No newline at end of file diff --git a/contrib/density/src/algorithms/algorithms.c b/contrib/density/src/algorithms/algorithms.c deleted file mode 100644 index f36e94cd08b..00000000000 --- a/contrib/density/src/algorithms/algorithms.c +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 14/10/15 02:06 - */ - -#include "algorithms.h" - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_algorithms_prepare_state(density_algorithm_state *const DENSITY_RESTRICT state, void *const DENSITY_RESTRICT dictionary) { - state->dictionary = dictionary; - state->copy_penalty = 0; - state->copy_penalty_start = 1; - state->previous_incompressible = false; - state->counter = 0; -} diff --git a/contrib/density/src/algorithms/algorithms.h b/contrib/density/src/algorithms/algorithms.h deleted file mode 100644 index b5e31377004..00000000000 --- a/contrib/density/src/algorithms/algorithms.h +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 3/02/15 19:51 - */ - -#ifndef DENSITY_ALGORITHMS_H -#define DENSITY_ALGORITHMS_H - -#include "../globals.h" - -typedef enum { - DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED = 0, - DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING, - DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL, - DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL -} density_algorithm_exit_status; - -typedef struct { - void *dictionary; - uint_fast8_t copy_penalty; - uint_fast8_t copy_penalty_start; - bool previous_incompressible; - uint_fast64_t counter; -} density_algorithm_state; - -#define DENSITY_ALGORITHM_COPY(work_block_size)\ - DENSITY_MEMCPY(*out, *in, work_block_size);\ - *in += work_block_size;\ - *out += work_block_size; - -#define DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START\ - if(!(--state->copy_penalty))\ - state->copy_penalty_start++; - -#define DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START\ - if (state->copy_penalty_start & ~0x1)\ - state->copy_penalty_start >>= 1; - -#define DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY(span, work_block_size)\ - if (DENSITY_UNLIKELY(span & ~(work_block_size - 1))) {\ - if (state->previous_incompressible)\ - state->copy_penalty = state->copy_penalty_start;\ - state->previous_incompressible = true;\ - } else\ - state->previous_incompressible = false; - -DENSITY_WINDOWS_EXPORT void density_algorithms_prepare_state(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, void *const DENSITY_RESTRICT_DECLARE); - -#endif diff --git a/contrib/density/src/algorithms/chameleon/chameleon.h b/contrib/density/src/algorithms/chameleon/chameleon.h deleted file mode 100644 index 2054f34cfd8..00000000000 --- a/contrib/density/src/algorithms/chameleon/chameleon.h +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/10/13 11:57 - * - * ------------------- - * Chameleon algorithm - * ------------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Hash based superfast kernel - */ - -#ifndef DENSITY_CHAMELEON_H -#define DENSITY_CHAMELEON_H - -#include "../../globals.h" - -#define DENSITY_CHAMELEON_HASH_BITS 16 -#define DENSITY_CHAMELEON_HASH_MULTIPLIER (uint32_t)0x9D6EF916lu - -#define DENSITY_CHAMELEON_HASH_ALGORITHM(value32) (uint16_t)((value32 * DENSITY_CHAMELEON_HASH_MULTIPLIER) >> (32 - DENSITY_CHAMELEON_HASH_BITS)) - -typedef enum { - DENSITY_CHAMELEON_SIGNATURE_FLAG_CHUNK = 0x0, - DENSITY_CHAMELEON_SIGNATURE_FLAG_MAP = 0x1, -} DENSITY_CHAMELEON_SIGNATURE_FLAG; - -typedef uint64_t density_chameleon_signature; - -#define DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE (density_bitsizeof(density_chameleon_signature) * sizeof(uint32_t)) // Uncompressed chunks -#define DENSITY_CHAMELEON_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE (density_bitsizeof(density_chameleon_signature) * sizeof(uint32_t)) - -#define DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE (sizeof(density_chameleon_signature) + DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE) -#define DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE (DENSITY_CHAMELEON_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE) - -#define DENSITY_CHAMELEON_WORK_BLOCK_SIZE 256 - -#endif diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_decode.c b/contrib/density/src/algorithms/chameleon/core/chameleon_decode.c deleted file mode 100644 index 240aaba8598..00000000000 --- a/contrib/density/src/algorithms/chameleon/core/chameleon_decode.c +++ /dev/null @@ -1,254 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 23/06/15 22:11 - * - * ------------------- - * Chameleon algorithm - * ------------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Hash based superfast kernel - */ - -#include "chameleon_decode.h" - -DENSITY_FORCE_INLINE void density_chameleon_decode_process_compressed(const uint16_t hash, uint8_t **DENSITY_RESTRICT out, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { - DENSITY_MEMCPY(*out, &dictionary->entries[hash].as_uint32_t, sizeof(uint32_t)); -} - -DENSITY_FORCE_INLINE void density_chameleon_decode_process_uncompressed(const uint32_t chunk, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { - const uint16_t hash = DENSITY_CHAMELEON_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(chunk)); - (&dictionary->entries[hash])->as_uint32_t = chunk; // Does not ensure dictionary content consistency between endiannesses -} - -DENSITY_FORCE_INLINE void density_chameleon_decode_kernel(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_bool compressed, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { - if (compressed) { - uint16_t hash; - DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); - density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16(hash), out, dictionary); - *in += sizeof(uint16_t); - } else { - uint32_t unit; - DENSITY_MEMCPY(&unit, *in, sizeof(uint32_t)); - density_chameleon_decode_process_uncompressed(unit, dictionary); - DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); - *in += sizeof(uint32_t); - } - *out += sizeof(uint32_t); -} - -DENSITY_FORCE_INLINE void density_chameleon_decode_kernel_dual(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_chameleon_signature signature, const uint_fast8_t shift, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { - uint32_t var_32; - uint64_t var_64; - - switch((signature >> shift) & 0x3) { - case 0x0: - DENSITY_MEMCPY(&var_64, *in, sizeof(uint32_t) + sizeof(uint32_t)); -#ifdef DENSITY_LITTLE_ENDIAN - density_chameleon_decode_process_uncompressed((uint32_t)(var_64 & 0xffffffff), dictionary); -#endif - density_chameleon_decode_process_uncompressed((uint32_t)(var_64 >> density_bitsizeof(uint32_t)), dictionary); -#ifdef DENSITY_BIG_ENDIAN - density_chameleon_decode_process_uncompressed((uint32_t)(var_64 & 0xffffffff), dictionary); -#endif - DENSITY_MEMCPY(*out, &var_64, sizeof(uint32_t) + sizeof(uint32_t)); - *in += (sizeof(uint32_t) + sizeof(uint32_t)); - *out += sizeof(uint64_t); - break; - case 0x1: - DENSITY_MEMCPY(&var_64, *in, sizeof(uint16_t) + sizeof(uint32_t)); -#ifdef DENSITY_LITTLE_ENDIAN - density_chameleon_decode_process_compressed((uint16_t)(var_64 & 0xffff), out, dictionary); - var_32 = (uint32_t)((var_64 >> density_bitsizeof(uint16_t)) & 0xffffffff); - density_chameleon_decode_process_uncompressed(var_32, dictionary); - DENSITY_MEMCPY(*out + sizeof(uint32_t), &var_32, sizeof(uint32_t)); - *out += sizeof(uint64_t); -#elif defined(DENSITY_BIG_ENDIAN) - density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)((var_64 >> (density_bitsizeof(uint16_t) + density_bitsizeof(uint32_t))) & 0xffff)), out, dictionary); - var_32 = (uint32_t)((var_64 >> density_bitsizeof(uint16_t)) & 0xffffffff); - density_chameleon_decode_process_uncompressed(var_32, dictionary); - DENSITY_MEMCPY(*out + sizeof(uint32_t), &var_32, sizeof(uint32_t)); - *out += sizeof(uint64_t); -#else -#error -#endif - *in += (sizeof(uint16_t) + sizeof(uint32_t)); - break; - case 0x2: - DENSITY_MEMCPY(&var_64, *in, sizeof(uint32_t) + sizeof(uint16_t)); -#ifdef DENSITY_LITTLE_ENDIAN - var_32 = (uint32_t)(var_64 & 0xffffffff); - density_chameleon_decode_process_uncompressed(var_32, dictionary); - DENSITY_MEMCPY(*out, &var_32, sizeof(uint32_t)); - *out += sizeof(uint32_t); - density_chameleon_decode_process_compressed((uint16_t)((var_64 >> density_bitsizeof(uint32_t)) & 0xffff), out, dictionary); - *out += sizeof(uint32_t); -#elif defined(DENSITY_BIG_ENDIAN) - var_32 = (uint32_t)((var_64 >> density_bitsizeof(uint32_t)) & 0xffffffff); - density_chameleon_decode_process_uncompressed(var_32, dictionary); - DENSITY_MEMCPY(*out, &var_32, sizeof(uint32_t)); - *out += sizeof(uint32_t); - density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)((var_64 >> density_bitsizeof(uint16_t)) & 0xffff)), out, dictionary); - *out += sizeof(uint32_t); -#else -#error -#endif - *in += (sizeof(uint32_t) + sizeof(uint16_t)); - break; - case 0x3: - DENSITY_MEMCPY(&var_32, *in, sizeof(uint16_t) + sizeof(uint16_t)); -#ifdef DENSITY_LITTLE_ENDIAN - density_chameleon_decode_process_compressed((uint16_t)(var_32 & 0xffff), out, dictionary); - *out += sizeof(uint32_t); -#endif - density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)(var_32 >> density_bitsizeof(uint16_t))), out, dictionary); - *out += sizeof(uint32_t); -#ifdef DENSITY_BIG_ENDIAN - density_chameleon_decode_process_compressed(DENSITY_LITTLE_ENDIAN_16((uint16_t)(var_32 & 0xffff)), out, dictionary); - *out += sizeof(uint32_t); -#endif - *in += (sizeof(uint16_t) + sizeof(uint16_t)); - break; - } -} - -DENSITY_FORCE_INLINE bool density_chameleon_decode_test_compressed(const density_chameleon_signature signature, const uint_fast8_t shift) { - return (density_bool const) ((signature >> shift) & DENSITY_CHAMELEON_SIGNATURE_FLAG_MAP); -} - -DENSITY_FORCE_INLINE void density_chameleon_decode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_chameleon_signature signature, const uint_fast8_t shift, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { - density_chameleon_decode_kernel(in, out, density_chameleon_decode_test_compressed(signature, shift), dictionary); -} - -DENSITY_FORCE_INLINE void density_chameleon_decode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const density_chameleon_signature signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary) { - uint_fast8_t count_a = 0; - uint_fast8_t count_b = 0; - -#if defined(__clang__) || defined(_MSC_VER) - do { - DENSITY_UNROLL_2(density_chameleon_decode_kernel_dual(in, out, signature, count_a, dictionary); count_a+= 2); - } while (++count_b & 0xf); -#else - do { - DENSITY_UNROLL_2(density_chameleon_decode_4(in, out, signature, count_a ++, dictionary)); - } while (++count_b & 0x1f); -#endif -} - -DENSITY_FORCE_INLINE void density_chameleon_decode_read_signature(const uint8_t **DENSITY_RESTRICT in, density_chameleon_signature *DENSITY_RESTRICT signature) { -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature, *in, sizeof(density_chameleon_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - density_chameleon_signature endian_signature; - DENSITY_MEMCPY(&endian_signature, *in, sizeof(density_chameleon_signature)); - *signature = DENSITY_LITTLE_ENDIAN_64(endian_signature); -#else -#error -#endif - *in += sizeof(density_chameleon_signature); -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_chameleon_decode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { - if (out_size < DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - density_chameleon_signature signature; - uint_fast8_t shift; - uint_fast64_t remaining; - - const uint8_t *start = *in; - - if (in_size < DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE) { - goto read_signature; - } - - const uint8_t *in_limit = *in + in_size - DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE; - uint8_t *out_limit = *out + out_size - DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE; - - while (DENSITY_LIKELY(*in <= in_limit && *out <= out_limit)) { - if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { - DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; - } - state->counter++; - if (DENSITY_UNLIKELY(state->copy_penalty)) { - DENSITY_ALGORITHM_COPY(DENSITY_CHAMELEON_WORK_BLOCK_SIZE); - DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; - } else { - const uint8_t *in_start = *in; - density_chameleon_decode_read_signature(in, &signature); - density_chameleon_decode_256(in, out, signature, (density_chameleon_dictionary *const) state->dictionary); - DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*in - in_start), DENSITY_CHAMELEON_WORK_BLOCK_SIZE); - } - } - - if (*out > out_limit) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - read_signature: - if (in_size - (*in - start) < sizeof(density_chameleon_signature)) - return DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL; - shift = 0; - density_chameleon_decode_read_signature(in, &signature); - read_and_decode_4: - switch (in_size - (*in - start)) { - case 0: - case 1: - if (density_chameleon_decode_test_compressed(signature, shift)) - return DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; - else // End marker - goto process_remaining_bytes; - case 2: - case 3: - if (density_chameleon_decode_test_compressed(signature, shift++)) - density_chameleon_decode_kernel(in, out, true, (density_chameleon_dictionary *const) state->dictionary); - else // End marker - goto process_remaining_bytes; - break; - default: - density_chameleon_decode_4(in, out, signature, shift++, (density_chameleon_dictionary *const) state->dictionary); - break; - } - - if (DENSITY_UNLIKELY(shift == density_bitsizeof(density_chameleon_signature))) - goto read_signature; - else - goto read_and_decode_4; - - process_remaining_bytes: - remaining = in_size - (*in - start); - DENSITY_ALGORITHM_COPY(remaining); - - return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; -} diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_decode.h b/contrib/density/src/algorithms/chameleon/core/chameleon_decode.h deleted file mode 100644 index cbf06e26a44..00000000000 --- a/contrib/density/src/algorithms/chameleon/core/chameleon_decode.h +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 23/06/15 21:49 - * - * ------------------- - * Chameleon algorithm - * ------------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Hash based superfast kernel - */ - -#ifndef DENSITY_CHAMELEON_DECODE_H -#define DENSITY_CHAMELEON_DECODE_H - -#include "../dictionary/chameleon_dictionary.h" -#include "../../algorithms.h" - -DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_chameleon_decode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_encode.c b/contrib/density/src/algorithms/chameleon/core/chameleon_encode.c deleted file mode 100644 index 2090cb99173..00000000000 --- a/contrib/density/src/algorithms/chameleon/core/chameleon_encode.c +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 23/06/15 22:02 - * - * ------------------- - * Chameleon algorithm - * ------------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Hash based superfast kernel - */ - -#include "chameleon_encode.h" - -DENSITY_FORCE_INLINE void density_chameleon_encode_prepare_signature(uint8_t **DENSITY_RESTRICT out, density_chameleon_signature **DENSITY_RESTRICT signature_pointer, density_chameleon_signature *const DENSITY_RESTRICT signature) { - *signature = 0; - *signature_pointer = (density_chameleon_signature *) *out; - *out += sizeof(density_chameleon_signature); -} - -DENSITY_FORCE_INLINE void density_chameleon_encode_kernel(uint8_t **DENSITY_RESTRICT out, const uint16_t hash, const uint_fast8_t shift, density_chameleon_signature *const DENSITY_RESTRICT signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { - density_chameleon_dictionary_entry *const found = &dictionary->entries[hash]; - - switch (*unit ^ found->as_uint32_t) { - case 0: - *signature |= ((uint64_t) DENSITY_CHAMELEON_SIGNATURE_FLAG_MAP << shift); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - break; - default: - found->as_uint32_t = *unit; // Does not ensure dictionary content consistency between endiannesses - DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); - *out += sizeof(uint32_t); - break; - } -} - -DENSITY_FORCE_INLINE void density_chameleon_encode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, const uint_fast8_t shift, density_chameleon_signature *const DENSITY_RESTRICT signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { - DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); - density_chameleon_encode_kernel(out, DENSITY_CHAMELEON_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)), shift, signature, dictionary, unit); - *in += sizeof(uint32_t); -} - -DENSITY_FORCE_INLINE void density_chameleon_encode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, density_chameleon_signature *const DENSITY_RESTRICT signature, density_chameleon_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { - uint_fast8_t count = 0; - -#ifdef __clang__ - for (uint_fast8_t count_b = 0; count_b < 32; count_b++) { - DENSITY_UNROLL_2(density_chameleon_encode_4(in, out, count++, signature, dictionary, unit)); - } -#else - for (uint_fast8_t count_b = 0; count_b < 16; count_b++) { - DENSITY_UNROLL_4(density_chameleon_encode_4(in, out, count++, signature, dictionary, unit)); - } -#endif -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_chameleon_encode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { - if (out_size < DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - density_chameleon_signature signature; - density_chameleon_signature *signature_pointer; - uint32_t unit; - - uint8_t *out_limit = *out + out_size - DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE; - uint_fast64_t limit_256 = (in_size >> 8); - - while (DENSITY_LIKELY(limit_256-- && *out <= out_limit)) { - if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { - DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; - } - state->counter++; - if (DENSITY_UNLIKELY(state->copy_penalty)) { - DENSITY_ALGORITHM_COPY(DENSITY_CHAMELEON_WORK_BLOCK_SIZE); - DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; - } else { - const uint8_t *out_start = *out; - density_chameleon_encode_prepare_signature(out, &signature_pointer, &signature); - DENSITY_PREFETCH(*in + DENSITY_CHAMELEON_WORK_BLOCK_SIZE); - density_chameleon_encode_256(in, out, &signature, (density_chameleon_dictionary *const) state->dictionary, &unit); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_chameleon_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_chameleon_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_chameleon_signature)); -#else -#error -#endif - DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*out - out_start), DENSITY_CHAMELEON_WORK_BLOCK_SIZE); - } - } - - if (*out > out_limit) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - uint_fast64_t remaining; - - switch (in_size & 0xff) { - case 0: - case 1: - case 2: - case 3: - density_chameleon_encode_prepare_signature(out, &signature_pointer, &signature); - signature = ((uint64_t) DENSITY_CHAMELEON_SIGNATURE_FLAG_CHUNK); // End marker -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_chameleon_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_chameleon_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_chameleon_signature)); -#else -#error -#endif - goto process_remaining_bytes; - default: - break; - } - - const uint_fast64_t limit_4 = (in_size & 0xff) >> 2; - density_chameleon_encode_prepare_signature(out, &signature_pointer, &signature); - for (uint_fast8_t shift = 0; shift != limit_4; shift++) - density_chameleon_encode_4(in, out, shift, &signature, (density_chameleon_dictionary *const) state->dictionary, &unit); - - signature |= ((uint64_t) DENSITY_CHAMELEON_SIGNATURE_FLAG_CHUNK << limit_4); // End marker -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_chameleon_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_chameleon_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_chameleon_signature)); -#else -#error -#endif - - process_remaining_bytes: - remaining = in_size & 0x3; - if (remaining) - DENSITY_ALGORITHM_COPY(remaining); - - return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; -} diff --git a/contrib/density/src/algorithms/chameleon/core/chameleon_encode.h b/contrib/density/src/algorithms/chameleon/core/chameleon_encode.h deleted file mode 100644 index f71fc23fbc0..00000000000 --- a/contrib/density/src/algorithms/chameleon/core/chameleon_encode.h +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 23/06/15 21:51 - * - * ------------------- - * Chameleon algorithm - * ------------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Hash based superfast kernel - */ - -#ifndef DENSITY_CHAMELEON_ENCODE_H -#define DENSITY_CHAMELEON_ENCODE_H - -#include "../dictionary/chameleon_dictionary.h" -#include "../../algorithms.h" - -DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_chameleon_encode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h b/contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h deleted file mode 100644 index 1f584a5936e..00000000000 --- a/contrib/density/src/algorithms/chameleon/dictionary/chameleon_dictionary.h +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/10/13 12:05 - * - * ------------------- - * Chameleon algorithm - * ------------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Hash based superfast kernel - */ - -#ifndef DENSITY_CHAMELEON_DICTIONARY_H -#define DENSITY_CHAMELEON_DICTIONARY_H - -#include "../chameleon.h" - -#include - -#pragma pack(push) -#pragma pack(4) -typedef struct { - uint32_t as_uint32_t; -} density_chameleon_dictionary_entry; - -typedef struct { - density_chameleon_dictionary_entry entries[1 << DENSITY_CHAMELEON_HASH_BITS]; -} density_chameleon_dictionary; -#pragma pack(pop) - -#endif diff --git a/contrib/density/src/algorithms/cheetah/cheetah.h b/contrib/density/src/algorithms/cheetah/cheetah.h deleted file mode 100644 index 0d47d0ab502..00000000000 --- a/contrib/density/src/algorithms/cheetah/cheetah.h +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 06/12/13 20:10 - * - * ----------------- - * Cheetah algorithm - * ----------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * Piotr Tarsa (https://github.com/tarsa) - * - * Description - * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup - */ - -#ifndef DENSITY_CHEETAH_H -#define DENSITY_CHEETAH_H - -#include "../../globals.h" - -#define DENSITY_CHEETAH_HASH_BITS 16 -#define DENSITY_CHEETAH_HASH_MULTIPLIER (uint32_t)0x9D6EF916lu - -#define DENSITY_CHEETAH_HASH_ALGORITHM(value32) (uint16_t)(value32 * DENSITY_CHEETAH_HASH_MULTIPLIER >> (32 - DENSITY_CHEETAH_HASH_BITS)) - -typedef enum { - DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED = 0x0, - DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A = 0x1, - DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B = 0x2, - DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK = 0x3, -} DENSITY_CHEETAH_SIGNATURE_FLAG; - -typedef uint64_t density_cheetah_signature; - -#define DENSITY_CHEETAH_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE ((density_bitsizeof(density_cheetah_signature) >> 1) * sizeof(uint32_t)) // Uncompressed chunks -#define DENSITY_CHEETAH_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE ((density_bitsizeof(density_cheetah_signature) >> 1) * sizeof(uint32_t)) - -#define DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE (sizeof(density_cheetah_signature) + DENSITY_CHEETAH_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE) -#define DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE (DENSITY_CHEETAH_DECOMPRESSED_BODY_SIZE_PER_SIGNATURE) - -#define DENSITY_CHEETAH_WORK_BLOCK_SIZE 128 - -#endif \ No newline at end of file diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_decode.c b/contrib/density/src/algorithms/cheetah/core/cheetah_decode.c deleted file mode 100644 index 4928938b41e..00000000000 --- a/contrib/density/src/algorithms/cheetah/core/cheetah_decode.c +++ /dev/null @@ -1,266 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/06/15 0:32 - * - * ----------------- - * Cheetah algorithm - * ----------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * Piotr Tarsa (https://github.com/tarsa) - * - * Description - * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup - */ - -#include "cheetah_decode.h" - -DENSITY_FORCE_INLINE void density_cheetah_decode_process_predicted(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { - const uint32_t unit = dictionary->prediction_entries[*last_hash].next_chunk_prediction; - DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); - *last_hash = DENSITY_CHEETAH_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(unit)); -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_process_compressed_a(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, const uint16_t hash) { - DENSITY_PREFETCH(&dictionary->prediction_entries[hash]); - const uint32_t unit = dictionary->entries[hash].chunk_a; - DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); - dictionary->prediction_entries[*last_hash].next_chunk_prediction = unit; - *last_hash = hash; -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_process_compressed_b(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, const uint16_t hash) { - DENSITY_PREFETCH(&dictionary->prediction_entries[hash]); - density_cheetah_dictionary_entry *const entry = &dictionary->entries[hash]; - const uint32_t unit = entry->chunk_b; - entry->chunk_b = entry->chunk_a; - entry->chunk_a = unit; // Does not ensure dictionary content consistency between endiannesses - DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); - dictionary->prediction_entries[*last_hash].next_chunk_prediction = unit; - *last_hash = hash; -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_process_uncompressed(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, const uint32_t unit) { - const uint16_t hash = DENSITY_CHEETAH_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(unit)); - DENSITY_PREFETCH(&dictionary->prediction_entries[hash]); - density_cheetah_dictionary_entry *const entry = &dictionary->entries[hash]; - entry->chunk_b = entry->chunk_a; - entry->chunk_a = unit; // Does not ensure dictionary content consistency between endiannesses - DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); - dictionary->prediction_entries[*last_hash].next_chunk_prediction = unit; // Does not ensure dictionary content consistency between endiannesses - *last_hash = hash; -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_kernel_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint8_t flag, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { - uint16_t hash; - uint32_t unit; - - switch (flag) { - case DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED: - density_cheetah_decode_process_predicted(out, last_hash, dictionary); - break; - case DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A: - DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); - density_cheetah_decode_process_compressed_a(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash)); - *in += sizeof(uint16_t); - break; - case DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B: - DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); - density_cheetah_decode_process_compressed_b(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash)); - *in += sizeof(uint16_t); - break; - default: // DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK - DENSITY_MEMCPY(&unit, *in, sizeof(uint32_t)); - density_cheetah_decode_process_uncompressed(out, last_hash, dictionary, unit); - *in += sizeof(uint32_t); - break; - } - - *out += sizeof(uint32_t); -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_kernel_16(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint8_t flags, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { - uint16_t hash; - uint32_t unit; - - switch (flags) { - DENSITY_CASE_GENERATOR_4_4_COMBINED(\ - density_cheetah_decode_process_predicted(out, last_hash, dictionary);, \ - DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED, \ - DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); \ - density_cheetah_decode_process_compressed_a(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash));\ - *in += sizeof(uint16_t);, \ - DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A, \ - DENSITY_MEMCPY(&hash, *in, sizeof(uint16_t)); \ - density_cheetah_decode_process_compressed_b(out, last_hash, dictionary, DENSITY_LITTLE_ENDIAN_16(hash));\ - *in += sizeof(uint16_t);, \ - DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B, \ - DENSITY_MEMCPY(&unit, *in, sizeof(uint32_t)); \ - density_cheetah_decode_process_uncompressed(out, last_hash, dictionary, unit);\ - *in += sizeof(uint32_t);, \ - DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK, \ - *out += sizeof(uint32_t);, \ - 2\ - ); - default: - break; - } - - *out += sizeof(uint32_t); -} - -DENSITY_FORCE_INLINE uint8_t density_cheetah_decode_read_flag(const density_cheetah_signature signature, const uint_fast8_t shift) { - return (uint8_t const) ((signature >> shift) & 0x3); -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const density_cheetah_signature signature, const uint_fast8_t shift, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { - density_cheetah_decode_kernel_4(in, out, last_hash, density_cheetah_decode_read_flag(signature, shift), dictionary); -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_16(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const density_cheetah_signature signature, const uint_fast8_t shift, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { - density_cheetah_decode_kernel_16(in, out, last_hash, (uint8_t const) ((signature >> shift) & 0xff), dictionary); -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_128(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const density_cheetah_signature signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary) { -#ifdef __clang__ - uint_fast8_t count = 0; - for (uint_fast8_t count_b = 0; count_b < 8; count_b ++) { - density_cheetah_decode_16(in, out, last_hash, signature, count, dictionary); - count += 8; - } -#else - for (uint_fast8_t count_b = 0; count_b < density_bitsizeof(density_cheetah_signature); count_b += 8) - density_cheetah_decode_16(in, out, last_hash, signature, count_b, dictionary); -#endif -} - -DENSITY_FORCE_INLINE void density_cheetah_decode_read_signature(const uint8_t **DENSITY_RESTRICT in, density_cheetah_signature *DENSITY_RESTRICT signature) { -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature, *in, sizeof(density_cheetah_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - density_cheetah_signature endian_signature; - DENSITY_MEMCPY(&endian_signature, *in, sizeof(density_cheetah_signature)); - *signature = DENSITY_LITTLE_ENDIAN_64(endian_signature); -#else -#error -#endif - *in += sizeof(density_cheetah_signature); -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_cheetah_decode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { - if (out_size < DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - density_cheetah_signature signature; - uint_fast8_t shift; - uint_fast64_t remaining; - uint_fast16_t last_hash = 0; - uint8_t flag; - - const uint8_t *start = *in; - - if (in_size < DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE) { - goto read_signature; - } - - const uint8_t *in_limit = *in + in_size - DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE; - uint8_t *out_limit = *out + out_size - DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE; - - while (DENSITY_LIKELY(*in <= in_limit && *out <= out_limit)) { - if (DENSITY_UNLIKELY(!(state->counter & 0x1f))) { - DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; - } - state->counter++; - if (DENSITY_UNLIKELY(state->copy_penalty)) { - DENSITY_ALGORITHM_COPY(DENSITY_CHEETAH_WORK_BLOCK_SIZE); - DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; - } else { - const uint8_t *in_start = *in; - density_cheetah_decode_read_signature(in, &signature); - density_cheetah_decode_128(in, out, &last_hash, signature, (density_cheetah_dictionary *const) state->dictionary); - DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*in - in_start), DENSITY_CHEETAH_WORK_BLOCK_SIZE); - } - } - - if (*out > out_limit) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - read_signature: - if (in_size - (*in - start) < sizeof(density_cheetah_signature)) - return DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL; - shift = 0; - density_cheetah_decode_read_signature(in, &signature); - read_and_decode_4: - switch (in_size - (*in - start)) { - case 0: - case 1: - switch (density_cheetah_decode_read_flag(signature, shift)) { - case DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK: - goto process_remaining_bytes; // End marker - case DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED: - density_cheetah_decode_kernel_4(in, out, &last_hash, DENSITY_CHEETAH_SIGNATURE_FLAG_PREDICTED, (density_cheetah_dictionary *const) state->dictionary); - shift += 2; - break; - default: - return DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; - } - break; - case 2: - case 3: - flag = density_cheetah_decode_read_flag(signature, shift); - switch (flag) { - case DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK: - goto process_remaining_bytes; // End marker - default: - density_cheetah_decode_kernel_4(in, out, &last_hash, flag, (density_cheetah_dictionary *const) state->dictionary); - shift += 2; - break; - } - break; - default: - density_cheetah_decode_4(in, out, &last_hash, signature, shift, (density_cheetah_dictionary *const) state->dictionary); - shift += 2; - break; - } - - if (DENSITY_UNLIKELY(shift == density_bitsizeof(density_cheetah_signature))) - goto read_signature; - else - goto read_and_decode_4; - - process_remaining_bytes: - remaining = in_size - (*in - start); - DENSITY_ALGORITHM_COPY(remaining); - - return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; -} diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_decode.h b/contrib/density/src/algorithms/cheetah/core/cheetah_decode.h deleted file mode 100644 index 278368e7938..00000000000 --- a/contrib/density/src/algorithms/cheetah/core/cheetah_decode.h +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/06/15 0:31 - * - * ----------------- - * Cheetah algorithm - * ----------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * Piotr Tarsa (https://github.com/tarsa) - * - * Description - * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup - */ - -#ifndef DENSITY_CHEETAH_DECODE_H -#define DENSITY_CHEETAH_DECODE_H - -#include "../dictionary/cheetah_dictionary.h" -#include "../../algorithms.h" - -DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_cheetah_decode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_encode.c b/contrib/density/src/algorithms/cheetah/core/cheetah_encode.c deleted file mode 100644 index 6cec16f4bf0..00000000000 --- a/contrib/density/src/algorithms/cheetah/core/cheetah_encode.c +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 23/06/15 23:29 - * - * ----------------- - * Cheetah algorithm - * ----------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * Piotr Tarsa (https://github.com/tarsa) - * - * Description - * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup - */ - -#include "cheetah_encode.h" - -DENSITY_FORCE_INLINE void density_cheetah_encode_prepare_signature(uint8_t **DENSITY_RESTRICT out, density_cheetah_signature **DENSITY_RESTRICT signature_pointer, density_cheetah_signature *const DENSITY_RESTRICT signature) { - *signature = 0; - *signature_pointer = (density_cheetah_signature *) *out; - *out += sizeof(density_cheetah_signature); -} - -DENSITY_FORCE_INLINE void density_cheetah_encode_kernel(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint_fast16_t hash, const uint_fast8_t shift, density_cheetah_signature *const DENSITY_RESTRICT signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { - uint32_t *predictedChunk = (uint32_t*) &dictionary->prediction_entries[*last_hash]; - - if (*predictedChunk ^ *unit) { - density_cheetah_dictionary_entry *found = &dictionary->entries[hash]; - uint32_t *found_a = &found->chunk_a; - if (*found_a ^ *unit) { - uint32_t *found_b = &found->chunk_b; - if (*found_b ^ *unit) { - *signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK << shift); - DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); - *out += sizeof(uint32_t); - } else { - *signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_B << shift); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - } - *found_b = *found_a; - *found_a = *unit; // Does not ensure dictionary content consistency between endiannesses - } else { - *signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_MAP_A << shift); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - } - *predictedChunk = *unit; // Does not ensure dictionary content consistency between endiannesses - } - *last_hash = hash; -} - -DENSITY_FORCE_INLINE void density_cheetah_encode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, const uint_fast8_t shift, density_cheetah_signature *const DENSITY_RESTRICT signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { - DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); - *in += sizeof(uint32_t); - density_cheetah_encode_kernel(out, last_hash, DENSITY_CHEETAH_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)), shift, signature, dictionary, unit); -} - -DENSITY_FORCE_INLINE void density_cheetah_encode_128(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_cheetah_signature *const DENSITY_RESTRICT signature, density_cheetah_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT unit) { - uint_fast8_t count = 0; - -#ifdef __clang__ - for(; count < density_bitsizeof(density_cheetah_signature); count += 2) { - density_cheetah_encode_4(in, out, last_hash, count, signature, dictionary, unit); - } -#else - for (uint_fast8_t count_b = 0; count_b < 16; count_b++) { - DENSITY_UNROLL_2(\ - density_cheetah_encode_4(in, out, last_hash, count, signature, dictionary, unit);\ - count += 2); - } -#endif -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_cheetah_encode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { - if (out_size < DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - density_cheetah_signature signature; - density_cheetah_signature *signature_pointer; - uint_fast16_t last_hash = 0; - uint32_t unit; - - uint8_t *out_limit = *out + out_size - DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE; - uint_fast64_t limit_128 = (in_size >> 7); - - while (DENSITY_LIKELY(limit_128-- && *out <= out_limit)) { - if (DENSITY_UNLIKELY(!(state->counter & 0x1f))) { - DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; - } - state->counter++; - if (DENSITY_UNLIKELY(state->copy_penalty)) { - DENSITY_ALGORITHM_COPY(DENSITY_CHEETAH_WORK_BLOCK_SIZE); - DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; - } else { - const uint8_t *out_start = *out; - density_cheetah_encode_prepare_signature(out, &signature_pointer, &signature); - DENSITY_PREFETCH(*in + DENSITY_CHEETAH_WORK_BLOCK_SIZE); - density_cheetah_encode_128(in, out, &last_hash, &signature, (density_cheetah_dictionary *const) state->dictionary, &unit); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_cheetah_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_cheetah_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_cheetah_signature)); -#else -#error -#endif - DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*out - out_start), DENSITY_CHEETAH_WORK_BLOCK_SIZE); - } - } - - if (*out > out_limit) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - uint_fast64_t remaining; - - switch (in_size & 0x7f) { - case 0: - case 1: - case 2: - case 3: - density_cheetah_encode_prepare_signature(out, &signature_pointer, &signature); - signature = (uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK; // End marker -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_cheetah_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_cheetah_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_cheetah_signature)); -#else -#error -#endif - goto process_remaining_bytes; - default: - break; - } - - const uint_fast64_t limit_4 = ((in_size & 0x7f) >> 2) << 1; // 4-byte units times number of signature flag bits - density_cheetah_encode_prepare_signature(out, &signature_pointer, &signature); - for (uint_fast8_t shift = 0; shift != limit_4; shift += 2) - density_cheetah_encode_4(in, out, &last_hash, shift, &signature, (density_cheetah_dictionary *const) state->dictionary, &unit); - - signature |= ((uint64_t) DENSITY_CHEETAH_SIGNATURE_FLAG_CHUNK << limit_4); // End marker -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_cheetah_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_cheetah_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_cheetah_signature)); -#else -#error -#endif - - process_remaining_bytes: - remaining = in_size & 0x3; - if (remaining) - DENSITY_ALGORITHM_COPY(remaining); - - return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; -} diff --git a/contrib/density/src/algorithms/cheetah/core/cheetah_encode.h b/contrib/density/src/algorithms/cheetah/core/cheetah_encode.h deleted file mode 100644 index 9f67e5bad9d..00000000000 --- a/contrib/density/src/algorithms/cheetah/core/cheetah_encode.h +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 23/06/15 23:29 - * - * ----------------- - * Cheetah algorithm - * ----------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * Piotr Tarsa (https://github.com/tarsa) - * - * Description - * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup - */ - -#ifndef DENSITY_CHEETAH_ENCODE_H -#define DENSITY_CHEETAH_ENCODE_H - -#include "../dictionary/cheetah_dictionary.h" -#include "../../algorithms.h" - -DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_cheetah_encode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h b/contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h deleted file mode 100644 index 5223d843574..00000000000 --- a/contrib/density/src/algorithms/cheetah/dictionary/cheetah_dictionary.h +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 06/12/13 20:20 - * - * ----------------- - * Cheetah algorithm - * ----------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * Piotr Tarsa (https://github.com/tarsa) - * - * Description - * Very fast two level dictionary hash algorithm derived from Chameleon, with predictions lookup - */ - -#ifndef DENSITY_CHEETAH_DICTIONARY_H -#define DENSITY_CHEETAH_DICTIONARY_H - -#include "../cheetah.h" - -#include - -#pragma pack(push) -#pragma pack(4) -typedef struct { - uint32_t chunk_a; - uint32_t chunk_b; -} density_cheetah_dictionary_entry; - -typedef struct { - uint32_t next_chunk_prediction; -} density_cheetah_dictionary_prediction_entry; - -typedef struct { - density_cheetah_dictionary_entry entries[1 << DENSITY_CHEETAH_HASH_BITS]; - density_cheetah_dictionary_prediction_entry prediction_entries[1 << DENSITY_CHEETAH_HASH_BITS]; -} density_cheetah_dictionary; -#pragma pack(pop) - -#endif diff --git a/contrib/density/src/algorithms/dictionaries.c b/contrib/density/src/algorithms/dictionaries.c deleted file mode 100644 index 1e8d5466014..00000000000 --- a/contrib/density/src/algorithms/dictionaries.c +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Density - * - * Copyright (c) 2018, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 13/01/18 17:31 - */ - -#include "dictionaries.h" - -DENSITY_WINDOWS_EXPORT size_t density_get_dictionary_size(DENSITY_ALGORITHM algorithm) { - switch(algorithm) { - case DENSITY_ALGORITHM_CHAMELEON: - return sizeof(density_chameleon_dictionary); - case DENSITY_ALGORITHM_CHEETAH: - return sizeof(density_cheetah_dictionary); - case DENSITY_ALGORITHM_LION: - return sizeof(density_lion_dictionary); - default: - return 0; - } -} diff --git a/contrib/density/src/algorithms/dictionaries.h b/contrib/density/src/algorithms/dictionaries.h deleted file mode 100644 index 291c361963e..00000000000 --- a/contrib/density/src/algorithms/dictionaries.h +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Density - * - * Copyright (c) 2018, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 13/01/18 17:26 - */ - -#ifndef DENSITY_DICTIONARIES_H -#define DENSITY_DICTIONARIES_H - -#include "../globals.h" -#include "../algorithms/chameleon/dictionary/chameleon_dictionary.h" -#include "../algorithms/cheetah/dictionary/cheetah_dictionary.h" -#include "../algorithms/lion/dictionary/lion_dictionary.h" - -DENSITY_WINDOWS_EXPORT size_t density_get_dictionary_size(DENSITY_ALGORITHM); - -#endif diff --git a/contrib/density/src/algorithms/lion/core/lion_decode.c b/contrib/density/src/algorithms/lion/core/lion_decode.c deleted file mode 100644 index c84262c0d50..00000000000 --- a/contrib/density/src/algorithms/lion/core/lion_decode.c +++ /dev/null @@ -1,327 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/06/15 20:55 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#include "lion_decode.h" - -DENSITY_FORCE_INLINE void density_lion_decode_read_signature(const uint8_t **DENSITY_RESTRICT in, uint_fast64_t *const DENSITY_RESTRICT signature) { -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature, *in, sizeof(density_lion_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - density_lion_signature endian_signature; - DENSITY_MEMCPY(&endian_signature, *in, sizeof(density_lion_signature)); - *signature = DENSITY_LITTLE_ENDIAN_64(endian_signature); -#else -#error -#endif - *in += sizeof(density_lion_signature); -} - -DENSITY_FORCE_INLINE void density_lion_decode_update_predictions_model(density_lion_dictionary_chunk_prediction_entry *const DENSITY_RESTRICT predictions, const uint32_t chunk) { - DENSITY_MEMMOVE((uint32_t *) predictions + 1, predictions, 2 * sizeof(uint32_t)); - *(uint32_t *) predictions = chunk; // Move chunk to the top of the predictions list, does not ensure dictionary content consistency between endiannesses -} - -DENSITY_FORCE_INLINE void density_lion_decode_update_dictionary_model(density_lion_dictionary_chunk_entry *const DENSITY_RESTRICT entry, const uint32_t chunk) { - DENSITY_MEMMOVE((uint32_t *) entry + 1, entry, 3 * sizeof(uint32_t)); - *(uint32_t *) entry = chunk; // Does not ensure dictionary content consistency between endiannesses -} - -DENSITY_FORCE_INLINE void density_lion_decode_read_hash(const uint8_t **DENSITY_RESTRICT in, uint16_t *DENSITY_RESTRICT const hash) { -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(hash, *in, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - uint16_t endian_hash; - DENSITY_MEMCPY(&endian_hash, *in, sizeof(uint16_t)); - *hash = DENSITY_LITTLE_ENDIAN_16(endian_hash); -#else -#error -#endif - *in += sizeof(uint16_t); -} - -DENSITY_FORCE_INLINE void density_lion_decode_prediction_generic(uint8_t **DENSITY_RESTRICT out, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - *hash = DENSITY_LION_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)); - DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); - *out += sizeof(uint32_t); -} - -DENSITY_FORCE_INLINE void density_lion_decode_dictionary_generic(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint32_t *DENSITY_RESTRICT const unit) { - DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); - *out += sizeof(uint32_t); - density_lion_dictionary_chunk_prediction_entry *prediction = &(dictionary->predictions[*last_hash]); - density_lion_decode_update_predictions_model(prediction, *unit); -} - -void density_lion_decode_prediction_a(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - (void)in; - *unit = dictionary->predictions[*last_hash].next_chunk_a; - density_lion_decode_prediction_generic(out, hash, unit); - - *last_hash = *hash; -} - -void density_lion_decode_prediction_b(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - (void)in; - density_lion_dictionary_chunk_prediction_entry *const prediction = &dictionary->predictions[*last_hash]; - *unit = prediction->next_chunk_b; - density_lion_decode_update_predictions_model(prediction, *unit); - density_lion_decode_prediction_generic(out, hash, unit); - - *last_hash = *hash; -} - -void density_lion_decode_prediction_c(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - (void)in; - density_lion_dictionary_chunk_prediction_entry *const prediction = &dictionary->predictions[*last_hash]; - *unit = prediction->next_chunk_c; - density_lion_decode_update_predictions_model(prediction, *unit); - density_lion_decode_prediction_generic(out, hash, unit); - - *last_hash = *hash; -} - -void density_lion_decode_dictionary_a(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - density_lion_decode_read_hash(in, hash); - DENSITY_PREFETCH(&dictionary->predictions[*hash]); - *unit = dictionary->chunks[*hash].chunk_a; - density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); - - *last_hash = *hash; -} - -void density_lion_decode_dictionary_b(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - density_lion_decode_read_hash(in, hash); - DENSITY_PREFETCH(&dictionary->predictions[*hash]); - density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; - *unit = entry->chunk_b; - density_lion_decode_update_dictionary_model(entry, *unit); - density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); - - *last_hash = *hash; -} - -void density_lion_decode_dictionary_c(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - density_lion_decode_read_hash(in, hash); - DENSITY_PREFETCH(&dictionary->predictions[*hash]); - density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; - *unit = entry->chunk_c; - density_lion_decode_update_dictionary_model(entry, *unit); - density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); - - *last_hash = *hash; -} - -void density_lion_decode_dictionary_d(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - density_lion_decode_read_hash(in, hash); - DENSITY_PREFETCH(&dictionary->predictions[*hash]); - density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; - *unit = entry->chunk_d; - density_lion_decode_update_dictionary_model(entry, *unit); - density_lion_decode_dictionary_generic(out, last_hash, dictionary, unit); - - *last_hash = *hash; -} - -void density_lion_decode_plain(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, uint16_t *DENSITY_RESTRICT const hash, uint32_t *DENSITY_RESTRICT const unit) { - DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); - *in += sizeof(uint32_t); - *hash = DENSITY_LION_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)); - density_lion_dictionary_chunk_entry *entry = &dictionary->chunks[*hash]; - density_lion_decode_update_dictionary_model(entry, *unit); - DENSITY_MEMCPY(*out, unit, sizeof(uint32_t)); - *out += sizeof(uint32_t); - density_lion_dictionary_chunk_prediction_entry *prediction = &(dictionary->predictions[*last_hash]); - density_lion_decode_update_predictions_model(prediction, *unit); - - *last_hash = *hash; -} - -DENSITY_FORCE_INLINE void density_lion_decode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, const DENSITY_LION_FORM form) { - uint16_t hash; - uint32_t unit; - - data->attachments[form](in, out, last_hash, dictionary, &hash, &unit); -} - -DENSITY_FORCE_INLINE DENSITY_LION_FORM density_lion_decode_read_form(const uint8_t **DENSITY_RESTRICT in, uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_form_data *const form_data) { - const uint_fast8_t trailing_zeroes = DENSITY_CTZ(0x80 | (*signature >> *shift)); - if (DENSITY_LIKELY(!trailing_zeroes)) { - *shift = (uint_fast8_t)((*shift + 1) & 0x3f); - return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool); - } else if (DENSITY_LIKELY(trailing_zeroes <= 6)) { - *shift = (uint_fast8_t)((*shift + (trailing_zeroes + 1)) & 0x3f); - return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool + trailing_zeroes); - } else { - if (DENSITY_LIKELY(*shift <= (density_bitsizeof(density_lion_signature) - 7))) { - *shift = (uint_fast8_t)((*shift + 7) & 0x3f); - return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool + 7); - } else { - density_lion_decode_read_signature(in, signature); - const uint_fast8_t primary_trailing_zeroes = (uint_fast8_t)(density_bitsizeof(density_lion_signature) - *shift); - const uint_fast8_t ctz_barrier_shift = (uint_fast8_t)(7 - primary_trailing_zeroes); - const uint_fast8_t secondary_trailing_zeroes = DENSITY_CTZ(((uint64_t)1 << ctz_barrier_shift) | *signature); - if (DENSITY_LIKELY(secondary_trailing_zeroes != ctz_barrier_shift)) - *shift = (uint_fast8_t)(secondary_trailing_zeroes + 1); - else - *shift = secondary_trailing_zeroes; - return density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool + primary_trailing_zeroes + secondary_trailing_zeroes); - } - } -} - -DENSITY_FORCE_INLINE void density_lion_decode_process_form(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const form_data, uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift) { - if (DENSITY_UNLIKELY(!*shift)) - density_lion_decode_read_signature(in, signature); - - switch ((*signature >> *shift) & 0x1) { - case 0: - density_lion_decode_4(in, out, last_hash, dictionary, form_data, density_lion_decode_read_form(in, signature, shift, form_data)); - break; - default: - density_lion_decode_4(in, out, last_hash, dictionary, form_data, density_lion_form_model_increment_usage(form_data, (density_lion_form_node *) form_data->formsPool)); - *shift = (uint_fast8_t)((*shift + 1) & 0x3f); - break; - } -} - -DENSITY_FORCE_INLINE void density_lion_decode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const form_data, uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift) { -#ifdef __clang__ - for (uint_fast8_t count = 0; count < (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG >> 2); count++) { - DENSITY_UNROLL_4(density_lion_decode_process_form(in, out, last_hash, dictionary, form_data, signature, shift)); - } -#else - for (uint_fast8_t count = 0; count < (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG >> 2); count++) { - DENSITY_UNROLL_4(density_lion_decode_process_form(in, out, last_hash, dictionary, form_data, signature, shift)); - } -#endif -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_lion_decode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { - if (out_size < DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - density_lion_signature signature = 0; - density_lion_form_data data; - density_lion_form_model_init(&data); - void (*attachments[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const) = {(void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_prediction_a, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_prediction_b, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_prediction_c, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_a, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_b, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_c, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_dictionary_d, (void (*)(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) &density_lion_decode_plain}; - density_lion_form_model_attach(&data, attachments); - uint_fast8_t shift = 0; - uint_fast64_t remaining; - uint_fast16_t last_hash = 0; - DENSITY_LION_FORM form; - - const uint8_t *start = *in; - - if (in_size < DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE) { - goto read_and_decode_4; - } - - const uint8_t *in_limit = *in + in_size - DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE; - uint8_t *out_limit = *out + out_size - DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE; - - while (DENSITY_LIKELY(*in <= in_limit && *out <= out_limit)) { - if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { - DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; - } - state->counter++; - if (DENSITY_UNLIKELY(state->copy_penalty)) { - DENSITY_ALGORITHM_COPY(DENSITY_LION_WORK_BLOCK_SIZE); - DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; - } else { - const uint8_t *in_start = *in; - density_lion_decode_256(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, &signature, &shift); - DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*in - in_start), DENSITY_LION_WORK_BLOCK_SIZE); - } - } - - if (*out > out_limit) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - read_and_decode_4: - if (DENSITY_UNLIKELY(!shift)) { - if (in_size - (*in - start) < sizeof(density_lion_signature)) - return DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL; - - density_lion_decode_read_signature(in, &signature); - } - form = density_lion_decode_read_form(in, &signature, &shift, &data); - switch (in_size - (*in - start)) { - case 0: - case 1: - switch (form) { - case DENSITY_LION_FORM_PLAIN: - goto process_remaining_bytes; // End marker - case DENSITY_LION_FORM_PREDICTIONS_A: - case DENSITY_LION_FORM_PREDICTIONS_B: - case DENSITY_LION_FORM_PREDICTIONS_C: - density_lion_decode_4(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, form); - break; - default: - return DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; // Not enough bytes to read a hash - } - break; - case 2: - case 3: - switch (form) { - case DENSITY_LION_FORM_PLAIN: - goto process_remaining_bytes; // End marker - default: - density_lion_decode_4(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, form); - break; - } - break; - default: - density_lion_decode_4(in, out, &last_hash, (density_lion_dictionary *const) state->dictionary, &data, form); - break; - } - goto read_and_decode_4; - - process_remaining_bytes: - remaining = in_size - (*in - start); - DENSITY_MEMCPY(*out, *in, remaining); - *in += remaining; - *out += remaining; - - return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; -} diff --git a/contrib/density/src/algorithms/lion/core/lion_decode.h b/contrib/density/src/algorithms/lion/core/lion_decode.h deleted file mode 100644 index 9972f0be388..00000000000 --- a/contrib/density/src/algorithms/lion/core/lion_decode.h +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/06/15 20:55 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#ifndef DENSITY_LION_DECODE_H -#define DENSITY_LION_DECODE_H - -#include "../dictionary/lion_dictionary.h" -#include "../forms/lion_form_model.h" -#include "../../algorithms.h" - -DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_lion_decode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/algorithms/lion/core/lion_encode.c b/contrib/density/src/algorithms/lion/core/lion_encode.c deleted file mode 100644 index 7cff2f3dda7..00000000000 --- a/contrib/density/src/algorithms/lion/core/lion_encode.c +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/06/15 18:57 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#include "lion_encode.h" - -DENSITY_FORCE_INLINE void density_lion_encode_prepare_signature(uint8_t **DENSITY_RESTRICT out, uint_fast64_t **DENSITY_RESTRICT signature_pointer, uint_fast64_t *const DENSITY_RESTRICT signature) { - *signature = 0; - *signature_pointer = (density_lion_signature *) *out; - *out += sizeof(density_lion_signature); -} - -DENSITY_FORCE_INLINE void density_lion_encode_push_to_proximity_signature(uint_fast64_t *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const uint64_t content, const uint_fast8_t bits) { - *signature |= (content << *shift); - *shift += bits; -} - -DENSITY_FORCE_INLINE void density_lion_encode_push_to_signature(uint8_t **DENSITY_RESTRICT out, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const uint64_t content, const uint_fast8_t bits) { - if (DENSITY_LIKELY(*shift)) { - density_lion_encode_push_to_proximity_signature(signature, shift, content, bits); - - if (DENSITY_UNLIKELY(*shift >= density_bitsizeof(density_lion_signature))) { -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*signature_pointer, signature, sizeof(density_lion_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(*signature); - DENSITY_MEMCPY(*signature_pointer, &endian_signature, sizeof(density_lion_signature)); -#else -#error -#endif - - const uint_fast8_t remainder = (uint_fast8_t)(*shift & 0x3f); - *shift = 0; - if (remainder) { - density_lion_encode_prepare_signature(out, signature_pointer, signature); - density_lion_encode_push_to_proximity_signature(signature, shift, content >> (bits - remainder), remainder); - } - } - } else { - density_lion_encode_prepare_signature(out, signature_pointer, signature); - density_lion_encode_push_to_proximity_signature(signature, shift, content, bits); - } -} - -DENSITY_FORCE_INLINE void density_lion_encode_push_zero_to_signature(uint8_t **DENSITY_RESTRICT out, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const uint_fast8_t bits) { - if (DENSITY_LIKELY(*shift)) { - *shift += bits; - - if (DENSITY_UNLIKELY(*shift >= density_bitsizeof(density_lion_signature))) { -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*signature_pointer, signature, sizeof(density_lion_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(*signature); - DENSITY_MEMCPY(*signature_pointer, &endian_signature, sizeof(density_lion_signature)); -#else -#error -#endif - - const uint_fast8_t remainder = (uint_fast8_t)(*shift & 0x3f); - if (remainder) { - density_lion_encode_prepare_signature(out, signature_pointer, signature); - *shift = remainder; - } else - *shift = 0; - } - } else { - density_lion_encode_prepare_signature(out, signature_pointer, signature); - *shift = bits; - } -} - -DENSITY_FORCE_INLINE void density_lion_encode_push_code_to_signature(uint8_t **DENSITY_RESTRICT out, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, const density_lion_entropy_code code) { - density_lion_encode_push_to_signature(out, signature_pointer, signature, shift, code.value, code.bitLength); -} - -DENSITY_FORCE_INLINE void density_lion_encode_kernel_4(uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, const uint16_t hash, density_lion_form_data *const data, const uint32_t unit) { - density_lion_dictionary_chunk_prediction_entry *const predictions = &dictionary->predictions[*last_hash]; - DENSITY_PREFETCH(&dictionary->predictions[hash]); - - if (*(uint32_t *) predictions ^ unit) { - if (*((uint32_t *) predictions + 1) ^ unit) { - if (*((uint32_t *) predictions + 2) ^ unit) { - density_lion_dictionary_chunk_entry *const in_dictionary = &dictionary->chunks[hash]; - if (*(uint32_t *) in_dictionary ^ unit) { - if (*((uint32_t *) in_dictionary + 1) ^ unit) { - if (*((uint32_t *) in_dictionary + 2) ^ unit) { - if (*((uint32_t *) in_dictionary + 3) ^ unit) { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PLAIN)); - DENSITY_MEMCPY(*out, &unit, sizeof(uint32_t)); - *out += sizeof(uint32_t); - } else { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_D)); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - } - } else { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_C)); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - } - } else { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_B)); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - } - DENSITY_MEMMOVE((uint32_t *) in_dictionary + 1, in_dictionary, 3 * sizeof(uint32_t)); - *(uint32_t *) in_dictionary = unit; // Does not ensure dictionary content consistency between endiannesses - } else { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_DICTIONARY_A)); -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(*out, &hash, sizeof(uint16_t)); -#elif defined(DENSITY_BIG_ENDIAN) - const uint16_t endian_hash = DENSITY_LITTLE_ENDIAN_16(hash); - DENSITY_MEMCPY(*out, &endian_hash, sizeof(uint16_t)); -#else -#error -#endif - *out += sizeof(uint16_t); - } - } else { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PREDICTIONS_C)); - } - } else { - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PREDICTIONS_B)); - } - DENSITY_MEMMOVE((uint32_t *) predictions + 1, predictions, 2 * sizeof(uint32_t)); - *(uint32_t *) predictions = unit; // Does not ensure dictionary content consistency between endiannesses - } else - density_lion_encode_push_code_to_signature(out, signature_pointer, signature, shift, density_lion_form_model_get_encoding(data, DENSITY_LION_FORM_PREDICTIONS_A)); - *last_hash = hash; -} - -DENSITY_FORCE_INLINE void density_lion_encode_4(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { - DENSITY_MEMCPY(unit, *in, sizeof(uint32_t)); - density_lion_encode_kernel_4(out, last_hash, signature_pointer, signature, shift, dictionary, DENSITY_LION_HASH_ALGORITHM(DENSITY_LITTLE_ENDIAN_32(*unit)), data, *unit); - *in += sizeof(uint32_t); -} - -DENSITY_FORCE_INLINE void density_lion_encode_generic(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, const uint_fast8_t chunks_per_process_unit, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { -#ifdef __clang__ - for (uint_fast8_t count = 0; count < (chunks_per_process_unit >> 2); count++) { - DENSITY_UNROLL_4(density_lion_encode_4(in, out, last_hash, signature_pointer, signature, shift, dictionary, data, unit)); - } -#else - for (uint_fast8_t count = 0; count < (chunks_per_process_unit >> 1); count++) { - DENSITY_UNROLL_2(density_lion_encode_4(in, out, last_hash, signature_pointer, signature, shift, dictionary, data, unit)); - } -#endif -} - -DENSITY_FORCE_INLINE void density_lion_encode_32(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { - density_lion_encode_generic(in, out, last_hash, signature_pointer, signature, shift, dictionary, DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_SMALL, data, unit); -} - -DENSITY_FORCE_INLINE void density_lion_encode_256(const uint8_t **DENSITY_RESTRICT in, uint8_t **DENSITY_RESTRICT out, uint_fast16_t *DENSITY_RESTRICT last_hash, density_lion_signature **DENSITY_RESTRICT signature_pointer, density_lion_signature *const DENSITY_RESTRICT signature, uint_fast8_t *const DENSITY_RESTRICT shift, density_lion_dictionary *const DENSITY_RESTRICT dictionary, density_lion_form_data *const data, uint32_t *DENSITY_RESTRICT unit) { - density_lion_encode_generic(in, out, last_hash, signature_pointer, signature, shift, dictionary, DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG, data, unit); -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_algorithm_exit_status density_lion_encode(density_algorithm_state *const DENSITY_RESTRICT state, const uint8_t **DENSITY_RESTRICT in, const uint_fast64_t in_size, uint8_t **DENSITY_RESTRICT out, const uint_fast64_t out_size) { - if (out_size < DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - density_lion_signature signature = 0; - density_lion_signature *signature_pointer = NULL; - uint_fast8_t shift = 0; - density_lion_form_data data; - density_lion_form_model_init(&data); - uint_fast16_t last_hash = 0; - uint32_t unit; - - uint8_t *out_limit = *out + out_size - DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE; - uint_fast64_t limit_256 = (in_size >> 8); - - while (DENSITY_LIKELY(limit_256-- && *out <= out_limit)) { - if (DENSITY_UNLIKELY(!(state->counter & 0xf))) { - DENSITY_ALGORITHM_REDUCE_COPY_PENALTY_START; - } - state->counter++; - if (DENSITY_UNLIKELY(state->copy_penalty)) { - DENSITY_ALGORITHM_COPY(DENSITY_LION_WORK_BLOCK_SIZE); - DENSITY_ALGORITHM_INCREASE_COPY_PENALTY_START; - } else { - const uint8_t *out_start = *out; - DENSITY_PREFETCH(*in + DENSITY_LION_WORK_BLOCK_SIZE); - density_lion_encode_256(in, out, &last_hash, &signature_pointer, &signature, &shift, (density_lion_dictionary *const) state->dictionary, &data, &unit); - DENSITY_ALGORITHM_TEST_INCOMPRESSIBILITY((*out - out_start), DENSITY_LION_WORK_BLOCK_SIZE); - } - } - - if (*out > out_limit) - return DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL; - - uint_fast64_t remaining; - - switch (in_size & 0xff) { - case 0: - case 1: - case 2: - case 3: - density_lion_encode_push_code_to_signature(out, &signature_pointer, &signature, &shift, density_lion_form_model_get_encoding(&data, DENSITY_LION_FORM_PLAIN)); // End marker -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_lion_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_lion_signature)); -#else -#error -#endif - goto process_remaining_bytes; - default: - break; - } - - uint_fast64_t limit_4 = (in_size & 0xff) >> 2; - while (limit_4--) - density_lion_encode_4(in, out, &last_hash, &signature_pointer, &signature, &shift, (density_lion_dictionary *const) state->dictionary, &data, &unit); - - density_lion_encode_push_code_to_signature(out, &signature_pointer, &signature, &shift, density_lion_form_model_get_encoding(&data, DENSITY_LION_FORM_PLAIN)); // End marker -#ifdef DENSITY_LITTLE_ENDIAN - DENSITY_MEMCPY(signature_pointer, &signature, sizeof(density_lion_signature)); -#elif defined(DENSITY_BIG_ENDIAN) - const density_lion_signature endian_signature = DENSITY_LITTLE_ENDIAN_64(signature); - DENSITY_MEMCPY(signature_pointer, &endian_signature, sizeof(density_lion_signature)); -#else -#error -#endif - - process_remaining_bytes: - remaining = in_size & 0x3; - if (remaining) { - DENSITY_MEMCPY(*out, *in, remaining); - *in += remaining; - *out += remaining; - } - - return DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED; -} diff --git a/contrib/density/src/algorithms/lion/core/lion_encode.h b/contrib/density/src/algorithms/lion/core/lion_encode.h deleted file mode 100644 index 71cd2cad4cf..00000000000 --- a/contrib/density/src/algorithms/lion/core/lion_encode.h +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 24/06/15 18:56 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#ifndef DENSITY_LION_ENCODE_H -#define DENSITY_LION_ENCODE_H - -#include "../dictionary/lion_dictionary.h" -#include "../forms/lion_form_model.h" -#include "../../algorithms.h" - -DENSITY_WINDOWS_EXPORT density_algorithm_exit_status density_lion_encode(density_algorithm_state *const DENSITY_RESTRICT_DECLARE, const uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t, uint8_t **DENSITY_RESTRICT_DECLARE, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h b/contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h deleted file mode 100644 index 68f71da6373..00000000000 --- a/contrib/density/src/algorithms/lion/dictionary/lion_dictionary.h +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 12/02/15 23:09 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#ifndef DENSITY_LION_DICTIONARY_H -#define DENSITY_LION_DICTIONARY_H - -#include "../lion.h" - -#pragma pack(push) -#pragma pack(4) - -typedef struct { - uint32_t chunk_a; - uint32_t chunk_b; - uint32_t chunk_c; - uint32_t chunk_d; - uint32_t chunk_e; -} density_lion_dictionary_chunk_entry; - -typedef struct { - uint32_t next_chunk_a; - uint32_t next_chunk_b; - uint32_t next_chunk_c; -} density_lion_dictionary_chunk_prediction_entry; - -typedef struct { - density_lion_dictionary_chunk_entry chunks[1 << DENSITY_LION_CHUNK_HASH_BITS]; - density_lion_dictionary_chunk_prediction_entry predictions[1 << DENSITY_LION_CHUNK_HASH_BITS]; -} density_lion_dictionary; -#pragma pack(pop) - -#endif diff --git a/contrib/density/src/algorithms/lion/forms/lion_form_model.c b/contrib/density/src/algorithms/lion/forms/lion_form_model.c deleted file mode 100644 index 21dfc23e3c9..00000000000 --- a/contrib/density/src/algorithms/lion/forms/lion_form_model.c +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 9/03/15 11:19 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#include "lion_form_model.h" - -const density_lion_entropy_code density_lion_form_entropy_codes[DENSITY_LION_NUMBER_OF_FORMS] = DENSITY_LION_FORM_MODEL_ENTROPY_CODES; - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_lion_form_model_init(density_lion_form_data *const data) { - density_lion_form_node *rank_0 = &data->formsPool[0]; - rank_0->form = DENSITY_LION_FORM_PLAIN; - rank_0->rank = 0; - rank_0->previousForm = NULL; - data->formsIndex[DENSITY_LION_FORM_PLAIN] = rank_0; - - density_lion_form_node *rank_1 = &data->formsPool[1]; - rank_1->form = DENSITY_LION_FORM_DICTIONARY_A; - rank_1->rank = 1; - rank_1->previousForm = rank_0; - data->formsIndex[DENSITY_LION_FORM_DICTIONARY_A] = rank_1; - - density_lion_form_node *rank_2 = &data->formsPool[2]; - rank_2->form = DENSITY_LION_FORM_DICTIONARY_B; - rank_2->rank = 2; - rank_2->previousForm = rank_1; - data->formsIndex[DENSITY_LION_FORM_DICTIONARY_B] = rank_2; - - density_lion_form_node *rank_3 = &data->formsPool[3]; - rank_3->form = DENSITY_LION_FORM_PREDICTIONS_A; - rank_3->rank = 3; - rank_3->previousForm = rank_2; - data->formsIndex[DENSITY_LION_FORM_PREDICTIONS_A] = rank_3; - - density_lion_form_node *rank_4 = &data->formsPool[4]; - rank_4->form = DENSITY_LION_FORM_PREDICTIONS_B; - rank_4->rank = 4; - rank_4->previousForm = rank_3; - data->formsIndex[DENSITY_LION_FORM_PREDICTIONS_B] = rank_4; - - density_lion_form_node *rank_5 = &data->formsPool[5]; - rank_5->form = DENSITY_LION_FORM_DICTIONARY_C; - rank_5->rank = 5; - rank_5->previousForm = rank_4; - data->formsIndex[DENSITY_LION_FORM_DICTIONARY_C] = rank_5; - - density_lion_form_node *rank_6 = &data->formsPool[6]; - rank_6->form = DENSITY_LION_FORM_PREDICTIONS_C; - rank_6->rank = 6; - rank_6->previousForm = rank_5; - data->formsIndex[DENSITY_LION_FORM_PREDICTIONS_C] = rank_6; - - density_lion_form_node *rank_7 = &data->formsPool[7]; - rank_7->form = DENSITY_LION_FORM_DICTIONARY_D; - rank_7->rank = 7; - rank_7->previousForm = rank_6; - data->formsIndex[DENSITY_LION_FORM_DICTIONARY_D] = rank_7; - - data->usages.usages_as_uint64_t = 0; -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_lion_form_model_attach(density_lion_form_data *const data, void (*attachments[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)) { - for(uint_fast8_t count = 0; count < DENSITY_LION_NUMBER_OF_FORMS; count ++) - data->attachments[count] = attachments[count]; -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_lion_form_model_update(density_lion_form_data *const DENSITY_RESTRICT data, density_lion_form_node *const DENSITY_RESTRICT form, const uint8_t usage, density_lion_form_node *const DENSITY_RESTRICT previous_form, const uint8_t previous_usage) { - if (DENSITY_UNLIKELY(previous_usage < usage)) { // Relative stability is assumed - const DENSITY_LION_FORM form_value = form->form; - const DENSITY_LION_FORM previous_form_value = previous_form->form; - - previous_form->form = form_value; - form->form = previous_form_value; - - data->formsIndex[form_value] = previous_form; - data->formsIndex[previous_form_value] = form; - } -} - -DENSITY_FORCE_INLINE void density_lion_form_model_flatten(density_lion_form_data *const data, const uint8_t usage) { - if (DENSITY_UNLIKELY(usage & 0x80)) - data->usages.usages_as_uint64_t = (data->usages.usages_as_uint64_t >> 1) & 0x7f7f7f7f7f7f7f7fllu; // Flatten usage values -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE DENSITY_LION_FORM density_lion_form_model_increment_usage(density_lion_form_data *const data, density_lion_form_node *const DENSITY_RESTRICT form) { - const DENSITY_LION_FORM form_value = form->form; - const uint8_t usage = ++data->usages.usages_as_uint8_t[form_value]; - - density_lion_form_node *const previous_form = form->previousForm; - - if (previous_form) - density_lion_form_model_update(data, form, usage, previous_form, data->usages.usages_as_uint8_t[previous_form->form]); - else - density_lion_form_model_flatten(data, usage); - - return form_value; -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE density_lion_entropy_code density_lion_form_model_get_encoding(density_lion_form_data *const data, const DENSITY_LION_FORM form) { - const uint8_t usage = ++data->usages.usages_as_uint8_t[form]; - - density_lion_form_node *const form_found = data->formsIndex[form]; - density_lion_form_node *const previous_form = form_found->previousForm; - - if (previous_form) { - density_lion_form_model_update(data, form_found, usage, previous_form, data->usages.usages_as_uint8_t[previous_form->form]); - - return density_lion_form_entropy_codes[form_found->rank]; - } else { - density_lion_form_model_flatten(data, usage); - - return density_lion_form_entropy_codes[0]; - } -} diff --git a/contrib/density/src/algorithms/lion/forms/lion_form_model.h b/contrib/density/src/algorithms/lion/forms/lion_form_model.h deleted file mode 100644 index 90164d94c78..00000000000 --- a/contrib/density/src/algorithms/lion/forms/lion_form_model.h +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 9/03/15 12:04 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#ifndef DENSITY_LION_FORM_MODEL_H -#define DENSITY_LION_FORM_MODEL_H - -#include "../../../globals.h" -#include "../lion.h" - -#define DENSITY_LION_NUMBER_OF_FORMS 8 - -// Unary codes (reversed) except the last one -#define DENSITY_LION_FORM_MODEL_ENTROPY_CODES {\ - {DENSITY_BINARY_TO_UINT(1), 1},\ - {DENSITY_BINARY_TO_UINT(10), 2},\ - {DENSITY_BINARY_TO_UINT(100), 3},\ - {DENSITY_BINARY_TO_UINT(1000), 4},\ - {DENSITY_BINARY_TO_UINT(10000), 5},\ - {DENSITY_BINARY_TO_UINT(100000), 6},\ - {DENSITY_BINARY_TO_UINT(1000000), 7},\ - {DENSITY_BINARY_TO_UINT(0000000), 7},\ -} - -#pragma pack(push) -#pragma pack(4) -typedef struct { - void* previousForm; - DENSITY_LION_FORM form; - uint8_t rank; -} density_lion_form_node; - -typedef struct { - union { - uint8_t usages_as_uint8_t[DENSITY_LION_NUMBER_OF_FORMS]; - uint64_t usages_as_uint64_t; - } usages; - - void (*attachments[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const); - density_lion_form_node formsPool[DENSITY_LION_NUMBER_OF_FORMS]; - density_lion_form_node *formsIndex[DENSITY_LION_NUMBER_OF_FORMS]; - uint8_t nextAvailableForm; -} density_lion_form_data; -#pragma pack(pop) - -DENSITY_WINDOWS_EXPORT void density_lion_form_model_init(density_lion_form_data *const); - -DENSITY_WINDOWS_EXPORT void density_lion_form_model_attach(density_lion_form_data *const, void (*[DENSITY_LION_NUMBER_OF_FORMS])(const uint8_t **, uint8_t **, uint_fast16_t *, void *const, uint16_t *const, uint32_t *const)); - -DENSITY_WINDOWS_EXPORT void density_lion_form_model_update(density_lion_form_data *const DENSITY_RESTRICT_DECLARE, density_lion_form_node *const DENSITY_RESTRICT_DECLARE, const uint8_t, density_lion_form_node *const DENSITY_RESTRICT_DECLARE, const uint8_t); - -DENSITY_WINDOWS_EXPORT DENSITY_LION_FORM density_lion_form_model_increment_usage(density_lion_form_data *const, density_lion_form_node *const DENSITY_RESTRICT_DECLARE); - -DENSITY_WINDOWS_EXPORT density_lion_entropy_code density_lion_form_model_get_encoding(density_lion_form_data *const, const DENSITY_LION_FORM); - -#endif diff --git a/contrib/density/src/algorithms/lion/lion.h b/contrib/density/src/algorithms/lion/lion.h deleted file mode 100644 index 30b40f8d92b..00000000000 --- a/contrib/density/src/algorithms/lion/lion.h +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 5/02/15 20:57 - * - * -------------- - * Lion algorithm - * -------------- - * - * Author(s) - * Guillaume Voirin (https://github.com/gpnuma) - * - * Description - * Multiform compression algorithm - */ - -#ifndef DENSITY_LION_H -#define DENSITY_LION_H - -#include "../../globals.h" - -#define DENSITY_LION_HASH32_MULTIPLIER (uint32_t)0x9D6EF916lu -#define DENSITY_LION_CHUNK_HASH_BITS 16 - -#define DENSITY_LION_HASH_ALGORITHM(value32) (uint16_t)(value32 * DENSITY_LION_HASH32_MULTIPLIER >> (32 - DENSITY_LION_CHUNK_HASH_BITS)) - -typedef enum { - DENSITY_LION_FORM_PREDICTIONS_A = 0, - DENSITY_LION_FORM_PREDICTIONS_B, - DENSITY_LION_FORM_PREDICTIONS_C, - DENSITY_LION_FORM_DICTIONARY_A, - DENSITY_LION_FORM_DICTIONARY_B, - DENSITY_LION_FORM_DICTIONARY_C, - DENSITY_LION_FORM_DICTIONARY_D, - DENSITY_LION_FORM_PLAIN, -} DENSITY_LION_FORM; - -typedef enum { - DENSITY_LION_PREDICTIONS_SIGNATURE_FLAG_A = 0x0, - DENSITY_LION_PREDICTIONS_SIGNATURE_FLAG_B = 0x1, -} DENSITY_LION_PREDICTIONS_SIGNATURE_FLAG; - -#pragma pack(push) -#pragma pack(4) -typedef struct { - uint_fast8_t value; - uint_fast8_t bitLength; -} density_lion_entropy_code; -#pragma pack(pop) - -typedef uint64_t density_lion_signature; - -#define DENSITY_LION_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE (density_bitsizeof(density_lion_signature) * sizeof(uint32_t)) // Plain writes -#define DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE (sizeof(density_lion_signature) + DENSITY_LION_MAXIMUM_COMPRESSED_BODY_SIZE_PER_SIGNATURE) - -#define DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE (density_bitsizeof(density_lion_signature) * sizeof(uint32_t)) // Smallest form size times work unit size - -#define DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_SMALL 8 -#define DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG 64 -#define DENSITY_LION_PROCESS_UNIT_SIZE_SMALL (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_SMALL * sizeof(uint32_t)) -#define DENSITY_LION_PROCESS_UNIT_SIZE_BIG (DENSITY_LION_CHUNKS_PER_PROCESS_UNIT_BIG * sizeof(uint32_t)) - -#define DENSITY_LION_WORK_BLOCK_SIZE 256 -#define DENSITY_LION_COPY_PENALTY 2 - -#endif diff --git a/contrib/density/src/buffers/buffer.c b/contrib/density/src/buffers/buffer.c deleted file mode 100644 index d3e9943c599..00000000000 --- a/contrib/density/src/buffers/buffer.c +++ /dev/null @@ -1,220 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 3/02/15 19:53 - */ - -#include "buffer.h" - -DENSITY_WINDOWS_EXPORT uint_fast64_t density_compress_safe_size(const uint_fast64_t input_size) { - const uint_fast64_t slack = DENSITY_MAX_3(DENSITY_CHAMELEON_MAXIMUM_COMPRESSED_UNIT_SIZE, DENSITY_CHEETAH_MAXIMUM_COMPRESSED_UNIT_SIZE, DENSITY_LION_MAXIMUM_COMPRESSED_UNIT_SIZE); - - // Chameleon longest output - uint_fast64_t chameleon_longest_output_size = 0; - chameleon_longest_output_size += sizeof(density_header); - chameleon_longest_output_size += sizeof(density_chameleon_signature) * (1 + (input_size >> (5 + 3))); // Signature space (1 bit <=> 4 bytes) - chameleon_longest_output_size += sizeof(density_chameleon_signature); // Eventual supplementary signature for end marker - chameleon_longest_output_size += input_size; // Everything encoded as plain data - - // Cheetah longest output - uint_fast64_t cheetah_longest_output_size = 0; - cheetah_longest_output_size += sizeof(density_header); - cheetah_longest_output_size += sizeof(density_cheetah_signature) * (1 + (input_size >> (4 + 3))); // Signature space (2 bits <=> 4 bytes) - cheetah_longest_output_size += sizeof(density_cheetah_signature); // Eventual supplementary signature for end marker - cheetah_longest_output_size += input_size; // Everything encoded as plain data - - // Lion longest output - uint_fast64_t lion_longest_output_size = 0; - lion_longest_output_size += sizeof(density_header); - lion_longest_output_size += sizeof(density_lion_signature) * (1 + ((input_size * 7) >> (5 + 3))); // Signature space (7 bits <=> 4 bytes), although this size is technically impossible - lion_longest_output_size += sizeof(density_lion_signature); // Eventual supplementary signature for end marker - lion_longest_output_size += input_size; // Everything encoded as plain data - - return DENSITY_MAX_3(chameleon_longest_output_size, cheetah_longest_output_size, lion_longest_output_size) + slack; -} - -DENSITY_WINDOWS_EXPORT uint_fast64_t density_decompress_safe_size(const uint_fast64_t expected_decompressed_output_size) { - const uint_fast64_t slack = DENSITY_MAX_3(DENSITY_CHAMELEON_DECOMPRESSED_UNIT_SIZE, DENSITY_CHEETAH_DECOMPRESSED_UNIT_SIZE, DENSITY_LION_MAXIMUM_DECOMPRESSED_UNIT_SIZE); - - return expected_decompressed_output_size + slack; -} - -DENSITY_FORCE_INLINE DENSITY_STATE density_convert_algorithm_exit_status(const density_algorithm_exit_status status) { - switch (status) { - case DENSITY_ALGORITHMS_EXIT_STATUS_FINISHED: - return DENSITY_STATE_OK; - case DENSITY_ALGORITHMS_EXIT_STATUS_INPUT_STALL: - return DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL; - case DENSITY_ALGORITHMS_EXIT_STATUS_OUTPUT_STALL: - return DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL; - default: - return DENSITY_STATE_ERROR_DURING_PROCESSING; - } -} - -DENSITY_FORCE_INLINE density_processing_result density_make_result(const DENSITY_STATE state, const uint_fast64_t read, const uint_fast64_t written, density_context *const context) { - density_processing_result result; - result.state = state; - result.bytesRead = read; - result.bytesWritten = written; - result.context = context; - return result; -} - -DENSITY_FORCE_INLINE density_context* density_allocate_context(const DENSITY_ALGORITHM algorithm, const bool custom_dictionary, void *(*mem_alloc)(size_t)) { - density_context* context = mem_alloc(sizeof(density_context)); - context->algorithm = algorithm; - context->dictionary_size = density_get_dictionary_size(context->algorithm); - context->dictionary_type = custom_dictionary; - if(!context->dictionary_type) { - context->dictionary = mem_alloc(context->dictionary_size); - DENSITY_MEMSET(context->dictionary, 0, context->dictionary_size); - } - return context; -} - -DENSITY_WINDOWS_EXPORT void density_free_context(density_context *const context, void (*mem_free)(void *)) { - if(mem_free == NULL) - mem_free = free; - if(!context->dictionary_type) - mem_free(context->dictionary); - mem_free(context); -} - -DENSITY_WINDOWS_EXPORT density_processing_result density_compress_prepare_context(const DENSITY_ALGORITHM algorithm, const bool custom_dictionary, void *(*mem_alloc)(size_t)) { - if(mem_alloc == NULL) - mem_alloc = malloc; - - return density_make_result(DENSITY_STATE_OK, 0, 0, density_allocate_context(algorithm, custom_dictionary, mem_alloc)); -} - -DENSITY_WINDOWS_EXPORT density_processing_result density_compress_with_context(const uint8_t * input_buffer, const uint_fast64_t input_size, uint8_t * output_buffer, const uint_fast64_t output_size, density_context *const context) { - if (output_size < sizeof(density_header)) - return density_make_result(DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL, 0, 0, context); - if(context == NULL) - return density_make_result(DENSITY_STATE_ERROR_INVALID_CONTEXT, 0, 0, context); - - // Variables setup - const uint8_t *in = input_buffer; - uint8_t *out = output_buffer; - density_algorithm_state state; - density_algorithm_exit_status status = DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; - - // Header - density_header_write(&out, context->algorithm); - - // Compression - density_algorithms_prepare_state(&state, context->dictionary); - switch (context->algorithm) { - case DENSITY_ALGORITHM_CHAMELEON: - status = density_chameleon_encode(&state, &in, input_size, &out, output_size); - break; - case DENSITY_ALGORITHM_CHEETAH: - status = density_cheetah_encode(&state, &in, input_size, &out, output_size); - break; - case DENSITY_ALGORITHM_LION: - status = density_lion_encode(&state, &in, input_size, &out, output_size); - break; - } - - // Result - return density_make_result(density_convert_algorithm_exit_status(status), in - input_buffer, out - output_buffer, context); -} - -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_prepare_context(const uint8_t *input_buffer, const uint_fast64_t input_size, const bool custom_dictionary, void *(*mem_alloc)(size_t)) { - if (input_size < sizeof(density_header)) - return density_make_result(DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL, 0, 0, NULL); - - // Variables setup - const uint8_t* in = input_buffer; - if(mem_alloc == NULL) - mem_alloc = malloc; - - // Read header - density_header main_header; - density_header_read(&in, &main_header); - - // Setup context - density_context *const context = density_allocate_context(main_header.algorithm, custom_dictionary, mem_alloc); - return density_make_result(DENSITY_STATE_OK, in - input_buffer, 0, context); -} - -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_with_context(const uint8_t * input_buffer, const uint_fast64_t input_size, uint8_t * output_buffer, const uint_fast64_t output_size, density_context *const context) { - if(context == NULL) - return density_make_result(DENSITY_STATE_ERROR_INVALID_CONTEXT, 0, 0, context); - - // Variables setup - const uint8_t *in = input_buffer; - uint8_t *out = output_buffer; - density_algorithm_state state; - density_algorithm_exit_status status = DENSITY_ALGORITHMS_EXIT_STATUS_ERROR_DURING_PROCESSING; - - // Decompression - density_algorithms_prepare_state(&state, context->dictionary); - switch (context->algorithm) { - case DENSITY_ALGORITHM_CHAMELEON: - status = density_chameleon_decode(&state, &in, input_size, &out, output_size); - break; - case DENSITY_ALGORITHM_CHEETAH: - status = density_cheetah_decode(&state, &in, input_size, &out, output_size); - break; - case DENSITY_ALGORITHM_LION: - status = density_lion_decode(&state, &in, input_size, &out, output_size); - break; - } - - // Result - return density_make_result(density_convert_algorithm_exit_status(status), in - input_buffer, out - output_buffer, context); -} - -DENSITY_WINDOWS_EXPORT density_processing_result density_compress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, const DENSITY_ALGORITHM algorithm) { - density_processing_result result = density_compress_prepare_context(algorithm, false, malloc); - if(result.state) { - density_free_context(result.context, free); - return result; - } - - result = density_compress_with_context(input_buffer, input_size, output_buffer, output_size, result.context); - density_free_context(result.context, free); - return result; -} - -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size) { - density_processing_result result = density_decompress_prepare_context(input_buffer, input_size, false, malloc); - if(result.state) { - density_free_context(result.context, free); - return result; - } - - result = density_decompress_with_context(input_buffer + result.bytesRead, input_size - result.bytesRead, output_buffer, output_size, result.context); - density_free_context(result.context, free); - return result; -} diff --git a/contrib/density/src/buffers/buffer.h b/contrib/density/src/buffers/buffer.h deleted file mode 100644 index f34ab5458d2..00000000000 --- a/contrib/density/src/buffers/buffer.h +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Density - * - * Copyright (c) 2015, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 3/02/15 19:51 - */ - -#ifndef DENSITY_BUFFER_H -#define DENSITY_BUFFER_H - -#include "../globals.h" -#include "../density_api.h" -#include "../structure/header.h" -#include "../algorithms/chameleon/core/chameleon_encode.h" -#include "../algorithms/chameleon/core/chameleon_decode.h" -#include "../algorithms/cheetah/core/cheetah_encode.h" -#include "../algorithms/cheetah/core/cheetah_decode.h" -#include "../algorithms/lion/core/lion_encode.h" -#include "../algorithms/lion/core/lion_decode.h" - -DENSITY_WINDOWS_EXPORT uint_fast64_t density_compress_safe_size(const uint_fast64_t); -DENSITY_WINDOWS_EXPORT uint_fast64_t density_decompress_safe_size(const uint_fast64_t); -DENSITY_WINDOWS_EXPORT void density_free_context(density_context *const, void (*)(void *)); -DENSITY_WINDOWS_EXPORT density_processing_result density_compress_prepare_context(const DENSITY_ALGORITHM, const bool, void *(*)(size_t)); -DENSITY_WINDOWS_EXPORT density_processing_result density_compress_with_context(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t, density_context *const); -DENSITY_WINDOWS_EXPORT density_processing_result density_compress(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t, const DENSITY_ALGORITHM); -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_prepare_context(const uint8_t *, const uint_fast64_t, const bool, void *(*)(size_t)); -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_with_context(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t, density_context *const); -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress(const uint8_t *, const uint_fast64_t, uint8_t *, const uint_fast64_t); - -#endif diff --git a/contrib/density/src/density_api.h b/contrib/density/src/density_api.h deleted file mode 100644 index 385e24e609a..00000000000 --- a/contrib/density/src/density_api.h +++ /dev/null @@ -1,220 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 18/10/13 22:41 - */ - -#ifndef DENSITY_API_H -#define DENSITY_API_H - -#ifdef __cplusplus -extern "C" { -#endif - -#include -#include -#include - -#if defined(_WIN64) || defined(_WIN32) -#define DENSITY_WINDOWS_EXPORT __declspec(dllexport) -#else -#define DENSITY_WINDOWS_EXPORT -#endif - - -/*********************************************************************************************************************** - * * - * API data structures * - * * - ***********************************************************************************************************************/ - -typedef uint8_t density_byte; -typedef bool density_bool; - -typedef enum { - DENSITY_ALGORITHM_CHAMELEON = 1, - DENSITY_ALGORITHM_CHEETAH = 2, - DENSITY_ALGORITHM_LION = 3, -} DENSITY_ALGORITHM; - -typedef enum { - DENSITY_STATE_OK = 0, // Everything went alright - DENSITY_STATE_ERROR_INPUT_BUFFER_TOO_SMALL, // Input buffer size is too small - DENSITY_STATE_ERROR_OUTPUT_BUFFER_TOO_SMALL, // Output buffer size is too small - DENSITY_STATE_ERROR_DURING_PROCESSING, // Error during processing - DENSITY_STATE_ERROR_INVALID_CONTEXT, // Invalid context - DENSITY_STATE_ERROR_INVALID_ALGORITHM, // Invalid algorithm -} DENSITY_STATE; - -typedef struct { - DENSITY_ALGORITHM algorithm; - bool dictionary_type; - size_t dictionary_size; - void* dictionary; -} density_context; - -typedef struct { - DENSITY_STATE state; - uint_fast64_t bytesRead; - uint_fast64_t bytesWritten; - density_context* context; -} density_processing_result; - - - -/*********************************************************************************************************************** - * * - * Density version information * - * * - ***********************************************************************************************************************/ - -/* - * Returns the major density version - */ -DENSITY_WINDOWS_EXPORT uint8_t density_version_major(void); - -/* - * Returns the minor density version - */ -DENSITY_WINDOWS_EXPORT uint8_t density_version_minor(void); - -/* - * Returns the density revision - */ -DENSITY_WINDOWS_EXPORT uint8_t density_version_revision(void); - - - -/*********************************************************************************************************************** - * * - * Density API functions * - * * - ***********************************************************************************************************************/ - -/* - * Return the required size of an algorithm's dictionary - * - * @param algorithm the algorithm to use this dictionary for - */ -DENSITY_WINDOWS_EXPORT size_t density_get_dictionary_size(DENSITY_ALGORITHM algorithm); - -/* - * Return an output buffer byte size which guarantees enough space for encoding input_size bytes - * - * @param input_size the size of the input data which is about to be compressed - */ -DENSITY_WINDOWS_EXPORT uint_fast64_t density_compress_safe_size(const uint_fast64_t input_size); - -/* - * Return an output buffer byte size which, if expected_decompressed_output_size is correct, will enable density to decompress properly - * - * @param expected_decompressed_output_size the expected (original) size of the decompressed data - */ -DENSITY_WINDOWS_EXPORT uint_fast64_t density_decompress_safe_size(const uint_fast64_t expected_decompressed_output_size); - -/* - * Releases a context from memory. - * - * @param context the context to free - * @param mem_free the memory freeing function. If set to NULL, free() is used - */ -DENSITY_WINDOWS_EXPORT void density_free_context(density_context *const context, void (*mem_free)(void *)); - -/* - * Allocate a context in memory using the provided function and optional dictionary - * - * @param algorithm the required algorithm - * @param custom_dictionary use an eventual custom dictionary ? If set to true the context's dictionary will have to be allocated - * @param mem_alloc the memory allocation function. If set to NULL, malloc() is used - */ -DENSITY_WINDOWS_EXPORT density_processing_result density_compress_prepare_context(const DENSITY_ALGORITHM algorithm, const bool custom_dictionary, void *(*mem_alloc)(size_t)); - -/* - * Compress an input_buffer of input_size bytes and store the result in output_buffer, using the provided context. - * Important note * this function could be unsafe memory-wise if not used properly. - * - * @param input_buffer a buffer of bytes - * @param input_size the size in bytes of input_buffer - * @param output_buffer a buffer of bytes - * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE - * @param context a pointer to a context structure - */ -DENSITY_WINDOWS_EXPORT density_processing_result density_compress_with_context(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, density_context *const context); - -/* - * Compress an input_buffer of input_size bytes and store the result in output_buffer. - * - * @param input_buffer a buffer of bytes - * @param input_size the size in bytes of input_buffer - * @param output_buffer a buffer of bytes - * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE - * @param algorithm the algorithm to use - */ -DENSITY_WINDOWS_EXPORT density_processing_result density_compress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, const DENSITY_ALGORITHM algorithm); - -/* - * Reads the compressed data's header and creates an adequate decompression context. - * - * @param input_buffer a buffer of bytes - * @param input_size the size in bytes of input_buffer - * @param custom_dictionary use a custom dictionary ? If set to true the context's dictionary will have to be allocated - * @param mem_alloc the memory allocation function. If set to NULL, malloc() is used - */ -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_prepare_context(const uint8_t *input_buffer, const uint_fast64_t input_size, const bool custom_dictionary, void *(*mem_alloc)(size_t)); - -/* - * Decompress an input_buffer of input_size bytes and store the result in output_buffer, using the provided dictionary. - * Important notes * You must know in advance the algorithm used for compression to provide the proper dictionary. - * * This function could be unsafe memory-wise if not used properly. - * - * @param input_buffer a buffer of bytes - * @param input_size the size in bytes of input_buffer - * @param output_buffer a buffer of bytes - * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE - * @param dictionaries a pointer to a dictionary - */ -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress_with_context(const uint8_t * input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size, density_context *const context); - -/* - * Decompress an input_buffer of input_size bytes and store the result in output_buffer. - * - * @param input_buffer a buffer of bytes - * @param input_size the size in bytes of input_buffer - * @param output_buffer a buffer of bytes - * @param output_size the size of output_buffer, must be at least DENSITY_MINIMUM_OUTPUT_BUFFER_SIZE - */ -DENSITY_WINDOWS_EXPORT density_processing_result density_decompress(const uint8_t *input_buffer, const uint_fast64_t input_size, uint8_t *output_buffer, const uint_fast64_t output_size); - -#ifdef __cplusplus -} -#endif - -#endif diff --git a/contrib/density/src/globals.c b/contrib/density/src/globals.c deleted file mode 100644 index 4c58cd6f067..00000000000 --- a/contrib/density/src/globals.c +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 01/11/13 13:39 - */ - -#include "globals.h" - -DENSITY_WINDOWS_EXPORT uint8_t density_version_major() { - return DENSITY_MAJOR_VERSION; -} - -DENSITY_WINDOWS_EXPORT uint8_t density_version_minor() { - return DENSITY_MINOR_VERSION; -} - -DENSITY_WINDOWS_EXPORT uint8_t density_version_revision() { - return DENSITY_REVISION; -} diff --git a/contrib/density/src/globals.h b/contrib/density/src/globals.h deleted file mode 100644 index eb44a52a673..00000000000 --- a/contrib/density/src/globals.h +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 11/10/13 02:01 - */ - -#ifndef DENSITY_GLOBALS_H -#define DENSITY_GLOBALS_H - -#include -#include -#include -#include - -#include "density_api.h" - -#if defined(__clang__) || defined(__GNUC__) -#define DENSITY_FORCE_INLINE inline __attribute__((always_inline)) -#define DENSITY_RESTRICT restrict -#define DENSITY_RESTRICT_DECLARE -#define DENSITY_MEMCPY __builtin_memcpy -#define DENSITY_MEMMOVE __builtin_memmove -#define DENSITY_MEMSET __builtin_memset -#define DENSITY_LIKELY(x) __builtin_expect(!!(x), 1) -#define DENSITY_UNLIKELY(x) __builtin_expect(!!(x), 0) -#define DENSITY_PREFETCH(x) __builtin_prefetch(x) -#define DENSITY_CTZ(x) __builtin_ctz(x) - -#if defined(__BYTE_ORDER__) -#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ -#define DENSITY_LITTLE_ENDIAN -#elif __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ -#define DENSITY_BIG_ENDIAN -#else -#error Unsupported endianness -#endif -#else -#error Unkwnown endianness -#endif - -#elif defined(_MSC_VER) -#include -#include - -#define DENSITY_FORCE_INLINE __forceinline -#define DENSITY_RESTRICT __restrict -#define DENSITY_RESTRICT_DECLARE __restrict -#define DENSITY_MEMCPY memcpy -#define DENSITY_MEMMOVE memmove -#define DENSITY_MEMSET memset -#define DENSITY_LIKELY(x) (x) -#define DENSITY_UNLIKELY(x) (x) -#define DENSITY_PREFETCH(x) ((void)(x)) - -DENSITY_FORCE_INLINE uint_fast8_t density_msvc_ctz(uint64_t value) { - unsigned long trailing_zero = 0; - if (_BitScanForward(&trailing_zero, (unsigned long)value)) - return (uint_fast8_t)trailing_zero; - else - return 0; -} -#define DENSITY_CTZ(x) density_msvc_ctz(x) - -#define DENSITY_LITTLE_ENDIAN // Little endian by default on Windows - -#else -#error Unsupported compiler -#endif - -#ifdef DENSITY_LITTLE_ENDIAN -#define DENSITY_LITTLE_ENDIAN_64(b) ((uint64_t)b) -#define DENSITY_LITTLE_ENDIAN_32(b) ((uint32_t)b) -#define DENSITY_LITTLE_ENDIAN_16(b) ((uint16_t)b) -#elif defined(DENSITY_BIG_ENDIAN) -#if __GNUC__ * 100 + __GNUC_MINOR__ >= 403 -#define DENSITY_LITTLE_ENDIAN_64(b) __builtin_bswap64(b) -#define DENSITY_LITTLE_ENDIAN_32(b) __builtin_bswap32(b) -#define DENSITY_LITTLE_ENDIAN_16(b) __builtin_bswap16(b) -#else -#warning Using bulk byte swap routines. Expect performance issues. -#define DENSITY_LITTLE_ENDIAN_64(b) ((((b) & 0xFF00000000000000ull) >> 56) | (((b) & 0x00FF000000000000ull) >> 40) | (((b) & 0x0000FF0000000000ull) >> 24) | (((b) & 0x000000FF00000000ull) >> 8) | (((b) & 0x00000000FF000000ull) << 8) | (((b) & 0x0000000000FF0000ull) << 24ull) | (((b) & 0x000000000000FF00ull) << 40) | (((b) & 0x00000000000000FFull) << 56)) -#define DENSITY_LITTLE_ENDIAN_32(b) ((((b) & 0xFF000000) >> 24) | (((b) & 0x00FF0000) >> 8) | (((b) & 0x0000FF00) << 8) | (((b) & 0x000000FF) << 24)) -#define DENSITY_LITTLE_ENDIAN_16(b) ((((b) & 0xFF00) >> 8) | (((b) & 0x00FF) << 8)) -#endif -#else -#error Unsupported endianness -#endif - -#define DENSITY_MAX_2(a, b) (((a)>(b))?(a):(b)) -#define DENSITY_MAX_3(a, b, c) (DENSITY_MAX_2(DENSITY_MAX_2(a, b), c)) - -#define DENSITY_FORMAT(v) 0##v##llu - -#define DENSITY_ISOLATE(b, p) ((DENSITY_FORMAT(b) / p) & 0x1) - -#define DENSITY_BINARY_TO_UINT(b) ((DENSITY_ISOLATE(b, 1llu) ? 0x1 : 0)\ - + (DENSITY_ISOLATE(b, 8llu) ? 0x2 : 0)\ - + (DENSITY_ISOLATE(b, 64llu) ? 0x4 : 0)\ - + (DENSITY_ISOLATE(b, 512llu) ? 0x8 : 0)\ - + (DENSITY_ISOLATE(b, 4096llu) ? 0x10 : 0)\ - + (DENSITY_ISOLATE(b, 32768llu) ? 0x20 : 0)\ - + (DENSITY_ISOLATE(b, 262144llu) ? 0x40 : 0)\ - + (DENSITY_ISOLATE(b, 2097152llu) ? 0x80 : 0)\ - + (DENSITY_ISOLATE(b, 16777216llu) ? 0x100 : 0)\ - + (DENSITY_ISOLATE(b, 134217728llu) ? 0x200 : 0)\ - + (DENSITY_ISOLATE(b, 1073741824llu) ? 0x400 : 0)\ - + (DENSITY_ISOLATE(b, 8589934592llu) ? 0x800 : 0)\ - + (DENSITY_ISOLATE(b, 68719476736llu) ? 0x1000 : 0)\ - + (DENSITY_ISOLATE(b, 549755813888llu) ? 0x2000 : 0)\ - + (DENSITY_ISOLATE(b, 4398046511104llu) ? 0x4000 : 0)\ - + (DENSITY_ISOLATE(b, 35184372088832llu) ? 0x8000 : 0)\ - + (DENSITY_ISOLATE(b, 281474976710656llu) ? 0x10000 : 0)\ - + (DENSITY_ISOLATE(b, 2251799813685248llu) ? 0x20000 : 0)) - -#define DENSITY_UNROLL_2(op) op; op -#define DENSITY_UNROLL_4(op) DENSITY_UNROLL_2(op); DENSITY_UNROLL_2(op) -#define DENSITY_UNROLL_8(op) DENSITY_UNROLL_4(op); DENSITY_UNROLL_4(op) -#define DENSITY_UNROLL_16(op) DENSITY_UNROLL_8(op); DENSITY_UNROLL_8(op) -#define DENSITY_UNROLL_32(op) DENSITY_UNROLL_16(op); DENSITY_UNROLL_16(op) -#define DENSITY_UNROLL_64(op) DENSITY_UNROLL_32(op); DENSITY_UNROLL_32(op) - -#define DENSITY_CASE_GENERATOR_2(op_a, flag_a, op_b, flag_b, op_mid, shift)\ - case ((flag_b << shift) | flag_a):\ - op_a;\ - op_mid;\ - op_b;\ - break; - -#define DENSITY_CASE_GENERATOR_4(op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - case ((flag_d << (shift * 3)) | (flag_c << (shift * 2)) | (flag_b << shift) | flag_a):\ - op_a;\ - op_mid;\ - op_b;\ - op_mid;\ - op_c;\ - op_mid;\ - op_d;\ - break; - -#define DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_1, flag_1, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_a, flag_a, op_mid, shift);\ - DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_b, flag_b, op_mid, shift);\ - DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_c, flag_c, op_mid, shift);\ - DENSITY_CASE_GENERATOR_2(op_1, flag_1, op_d, flag_d, op_mid, shift); - -#define DENSITY_CASE_GENERATOR_4_2_COMBINED(op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_2_LAST_1_COMBINED(op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); - -#define DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_a, flag_a, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_b, flag_b, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_c, flag_c, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4(op_1, flag_1, op_2, flag_2, op_3, flag_3, op_d, flag_d, op_mid, shift); - -#define DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_2, flag_2, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_1_COMBINED(op_1, flag_1, op_2, flag_2, op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); - -#define DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_1, flag_1, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_2_COMBINED(op_1, flag_1, op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); - -#define DENSITY_CASE_GENERATOR_4_4_COMBINED(op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift)\ - DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_a, flag_a, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_b, flag_b, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_c, flag_c, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift);\ - DENSITY_CASE_GENERATOR_4_4_LAST_3_COMBINED(op_d, flag_d, op_a, flag_a, op_b, flag_b, op_c, flag_c, op_d, flag_d, op_mid, shift); - -#define DENSITY_DICTIONARY_PREFERRED_RESET_CYCLE_SHIFT 6 -#define DENSITY_DICTIONARY_PREFERRED_RESET_CYCLE (1 << DENSITY_DICTIONARY_PREFERRED_RESET_CYCLE_SHIFT) - - -#define density_bitsizeof(x) (8 * sizeof(x)) - -#define DENSITY_SPOOKYHASH_SEED_1 (0xabc) -#define DENSITY_SPOOKYHASH_SEED_2 (0xdef) - -DENSITY_WINDOWS_EXPORT uint8_t density_version_major(); - -DENSITY_WINDOWS_EXPORT uint8_t density_version_minor(); - -DENSITY_WINDOWS_EXPORT uint8_t density_version_revision(); - - -/********************************************************************************************************************** - * * - * Global compile-time switches * - * * - **********************************************************************************************************************/ - -#define DENSITY_MAJOR_VERSION 0 -#define DENSITY_MINOR_VERSION 14 -#define DENSITY_REVISION 2 - - - -#endif diff --git a/contrib/density/src/structure/header.c b/contrib/density/src/structure/header.c deleted file mode 100644 index 4306a5962cf..00000000000 --- a/contrib/density/src/structure/header.c +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 11/10/13 17:56 - */ - -#include "header.h" - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_header_read(const uint8_t **DENSITY_RESTRICT in, density_header *DENSITY_RESTRICT header) { - header->version[0] = *(*in); - header->version[1] = *(*in + 1); - header->version[2] = *(*in + 2); - header->algorithm = *(*in + 3); - - *in += sizeof(density_header); -} - -DENSITY_WINDOWS_EXPORT DENSITY_FORCE_INLINE void density_header_write(uint8_t **DENSITY_RESTRICT out, const DENSITY_ALGORITHM algorithm) { - *(*out) = DENSITY_MAJOR_VERSION; - *(*out + 1) = DENSITY_MINOR_VERSION; - *(*out + 2) = DENSITY_REVISION; - *(*out + 3) = algorithm; - *(*out + 4) = 0; - *(*out + 5) = 0; - *(*out + 6) = 0; - *(*out + 7) = 0; - - *out += sizeof(density_header); -} diff --git a/contrib/density/src/structure/header.h b/contrib/density/src/structure/header.h deleted file mode 100644 index d4065b5600c..00000000000 --- a/contrib/density/src/structure/header.h +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Density - * - * Copyright (c) 2013, Guillaume Voirin - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * 3. Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - * 11/10/13 02:06 - */ - -#ifndef DENSITY_FILE_HEADER_H -#define DENSITY_FILE_HEADER_H - -#include -#include - -#include "../globals.h" -#include "../density_api.h" - -#pragma pack(push) -#pragma pack(4) - -typedef struct { - density_byte version[3]; - density_byte algorithm; - density_byte reserved[4]; -} density_header; - -#pragma pack(pop) - -DENSITY_WINDOWS_EXPORT void density_header_read(const uint8_t ** DENSITY_RESTRICT_DECLARE, density_header * DENSITY_RESTRICT_DECLARE); -DENSITY_WINDOWS_EXPORT void density_header_write(uint8_t ** DENSITY_RESTRICT_DECLARE, const DENSITY_ALGORITHM); - -#endif diff --git a/contrib/lizard/.gitattributes b/contrib/lizard/.gitattributes deleted file mode 100644 index 6212bd405b4..00000000000 --- a/contrib/lizard/.gitattributes +++ /dev/null @@ -1,21 +0,0 @@ -# Set the default behavior -* text eol=lf - -# Explicitly declare source files -*.c text eol=lf -*.h text eol=lf - -# Denote files that should not be modified. -*.odt binary -*.png binary - -# Visual Studio -*.sln text eol=crlf -*.vcxproj* text eol=crlf -*.vcproj* text eol=crlf -*.suo binary -*.rc text eol=crlf - -# Windows -*.bat text eol=crlf -*.cmd text eol=crlf diff --git a/contrib/lizard/.gitignore b/contrib/lizard/.gitignore deleted file mode 100644 index dc30b0e5504..00000000000 --- a/contrib/lizard/.gitignore +++ /dev/null @@ -1,37 +0,0 @@ -# Object files -*.o -*.ko - -# Libraries -*.lib -*.a - -# Shared objects (inc. Windows DLLs) -*.dll -*.so -*.so.* -*.dylib - -# Executables -*.exe -*.out -*.app - -# IDE / editors files -*.suo -*.user -.clang_complete - -# Directories -_lizardbench/ -_visual/ -_codelite/ -_backup/ -_codelite_lz4/ - -# Archives -*.zip -*.liz - -*.txt -*.bat \ No newline at end of file diff --git a/contrib/lizard/.travis.yml b/contrib/lizard/.travis.yml deleted file mode 100644 index 29f75591a31..00000000000 --- a/contrib/lizard/.travis.yml +++ /dev/null @@ -1,268 +0,0 @@ -language: c -matrix: - fast_finish: true - include: - - name: macOS - os: osx - env: Ubu=OS_X_Mavericks Cmd='make -C tests test-lizard CC=clang MOREFLAGS="-Werror -Wconversion -Wno-sign-conversion"' COMPILER=clang - - - # Ubuntu 18.04 - - name: Ubuntu 18.04, gcc-9, 64-bit and 32-bit tests - dist: bionic - script: - - CC=gcc-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-9; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - before_install: - - sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test - - sudo apt-get -q update - - sudo apt-get --no-install-suggests --no-install-recommends -yq install gcc-multilib gcc-9 gcc-9-multilib - - - name: Ubuntu 18.04, clang-9, 64-bit tests - dist: bionic - script: - - CC=clang-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - sourceline: 'deb https://apt.llvm.org/bionic/ llvm-toolchain-bionic-9 main' - key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' - packages: - - clang-9 - - - - # Ubuntu 16.04 gcc - - name: Ubuntu 16.04, gcc-9, 64-bit and 32-bit tests - dist: xenial - script: - - CC=gcc-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-9; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-multilib - - gcc-9 - - gcc-9-multilib - - - name: Ubuntu 16.04, gcc-8, 64-bit and 32-bit tests - dist: xenial - script: - - CC=gcc-8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-8; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-multilib - - gcc-8 - - gcc-8-multilib - - - name: Ubuntu 16.04, gcc-7, 64-bit and 32-bit tests - dist: xenial - script: - - CC=gcc-7; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-7; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-multilib - - gcc-7 - - gcc-7-multilib - - - name: Ubuntu 16.04, gcc-6, 64-bit and 32-bit tests - dist: xenial - script: - - CC=gcc-6; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-6; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-multilib - - gcc-6 - - gcc-6-multilib - - - name: Ubuntu 16.04, gcc-5, 64-bit and 32-bit tests - dist: xenial - script: - - CC=gcc-5; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-5; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - packages: - - gcc-multilib - - gcc-5 - - gcc-5-multilib - - - name: Ubuntu 16.04, gcc-4.8, 64-bit and 32-bit tests - dist: xenial - script: - - CC=gcc-4.8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - - CC=gcc-4.8; $CC -v; make -C tests test-lizardc32 test-fullbench32 test-frametest32 test-fuzzer32 MOREFLAGS=-Werror - addons: - apt: - packages: - - gcc-multilib - - gcc-4.8 - - gcc-4.8-multilib - - - - # Ubuntu 16.04 clang - - name: Ubuntu 16.04, clang-9, 64-bit tests - dist: xenial - script: - - CC=clang-9; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - sourceline: 'deb https://apt.llvm.org/xenial/ llvm-toolchain-xenial-9 main' - key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key' - packages: - - clang-9 - - - name: Ubuntu 16.04, clang-8, 64-bit tests - dist: xenial - script: - - CC=clang-8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-xenial-8 - packages: - - clang-8 - - - name: Ubuntu 16.04, clang-7, 64-bit tests - dist: xenial - script: - - CC=clang-7; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-xenial-7 - packages: - - clang-7 - - - name: Ubuntu 16.04, clang-6.0, 64-bit tests - dist: xenial - script: - - CC=clang-6.0; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-xenial-6.0 - packages: - - clang-6.0 - - - name: Ubuntu 16.04, clang-5.0, 64-bit tests - dist: xenial - script: - - CC=clang-5.0; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-xenial-5.0 - packages: - - clang-5.0 - - - name: Ubuntu 16.04, clang-4.0, 64-bit tests - dist: xenial - script: - - CC=clang-4.0; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-xenial-4.0 - packages: - - clang-4.0 - - - name: Ubuntu 14.04, clang-3.8, 64-bit tests - dist: trusty - script: - - CC=clang-3.8; $CC -v; make -C tests test-lizard test-fullbench test-frametest test-fuzzer MOREFLAGS=-Werror - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - llvm-toolchain-trusty-3.8 - packages: - - clang-3.8 - - - - - name: Ubuntu 16.04, misc tests - env: Cmd="make c_standards && make gpptest-native && make clean && make examples && make clean && make cmake && make clean && make travis-install && make clean && make clangtest-native" COMPILER=cc - dist: xenial - - - name: Ubuntu 14.04, arm and aarch64 - env: Cmd='make platformTest CC=arm-linux-gnueabi-gcc QEMU_SYS=qemu-arm-static && make platformTest CC=aarch64-linux-gnu-gcc QEMU_SYS=qemu-aarch64-static' COMPILER=arm-linux-gnueabi-gcc - dist: trusty - sudo: required - addons: - apt: - packages: - - qemu-system-arm - - qemu-user-static - - gcc-arm-linux-gnueabi - - libc6-dev-armel-cross - - gcc-aarch64-linux-gnu - - libc6-dev-arm64-cross - - - name: Ubuntu 14.04, powerpc and ppc64 - env: Cmd='make platformTest CC=powerpc-linux-gnu-gcc QEMU_SYS=qemu-ppc-static && make platformTest CC=powerpc-linux-gnu-gcc QEMU_SYS=qemu-ppc64-static MOREFLAGS=-m64' COMPILER=powerpc-linux-gnu-gcc - dist: trusty - sudo: required - addons: - apt: - packages: - - qemu-system-ppc - - qemu-user-static - - gcc-powerpc-linux-gnu - - - name: Ubuntu 14.04, valgrind - env: Cmd='make -C tests test-mem MOREFLAGS=-DLIZARD_RESET_MEM' COMPILER=cc - dist: trusty - sudo: required - addons: - apt: - packages: - - valgrind - - - #- env: Ubu=14.04 Cmd='make sanitize' COMPILER=clang - # dist: trusty - # sudo: required - # addons: - # apt: - # packages: - # - valgrind - - #- env: Ubu=14.04 Cmd='make staticAnalyze' COMPILER=clang - # dist: trusty - # sudo: required - # addons: - # apt: - # packages: - # - clang - - -script: - - echo Cmd=$Cmd - - $COMPILER -v - - sh -c "$Cmd" diff --git a/contrib/lizard/LICENSE b/contrib/lizard/LICENSE deleted file mode 100644 index df1edbca85e..00000000000 --- a/contrib/lizard/LICENSE +++ /dev/null @@ -1,15 +0,0 @@ -This repository uses 2 different licenses : -- all files in the `lib` directory use a BSD 2-Clause license -- all other files use a GPLv2 license, unless explicitly stated otherwise - -Relevant license is reminded at the top of each source file, -and with the presence of COPYING or LICENSE file. - -This model emphasizes the fact that -only files in the `lib` directory are designed to be included into 3rd party projects. - -Other files, such as those from `programs` or `examples` directory, -are not intended to be compiled outside of their context. -They can serve as source of inspiration, -but they should not be copy/pasted into 3rd party projects, -as this scenario is not supported. diff --git a/contrib/lizard/NEWS b/contrib/lizard/NEWS deleted file mode 100644 index 05d424656e9..00000000000 --- a/contrib/lizard/NEWS +++ /dev/null @@ -1,41 +0,0 @@ -Lizard v1.0 -- LZ5 v2.0 was renamed to Lizard v1.0 -- improved compression speed with many small files - -LZ5 v2.0 -- this version is optimized for decompression speed (instead of ratio for previous versions) -- LZ5 v2.0 contains 4 compression methods: - fastLZ4 : compression levels -10...-19 are designed to give better decompression speed than LZ4 i.e. over 2000 MB/s - LIZv1 : compression levels -20...-29 are designed to give better ratio than LZ4 keeping 75% decompression speed - fastLZ4 + Huffman : compression levels -30...-39 add Huffman coding to fastLZ4 - LIZv1 + Huffman : compression levels -40...-49 give the best ratio (comparable to zlib and low levels of zstd/brotli) at decompression speed of 1000 MB/s - -LZ5 v1.5 -- introduced compatibility with Visual C++ 2010 and newer -- attached Visual Studio 2010 project -- thoroughly tested with 21 Travis CI and 7 AppVeyor CI tests -- fixed bug with reusing a context in lizard_frame.c (LizardF_compressBegin and Lizard_compress_HC_continue) -- fixed rare bug in match finder (concerns levels 4 - 15) - -LZ5 v1.4.1 -- fixed bug with a backward match extension (level 11 and 12) - -LZ5 v1.4 -- improved: levels from 13 to 15 (maximum compression ratio) -- added a new parser: LizardHC_optimal_price_bt -- updated documentation: lizard_Block_format.md and lizard_Frame_format.md -- changed lizard.exe: the "-B" option with block size [1-7] = 64KB, 256KB, 1MB, 4MB, 16MB, 64MB, 256MB (default : 4 = 4MB) - -LZ5 v1.3.3 -- added: new levels from 11 to 18 (maximum compression ratio) -- added: a new parser: LizardHC_optimal_price -- fixed: buffer-overflow during decompression (thanks to m^2) - -LZ5 r132 -- improved compression ratio -- added: new parsers: LizardHC_fast, LizardHC_price_fast, LizardHC_lowest_price -- added: a special 1-byte codeword for the last occured offset -- added: support for 3-byte long matches (MINMATCH = 3) - -LZ5 r131 -The first release based on LZ4 r132 dev diff --git a/contrib/lizard/README.md b/contrib/lizard/README.md deleted file mode 100644 index 50dd737fb8d..00000000000 --- a/contrib/lizard/README.md +++ /dev/null @@ -1,95 +0,0 @@ -Lizard - efficient compression with very fast decompression --------------------------------------------------------- - -Lizard (formerly LZ5) is a lossless compression algorithm which contains 4 compression methods: -- fastLZ4 : compression levels -10...-19 are designed to give better decompression speed than [LZ4] i.e. over 2000 MB/s -- LIZv1 : compression levels -20...-29 are designed to give better ratio than [LZ4] keeping 75% decompression speed -- fastLZ4 + Huffman : compression levels -30...-39 add Huffman coding to fastLZ4 -- LIZv1 + Huffman : compression levels -40...-49 give the best ratio (comparable to [zlib] and low levels of [zstd]/[brotli]) at decompression speed of 1000 MB/s - -Lizard library is based on frequently used [LZ4] library by Yann Collet but the Lizard compression format is not compatible with LZ4. -Lizard library is provided as open-source software using BSD 2-Clause license. -The high compression/decompression speed is achieved without any SSE and AVX extensions. - - -|Branch |Status | -|------------|---------| -|lz5_v1.5 | [![Build Status][travis15Badge]][travisLink] [![Build status][Appveyor15Badge]][AppveyorLink] | -|lizard | [![Build Status][travis20Badge]][travisLink] [![Build status][Appveyor20Badge]][AppveyorLink] | - -[travis15Badge]: https://travis-ci.org/inikep/lizard.svg?branch=lz5_v1.5 "Continuous Integration test suite" -[travis20Badge]: https://travis-ci.org/inikep/lizard.svg?branch=lizard "Continuous Integration test suite" -[travisLink]: https://travis-ci.org/inikep/lizard -[Appveyor15Badge]: https://ci.appveyor.com/api/projects/status/cqw7emcuqge369p0/branch/lz5_v1.5?svg=true "Visual test suite" -[Appveyor20Badge]: https://ci.appveyor.com/api/projects/status/cqw7emcuqge369p0/branch/lizard?svg=true "Visual test suite" -[AppveyorLink]: https://ci.appveyor.com/project/inikep/lizard -[LZ4]: https://github.com/lz4/lz4 -[zlib]: https://github.com/madler/zlib -[zstd]: https://github.com/facebook/zstd -[brotli]: https://github.com/google/brotli - - -Benchmarks -------------------------- - -The following results are obtained with [lzbench](https://github.com/inikep/lzbench) and `-t16,16` -using 1 core of Intel Core i5-4300U, Windows 10 64-bit (MinGW-w64 compilation under gcc 6.2.0) -with [silesia.tar] which contains tarred files from [Silesia compression corpus](http://sun.aei.polsl.pl/~sdeor/index.php?page=silesia). - -| Compressor name | Compression| Decompress.| Compr. size | Ratio | -| --------------- | -----------| -----------| ----------- | ----- | -| memcpy | 7332 MB/s | 8719 MB/s | 211947520 |100.00 | -| lz4 1.7.3 | 440 MB/s | 2318 MB/s | 100880800 | 47.60 | -| lz4hc 1.7.3 -1 | 98 MB/s | 2121 MB/s | 87591763 | 41.33 | -| lz4hc 1.7.3 -4 | 55 MB/s | 2259 MB/s | 79807909 | 37.65 | -| lz4hc 1.7.3 -9 | 22 MB/s | 2315 MB/s | 77892285 | 36.75 | -| lz4hc 1.7.3 -12 | 17 MB/s | 2323 MB/s | 77849762 | 36.73 | -| lz4hc 1.7.3 -16 | 10 MB/s | 2323 MB/s | 77841782 | 36.73 | -| lizard 1.0 -10 | 346 MB/s | 2610 MB/s | 103402971 | 48.79 | -| lizard 1.0 -12 | 103 MB/s | 2458 MB/s | 86232422 | 40.69 | -| lizard 1.0 -15 | 50 MB/s | 2552 MB/s | 81187330 | 38.31 | -| lizard 1.0 -19 | 3.04 MB/s | 2497 MB/s | 77416400 | 36.53 | -| lizard 1.0 -21 | 157 MB/s | 1795 MB/s | 89239174 | 42.10 | -| lizard 1.0 -23 | 30 MB/s | 1778 MB/s | 81097176 | 38.26 | -| lizard 1.0 -26 | 6.63 MB/s | 1734 MB/s | 74503695 | 35.15 | -| lizard 1.0 -29 | 1.37 MB/s | 1634 MB/s | 68694227 | 32.41 | -| lizard 1.0 -30 | 246 MB/s | 909 MB/s | 85727429 | 40.45 | -| lizard 1.0 -32 | 94 MB/s | 1244 MB/s | 76929454 | 36.30 | -| lizard 1.0 -35 | 47 MB/s | 1435 MB/s | 73850400 | 34.84 | -| lizard 1.0 -39 | 2.94 MB/s | 1502 MB/s | 69807522 | 32.94 | -| lizard 1.0 -41 | 126 MB/s | 961 MB/s | 76100661 | 35.91 | -| lizard 1.0 -43 | 28 MB/s | 1101 MB/s | 70955653 | 33.48 | -| lizard 1.0 -46 | 6.25 MB/s | 1073 MB/s | 65413061 | 30.86 | -| lizard 1.0 -49 | 1.27 MB/s | 1064 MB/s | 60679215 | 28.63 | -| zlib 1.2.8 -1 | 66 MB/s | 244 MB/s | 77259029 | 36.45 | -| zlib 1.2.8 -6 | 20 MB/s | 263 MB/s | 68228431 | 32.19 | -| zlib 1.2.8 -9 | 8.37 MB/s | 266 MB/s | 67644548 | 31.92 | -| zstd 1.1.1 -1 | 235 MB/s | 645 MB/s | 73659468 | 34.75 | -| zstd 1.1.1 -2 | 181 MB/s | 600 MB/s | 70168955 | 33.11 | -| zstd 1.1.1 -5 | 88 MB/s | 565 MB/s | 65002208 | 30.67 | -| zstd 1.1.1 -8 | 31 MB/s | 619 MB/s | 61026497 | 28.79 | -| zstd 1.1.1 -11 | 16 MB/s | 613 MB/s | 59523167 | 28.08 | -| zstd 1.1.1 -15 | 4.97 MB/s | 639 MB/s | 58007773 | 27.37 | -| zstd 1.1.1 -18 | 2.87 MB/s | 583 MB/s | 55294241 | 26.09 | -| zstd 1.1.1 -22 | 1.44 MB/s | 505 MB/s | 52731930 | 24.88 | -| brotli 0.5.2 -0 | 217 MB/s | 244 MB/s | 78226979 | 36.91 | -| brotli 0.5.2 -2 | 96 MB/s | 283 MB/s | 68066621 | 32.11 | -| brotli 0.5.2 -5 | 24 MB/s | 312 MB/s | 60801716 | 28.69 | -| brotli 0.5.2 -8 | 5.56 MB/s | 324 MB/s | 57382470 | 27.07 | -| brotli 0.5.2 -11 | 0.39 MB/s | 266 MB/s | 51138054 | 24.13 | - -[silesia.tar]: https://drive.google.com/file/d/0BwX7dtyRLxThenZpYU9zLTZhR1k/view?usp=sharing - - -Documentation -------------------------- - -The raw Lizard block compression format is detailed within [lizard_Block_format]. - -To compress an arbitrarily long file or data stream, multiple blocks are required. -Organizing these blocks and providing a common header format to handle their content -is the purpose of the Frame format, defined into [lizard_Frame_format]. -Interoperable versions of Lizard must respect this frame format. - -[lizard_Block_format]: doc/lizard_Block_format.md -[lizard_Frame_format]: doc/lizard_Frame_format.md diff --git a/contrib/lizard/appveyor.yml b/contrib/lizard/appveyor.yml deleted file mode 100644 index d5863f7c06b..00000000000 --- a/contrib/lizard/appveyor.yml +++ /dev/null @@ -1,143 +0,0 @@ -version: 1.0.{build} -environment: - matrix: - - COMPILER: "visual" - CONFIGURATION: "Debug" - PLATFORM: "x64" - - COMPILER: "visual" - CONFIGURATION: "Debug" - PLATFORM: "Win32" - - COMPILER: "visual" - CONFIGURATION: "Release" - PLATFORM: "x64" - - COMPILER: "visual" - CONFIGURATION: "Release" - PLATFORM: "Win32" - - COMPILER: "gcc" - PLATFORM: "mingw64" - - COMPILER: "gcc" - PLATFORM: "mingw32" - - COMPILER: "gcc" - PLATFORM: "clang" - -install: - - ECHO Installing %COMPILER% %PLATFORM% %CONFIGURATION% - - MKDIR bin - - if [%COMPILER%]==[gcc] SET PATH_ORIGINAL=%PATH% - - if [%COMPILER%]==[gcc] ( - SET "PATH_MINGW32=c:\MinGW\bin;c:\MinGW\usr\bin" && - SET "PATH_MINGW64=c:\msys64\mingw64\bin;c:\msys64\usr\bin" && - COPY C:\MinGW\bin\mingw32-make.exe C:\MinGW\bin\make.exe && - COPY C:\MinGW\bin\gcc.exe C:\MinGW\bin\cc.exe - ) else ( - IF [%PLATFORM%]==[x64] (SET ADDITIONALPARAM=/p:LibraryPath="C:\Program Files\Microsoft SDKs\Windows\v7.1\lib\x64;c:\Program Files (x86)\Microsoft Visual Studio 10.0\VC\lib\amd64;C:\Program Files (x86)\Microsoft Visual Studio 10.0\;C:\Program Files (x86)\Microsoft Visual Studio 10.0\lib\amd64;") - ) - -build_script: - - if [%PLATFORM%]==[mingw32] SET PATH=%PATH_MINGW32%;%PATH_ORIGINAL% - - if [%PLATFORM%]==[mingw64] SET PATH=%PATH_MINGW64%;%PATH_ORIGINAL% - - if [%PLATFORM%]==[clang] SET PATH=%PATH_MINGW64%;%PATH_ORIGINAL% - - ECHO *** && - ECHO Building %COMPILER% %PLATFORM% %CONFIGURATION% && - ECHO *** - - if [%PLATFORM%]==[clang] (clang -v) - - if [%COMPILER%]==[gcc] (gcc -v) - - if [%COMPILER%]==[gcc] ( - echo ----- && - make -v && - echo ----- && - if not [%PLATFORM%]==[clang] ( - make -C programs lizard && make -C tests fullbench && make -C lib lib - ) ELSE ( - make -C programs lizard CC=clang MOREFLAGS="--target=x86_64-w64-mingw32 -Werror -Wconversion -Wno-sign-conversion" && - make -C tests fullbench CC=clang MOREFLAGS="--target=x86_64-w64-mingw32 -Werror -Wconversion -Wno-sign-conversion" && - make -C lib lib CC=clang MOREFLAGS="--target=x86_64-w64-mingw32 -Werror -Wconversion -Wno-sign-conversion" - ) - ) - - if [%COMPILER%]==[gccX] if not [%PLATFORM%]==[clang] ( - MKDIR bin\dll bin\static bin\example bin\include && - COPY tests\fullbench.c bin\example\ && - COPY lib\xxhash.c bin\example\ && - COPY lib\xxhash.h bin\example\ && - COPY lib\lizard.h bin\include\ && - COPY lib\lizardhc.h bin\include\ && - COPY lib\lizard_frame.h bin\include\ && - COPY lib\liblizard.a bin\static\liblizard_static.lib && - COPY lib\dll\liblizard.* bin\dll\ && - COPY lib\dll\example\Makefile bin\example\ && - COPY lib\dll\example\fullbench-dll.* bin\example\ && - COPY lib\dll\example\README.md bin\ && - COPY programs\lizard.exe bin\lizard.exe - ) - - if [%COMPILER%]==[gccX] if [%PLATFORM%]==[mingw64] ( - 7z.exe a bin\lizard_x64.zip NEWS .\bin\lizard.exe .\bin\README.md .\bin\example .\bin\dll .\bin\static .\bin\include && - appveyor PushArtifact bin\lizard_x64.zip - ) - - if [%COMPILER%]==[gccX] if [%PLATFORM%]==[mingw32] ( - 7z.exe a bin\lizard_x86.zip NEWS .\bin\lizard.exe .\bin\README.md .\bin\example .\bin\dll .\bin\static .\bin\include && - appveyor PushArtifact bin\lizard_x86.zip - ) - - if [%COMPILER%]==[gcc] (COPY tests\fullbench.exe programs\) - - if [%COMPILER%]==[visual] ( - ECHO *** && - ECHO *** Building Visual Studio 2010 %PLATFORM%\%CONFIGURATION% && - ECHO *** && - msbuild "visual\VS2010\lizard.sln" %ADDITIONALPARAM% /m /verbosity:minimal /property:PlatformToolset=v100 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && - ECHO *** && - ECHO *** Building Visual Studio 2012 %PLATFORM%\%CONFIGURATION% && - ECHO *** && - msbuild "visual\VS2010\lizard.sln" /m /verbosity:minimal /property:PlatformToolset=v110 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && - ECHO *** && - ECHO *** Building Visual Studio 2013 %PLATFORM%\%CONFIGURATION% && - ECHO *** && - msbuild "visual\VS2010\lizard.sln" /m /verbosity:minimal /property:PlatformToolset=v120 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && - ECHO *** && - ECHO *** Building Visual Studio 2015 %PLATFORM%\%CONFIGURATION% && - ECHO *** && - msbuild "visual\VS2010\lizard.sln" /m /verbosity:minimal /property:PlatformToolset=v140 /t:Clean,Build /p:Platform=%PLATFORM% /p:Configuration=%CONFIGURATION% /logger:"C:\Program Files\AppVeyor\BuildAgent\Appveyor.MSBuildLogger.dll" && - COPY visual\VS2010\bin\%PLATFORM%_%CONFIGURATION%\*.exe programs\ - ) - -test_script: - - ECHO *** && - ECHO Testing %COMPILER% %PLATFORM% %CONFIGURATION% && - ECHO *** - - if not [%COMPILER%]==[unknown] ( - CD programs && - lizard -h && - lizard -i1b lizard.exe && - lizard -i1b5 lizard.exe && - lizard -i1b10 lizard.exe && - lizard -i1b15 lizard.exe && - echo ------- lizard tested ------- && - fullbench.exe -i1 fullbench.exe - ) - -artifacts: - - path: bin\lizard_x64.zip - - path: bin\lizard_x86.zip - -deploy: -- provider: GitHub - artifact: bin\lizard_x64.zip - auth_token: - secure: LgJo8emYc3sFnlNWkGl4/VYK3nk/8+RagcsqDlAi3xeqNGNutnKjcftjg84uJoT4 - force_update: true - prerelease: true - on: - COMPILER: gcc - PLATFORM: "mingw64" - appveyor_repo_tag: true - branch: autobuild - -- provider: GitHub - artifact: bin\lizard_x86.zip - auth_token: - secure: LgJo8emYc3sFnlNWkGl4/VYK3nk/8+RagcsqDlAi3xeqNGNutnKjcftjg84uJoT4 - force_update: true - prerelease: true - on: - COMPILER: gcc - PLATFORM: "mingw32" - appveyor_repo_tag: true - branch: autobuild diff --git a/contrib/lizard/contrib/djgpp/LICENSE b/contrib/lizard/contrib/djgpp/LICENSE deleted file mode 100644 index fee0d3b3e23..00000000000 --- a/contrib/lizard/contrib/djgpp/LICENSE +++ /dev/null @@ -1,24 +0,0 @@ -Copyright (c) 2014, lpsantil -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE -FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER -CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, -OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - diff --git a/contrib/lizard/contrib/djgpp/README.MD b/contrib/lizard/contrib/djgpp/README.MD deleted file mode 100644 index 8dfc3a838c4..00000000000 --- a/contrib/lizard/contrib/djgpp/README.MD +++ /dev/null @@ -1,21 +0,0 @@ -# lizard for DOS/djgpp -This file details on how to compile lizard.exe, and liblizard.a for use on DOS/djgpp using -Andrew Wu's build-djgpp cross compilers ([GH][0], [Binaries][1]) on OSX, Linux. - -## Setup -* Download a djgpp tarball [binaries][1] for your platform. -* Extract and install it (`tar jxvf djgpp-linux64-gcc492.tar.bz2`). Note the path. We'll assume `/home/user/djgpp`. -* Add the `bin` folder to your `PATH`. In bash, do `export PATH=/home/user/djgpp/bin:$PATH`. -* The `Makefile` in `contrib/djgpp/` sets up `CC`, `AR`, `LD` for you. So, `CC=i586-pc-msdosdjgpp-gcc`, `AR=i586-pc-msdosdjgpp-ar`, `LD=i586-pc-msdosdjgpp-gcc`. - -## Building Lizard for DOS -In the base dir of lizard and with `contrib/djgpp/Makefile`, try: -Try: -* `make -f contrib/djgpp/Makefile` -* `make -f contrib/djgpp/Makefile liblizard.a` -* `make -f contrib/djgpp/Makefile lizard.exe` -* `make -f contrib/djgpp/Makefile DESTDIR=/home/user/dos install`, however it doesn't make much sense on a \*nix. -* You can also do `make -f contrib/djgpp/Makefile uninstall` - -[0]: https://github.com/andrewwutw/build-djgpp -[1]: https://github.com/andrewwutw/build-djgpp/releases diff --git a/contrib/lizard/doc/lizard_Block_format.md b/contrib/lizard/doc/lizard_Block_format.md deleted file mode 100644 index 45307a1fce5..00000000000 --- a/contrib/lizard/doc/lizard_Block_format.md +++ /dev/null @@ -1,181 +0,0 @@ -Lizard v1.x Block Format Description -============================ -Last revised: 2016-10-08 -Authors : Yann Collet, Przemyslaw Skibinski - - -This specification is intended for developers -willing to produce Lizard-compatible compressed data blocks -using any programming language. - -Lizard is an LZ77-type compressor with a fixed, byte-oriented encoding. -There is no framing layer as it is assumed to be handled by other parts of the system (see [Lizard Frame format]). -This design is assumed to favor simplicity and speed. -It helps later on for optimizations, compactness, and features. - -This document describes only the block format, -not how the compressor nor decompressor actually work. -The correctness of the decompressor should not depend -on implementation details of the compressor, and vice versa. - -[Lizard Frame format]: lizard_Frame_format.md - - -Division into blocks --------------------- - -The input data is divided into blocks of maximum size LIZARD_BLOCK_SIZE (which is 128 KB). The subsequent blocks use the same sliding window and are dependent on previous blocks. -Our impementation of Lizard compressor divides input data into blocks of size of LIZARD_BLOCK_SIZE except the last one which usually will be smaller. -The output data is a single byte 'Compression_Level' and one or more blocks in the format described below. - - -Block header format ------------------------ - -The block header is a single byte `Header_Byte` that is combination of following flags: - -| Name | Value | -| --------------------- | --- | -| LIZARD_FLAG_LITERALS | 1 | -| LIZARD_FLAG_FLAGS | 2 | -| LIZARD_FLAG_OFF16LEN | 4 | -| LIZARD_FLAG_OFF24LEN | 8 | -| LIZARD_FLAG_LEN | 16 | -| LIZARD_FLAG_UNCOMPRESSED | 128 | - -When `Header_Byte & LIZARD_FLAG_UNCOMPRESSED` is true then the block is followed by 3-byte `Uncompressed_length` and uncompressed data of given size. - - -Compressed block content ------------------------- - -When `Header_Byte & LIZARD_FLAG_UNCOMPRESSED` is false then compressed block contains of 5 streams: -- `Lengths_Stream` (compressed with Huffman if LIZARD_FLAG_LEN is set) -- `16-bit_Offsets_Stream` (compressed with Huffman if LIZARD_FLAG_OFF16LEN is set) -- `24-bit_Offsets_Stream` (compressed with Huffman if LIZARD_FLAG_OFF24LEN is set) -- `Tokens_Stream` (compressed with Huffman if LIZARD_FLAG_FLAGS is set) -- `Literals_Stream` (compressed with Huffman if LIZARD_FLAG_LITERALS is set) - - -Stream format -------------- -The single stream is either: -- if LIZARD_FLAG_XXX is not set: 3 byte `Stream_Length` followed by a given number bytes -- if LIZARD_FLAG_XXX is set: 3 byte `Original_Stream_Length`, 3 byte `Compressed_Stream_Length`, followed by a given number of Huffman compressed bytes - - -Lizard block decompression ------------------------ -At the beginning we have 5 streams and their sizes. -Decompressor should iterate through `Tokens_Stream`. Each token is 1-byte long and describes how to get data from other streams. -If token points a stream that is already empty it means that data is corrupted. - - -Lizard token decompression ------------------------ -The token is a one byte. Token decribes: -- how many literals should be copied from `Literals_Stream` -- if offset should be read from `16-bit_Offsets_Stream` or `24-bit_Offsets_Stream` -- how many bytes are part of a match and should be copied from a sliding window - -Lizard uses 4 types of tokens: -- [0_MMMM_LLL] - 3-bit literal length (0-7+), use offset from `16-bit_Offsets_Stream`, 4-bit match length (4-15+) -- [1_MMMM_LLL] - 3-bit literal length (0-7+), use last offset, 4-bit match length (0-15+) -- token 31 - no literal length, use offset from `24-bit_Offsets_Stream`, match length (47+) -- token 0-30 - no literal length, use offset from `24-bit_Offsets_Stream`, 31 match lengths (16-46) - -Lizard uses different output codewords and is not compatible with LZ4. LZ4 output codewords are 3 byte long (24-bit) and look as follows: -- LLLL_MMMM OOOOOOOO OOOOOOOO - 16-bit offset, 4-bit match length, 4-bit literal length - - -The format of `Lengths_Stream` ------------------------------- -`Lengths_Stream` contains lenghts in the the following format: -- when 'First_Byte' is < 254 then lenght is equal 'First_Byte' -- when 'First_Byte' is 254 then lenght is equal to value of 2-bytes after 'First_Byte' i.e. 0-65536 -- when 'First_Byte' is 255 then lenght is equal to value of 3-bytes after 'First_Byte' i.e. 0-16777215 - - -[0_MMMM_LLL] and [1_MMMM_LLL] tokens ---------------------------------------- -The length of literals to be copied from `Literals_Stream` depends on the literal length field (LLL) that uses 3 bits of the token. -Therefore each field ranges from 0 to 7. -If the value is 7, then the lenght is increased with a length taken from `Lengths_Stream`. - -Example 1 : A literal length of 48 will be represented as : - - - 7 : value for the 3-bits LLL field - - 41 : (=48-7) remaining length to reach 48 (in `Lengths_Stream`) - -Example 2 : A literal length of 280 for will be represented as : - - - 7 : value for the 3-bits LLL field - - 254 : informs that remaining length (=280-7) must be represented as 2-bytes (in `Lengths_Stream`) - - 273 : (=280-7) encoded as 2-bytes (in `Lengths_Stream`) - -Example 3 : A literal length of 7 for will be represented as : - - - 7 : value for the 3-bits LLL field - - 0 : (=7-7) yes, the zero must be output (in `Lengths_Stream`) - -After copying 0 or more literals from `Literals_Stream` we can prepare the match copy operation which depends on a offset and a match length. -The flag "0" informs that decoder should use the last encoded offset. -The flag "1" informs that the offset is a 2 bytes value (16-bit), in little endian format and should be taken from `16-bit_Offsets_Stream`. - -The match length depends on the match length field (MMMM) that uses 4 bits of the token. -Therefore each field ranges from 0 to 15. Values from 0-3 are forbidden with offset taken from `16-bit_Offsets_Stream`. -If the value is 15, then the lenght is increased with a length taken from `Lengths_Stream`. - -With the offset and the match length, -the decoder can now proceed to copy the data from the already decoded buffer. - - -Lizard block epilogue ------------------- -When all tokens are read from `Tokens_Stream` and interpreted all remaining streams should also be empty. -Otherwise, it means that the data is corrupted. The only exception is `Literals_Stream` that should have at least 16 remaining literals what -allows fast memory copy operations. The remaining literals up to the end of `Literals_Stream` should be appended to the output data. - - -Tokens 0-31 ------------ -The offset is a 3 bytes value (24-bit), in little endian format and should be taken from `24-bit_Offsets_Stream`. -The offset represents the position of the match to be copied from. -1 means "current position - 1 byte". -The maximum offset value is (1<<24)-1, 1<<24 cannot be coded. -Note that 0 is an invalid value, not used. - -The 'Token_Value' ranges from 0 to 31. -The match length is equal to 'Token_Value + 16 that is from 16 to 47. -If match length is 47, the lenght is increased with a length taken from `Lengths_Stream`. - - -Parsing restrictions ------------------------ -There are specific parsing rules to respect in order to remain compatible -with assumptions made by the decoder : - -1. The last 16 bytes are always literals what allows fast memory copy operations. -2. The last match must start at least 20 bytes before end of block. - Consequently, a block with less than 20 bytes cannot be compressed. - -These rules are in place to ensure that the decoder -will never read beyond the input buffer, nor write beyond the output buffer. - -Note that the last sequence is also incomplete, -and stops right after literals. - - -Additional notes ------------------------ -There is no assumption nor limits to the way the compressor -searches and selects matches within the source data block. -It could be a fast scan, a multi-probe, a full search using BST, -standard hash chains or MMC, well whatever. - -Advanced parsing strategies can also be implemented, such as lazy match, -or full optimal parsing. - -All these trade-off offer distinctive speed/memory/compression advantages. -Whatever the method used by the compressor, its result will be decodable -by any Lizard decoder if it follows the format specification described above. diff --git a/contrib/lizard/doc/lizard_Frame_format.md b/contrib/lizard/doc/lizard_Frame_format.md deleted file mode 100644 index ce24b0e23e2..00000000000 --- a/contrib/lizard/doc/lizard_Frame_format.md +++ /dev/null @@ -1,312 +0,0 @@ -Lizard v1.x Frame Format Description -================================= - -###Notices - -Copyright (c) 2013-2015 Yann Collet - -Copyright (c) 2016 Przemyslaw Skibinski - -Permission is granted to copy and distribute this document -for any purpose and without charge, -including translations into other languages -and incorporation into compilations, -provided that the copyright notice and this notice are preserved, -and that any substantive changes or deletions from the original -are clearly marked. -Distribution of this document is unlimited. - -###Version - -1.0 (8-10-2016) - - -Introduction ------------- - -The purpose of this document is to define a lossless compressed data format, -that is independent of CPU type, operating system, -file system and character set, suitable for -File compression, Pipe and streaming compression -using the Lizard algorithm. - -The data can be produced or consumed, -even for an arbitrarily long sequentially presented input data stream, -using only an a priori bounded amount of intermediate storage, -and hence can be used in data communications. -The format uses the Lizard compression method, -and optional [xxHash-32 checksum method](https://github.com/Cyan4973/xxHash), -for detection of data corruption. - -The data format defined by this specification -does not attempt to allow random access to compressed data. - -This specification is intended for use by implementers of software -to compress data into Lizard format and/or decompress data from Lizard format. -The text of the specification assumes a basic background in programming -at the level of bits and other primitive data representations. - -Unless otherwise indicated below, -a compliant compressor must produce data sets -that conform to the specifications presented here. -It doesn’t need to support all options though. - -A compliant decompressor must be able to decompress -at least one working set of parameters -that conforms to the specifications presented here. -It may also ignore checksums. -Whenever it does not support a specific parameter within the compressed stream, -it must produce a non-ambiguous error code -and associated error message explaining which parameter is unsupported. - - -General Structure of Lizard Frame format -------------------------------------- - -| MagicNb | F. Descriptor | Block | (...) | EndMark | C. Checksum | -|:-------:|:-------------:| ----- | ----- | ------- | ----------- | -| 4 bytes | 3-11 bytes | | | 4 bytes | 0-4 bytes | - -__Magic Number__ - -4 Bytes, Little endian format. -Value : 0x184D2206 (it was 0x184D2204 for LZ4 and 0x184D2205 for LZ5 v1.x) - -__Frame Descriptor__ - -3 to 11 Bytes, to be detailed in the next part. -Most important part of the spec. - -__Data Blocks__ - -To be detailed later on. -That’s where compressed data is stored. - -__EndMark__ - -The flow of blocks ends when the last data block has a size of “0”. -The size is expressed as a 32-bits value. - -__Content Checksum__ - -Content Checksum verify that the full content has been decoded correctly. -The content checksum is the result -of [xxh32() hash function](https://github.com/Cyan4973/xxHash) -digesting the original (decoded) data as input, and a seed of zero. -Content checksum is only present when its associated flag -is set in the frame descriptor. -Content Checksum validates the result, -that all blocks were fully transmitted in the correct order and without error, -and also that the encoding/decoding process itself generated no distortion. -Its usage is recommended. - -__Frame Concatenation__ - -In some circumstances, it may be preferable to append multiple frames, -for example in order to add new data to an existing compressed file -without re-framing it. - -In such case, each frame has its own set of descriptor flags. -Each frame is considered independent. -The only relation between frames is their sequential order. - -The ability to decode multiple concatenated frames -within a single stream or file -is left outside of this specification. -As an example, the reference lizard command line utility behavior is -to decode all concatenated frames in their sequential order. - - -Frame Descriptor ----------------- - -| FLG | BD | (Content Size) | HC | -| ------- | ------- |:--------------:| ------- | -| 1 byte | 1 byte | 0 - 8 bytes | 1 byte | - -The descriptor uses a minimum of 3 bytes, -and up to 11 bytes depending on optional parameters. - -__FLG byte__ - -| BitNb | 7-6 | 5 | 4 | 3 | 2 | 1-0 | -| ------- | ------- | ------- | --------- | ------- | --------- | -------- | -|FieldName| Version | B.Indep | B.Checksum| C.Size | C.Checksum|*Reserved*| - - -__BD byte__ - -| BitNb | 7 | 6-5-4 | 3-2-1-0 | -| ------- | -------- | ------------ | -------- | -|FieldName|*Reserved*| Block MaxSize|*Reserved*| - -In the tables, bit 7 is highest bit, while bit 0 is lowest. - -__Version Number__ - -2-bits field, must be set to “01”. -Any other value cannot be decoded by this version of the specification. -Other version numbers will use different flag layouts. - -__Block Independence flag__ - -If this flag is set to “1”, blocks are independent. -If this flag is set to “0”, each block depends on previous ones -(up to Lizard window size, which is 16 MB). -In such case, it’s necessary to decode all blocks in sequence. - -Block dependency improves compression ratio, especially for small blocks. -On the other hand, it makes direct jumps or multi-threaded decoding impossible. - -__Block checksum flag__ - -If this flag is set, each data block will be followed by a 4-bytes checksum, -calculated by using the xxHash-32 algorithm on the raw (compressed) data block. -The intention is to detect data corruption (storage or transmission errors) -immediately, before decoding. -Block checksum usage is optional. - -__Content Size flag__ - -If this flag is set, the uncompressed size of data included within the frame -will be present as an 8 bytes unsigned little endian value, after the flags. -Content Size usage is optional. - -__Content checksum flag__ - -If this flag is set, a content checksum will be appended after the EndMark. - -Recommended value : “1” (content checksum is present) - -__Block Maximum Size__ - -This information is intended to help the decoder allocate memory. -Size here refers to the original (uncompressed) data size. -Block Maximum Size is one value among the following table : - -| 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | -| --- | ------ | ------ | ---- | ---- | ----- | ----- | ------ | -| N/A | 128 KB | 256 KB | 1 MB | 4 MB | 16 MB | 64 MB | 256 MB | - - -The decoder may refuse to allocate block sizes above a (system-specific) size. -Unused values may be used in a future revision of the spec. -A decoder conformant to the current version of the spec -is only able to decode blocksizes defined in this spec. - -__Reserved bits__ - -Value of reserved bits **must** be 0 (zero). -Reserved bit might be used in a future version of the specification, -typically enabling new optional features. -If this happens, a decoder respecting the current version of the specification -shall not be able to decode such a frame. - -__Content Size__ - -This is the original (uncompressed) size. -This information is optional, and only present if the associated flag is set. -Content size is provided using unsigned 8 Bytes, for a maximum of 16 HexaBytes. -Format is Little endian. -This value is informational, typically for display or memory allocation. -It can be skipped by a decoder, or used to validate content correctness. - -__Header Checksum__ - -One-byte checksum of combined descriptor fields, including optional ones. -The value is the second byte of xxh32() : ` (xxh32()>>8) & 0xFF ` -using zero as a seed, -and the full Frame Descriptor as an input -(including optional fields when they are present). -A wrong checksum indicates an error in the descriptor. -Header checksum is informational and can be skipped. - - -Data Blocks ------------ - -| Block Size | data | (Block Checksum) | -|:----------:| ------ |:----------------:| -| 4 bytes | | 0 - 4 bytes | - - -__Block Size__ - -This field uses 4-bytes, format is little-endian. - -The highest bit is “1” if data in the block is uncompressed. - -The highest bit is “0” if data in the block is compressed by Lizard. - -All other bits give the size, in bytes, of the following data block -(the size does not include the block checksum if present). - -Block Size shall never be larger than Block Maximum Size. -Such a thing could happen for incompressible source data. -In such case, such a data block shall be passed in uncompressed format. - -__Data__ - -Where the actual data to decode stands. -It might be compressed or not, depending on previous field indications. -Uncompressed size of Data can be any size, up to “block maximum size”. -Note that data block is not necessarily full : -an arbitrary “flush” may happen anytime. Any block can be “partially filled”. - -__Block checksum__ - -Only present if the associated flag is set. -This is a 4-bytes checksum value, in little endian format, -calculated by using the xxHash-32 algorithm on the raw (undecoded) data block, -and a seed of zero. -The intention is to detect data corruption (storage or transmission errors) -before decoding. - -Block checksum is cumulative with Content checksum. - - -Skippable Frames ----------------- - -| Magic Number | Frame Size | User Data | -|:------------:|:----------:| --------- | -| 4 bytes | 4 bytes | | - -Skippable frames allow the integration of user-defined data -into a flow of concatenated frames. -Its design is pretty straightforward, -with the sole objective to allow the decoder to quickly skip -over user-defined data and continue decoding. - -For the purpose of facilitating identification, -it is discouraged to start a flow of concatenated frames with a skippable frame. -If there is a need to start such a flow with some user data -encapsulated into a skippable frame, -it’s recommended to start with a zero-byte Lizard frame -followed by a skippable frame. -This will make it easier for file type identifiers. - - -__Magic Number__ - -4 Bytes, Little endian format. -Value : 0x184D2A5X, which means any value from 0x184D2A50 to 0x184D2A5F. -All 16 values are valid to identify a skippable frame. - -__Frame Size__ - -This is the size, in bytes, of the following User Data -(without including the magic number nor the size field itself). -4 Bytes, Little endian format, unsigned 32-bits. -This means User Data can’t be bigger than (2^32-1) Bytes. - -__User Data__ - -User Data can be anything. Data will just be skipped by the decoder. - - - -Version changes ---------------- - -1.0 : based on LZ4 Frame Format Description 1.5.1 (31/03/2015) diff --git a/contrib/lizard/examples/.gitignore b/contrib/lizard/examples/.gitignore deleted file mode 100644 index f6f8bbcca09..00000000000 --- a/contrib/lizard/examples/.gitignore +++ /dev/null @@ -1,8 +0,0 @@ -/Makefile.liz* -/printVersion -/doubleBuffer -/ringBuffer -/ringBufferHC -/lineCompress -/frameCompress -/*.exe diff --git a/contrib/lizard/examples/HCStreaming_ringBuffer.c b/contrib/lizard/examples/HCStreaming_ringBuffer.c deleted file mode 100644 index 3e4d75a5d78..00000000000 --- a/contrib/lizard/examples/HCStreaming_ringBuffer.c +++ /dev/null @@ -1,241 +0,0 @@ -// Lizard streaming API example : ring buffer -// Based on previous work from Takayuki Matsuoka - - -/************************************** - * Compiler Options - **************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define _CRT_SECURE_NO_WARNINGS // for MSVC -# define snprintf sprintf_s -#endif - -#define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) -#ifdef __GNUC__ -# pragma GCC diagnostic ignored "-Wmissing-braces" /* GCC bug 53119 : doesn't accept { 0 } as initializer (https://gcc.gnu.org/bugzilla/show_bug.cgi?id=53119) */ -#endif - - -/************************************** - * Includes - **************************************/ -#include "lizard_compress.h" -#include "lizard_decompress.h" -#include "lizard_common.h" - -#include -#include -#include -#include - -enum { - MESSAGE_MAX_BYTES = 1024, - RING_BUFFER_BYTES = 1024 * 8 + MESSAGE_MAX_BYTES, - DEC_BUFFER_BYTES = RING_BUFFER_BYTES + MESSAGE_MAX_BYTES // Intentionally larger to test unsynchronized ring buffers -}; - - -size_t write_int32(FILE* fp, int32_t i) { - return fwrite(&i, sizeof(i), 1, fp); -} - -size_t write_bin(FILE* fp, const void* array, int arrayBytes) { - return fwrite(array, 1, arrayBytes, fp); -} - -size_t read_int32(FILE* fp, int32_t* i) { - return fread(i, sizeof(*i), 1, fp); -} - -size_t read_bin(FILE* fp, void* array, int arrayBytes) { - return fread(array, 1, arrayBytes, fp); -} - - -void test_compress(FILE* outFp, FILE* inpFp) -{ - Lizard_stream_t* lizardStream = Lizard_createStream(0); - static char inpBuf[RING_BUFFER_BYTES]; - int inpOffset = 0; - - if (!lizardStream) return; - - for(;;) - { - // Read random length ([1,MESSAGE_MAX_BYTES]) data to the ring buffer. - char* const inpPtr = &inpBuf[inpOffset]; - const int randomLength = (rand() % MESSAGE_MAX_BYTES) + 1; - const int inpBytes = (int) read_bin(inpFp, inpPtr, randomLength); - if (0 == inpBytes) break; - - { - char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; - const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, Lizard_compressBound(inpBytes)); - - if(cmpBytes <= 0) break; - write_int32(outFp, cmpBytes); - write_bin(outFp, cmpBuf, cmpBytes); - - inpOffset += inpBytes; - - // Wraparound the ringbuffer offset - if(inpOffset >= RING_BUFFER_BYTES - MESSAGE_MAX_BYTES) - inpOffset = 0; - } - } - - write_int32(outFp, 0); - Lizard_freeStream(lizardStream); -} - - -void test_decompress(FILE* outFp, FILE* inpFp) -{ - static char decBuf[DEC_BUFFER_BYTES]; - int decOffset = 0; - Lizard_streamDecode_t lizardStreamDecode_body = { 0 }; - Lizard_streamDecode_t* lizardStreamDecode = &lizardStreamDecode_body; - - for(;;) - { - int cmpBytes = 0; - char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; - - { - const size_t r0 = read_int32(inpFp, &cmpBytes); - size_t r1; - if(r0 != 1 || cmpBytes <= 0) - break; - - r1 = read_bin(inpFp, cmpBuf, cmpBytes); - if(r1 != (size_t) cmpBytes) - break; - } - - { - char* const decPtr = &decBuf[decOffset]; - const int decBytes = Lizard_decompress_safe_continue( - lizardStreamDecode, cmpBuf, decPtr, cmpBytes, MESSAGE_MAX_BYTES); - if(decBytes <= 0) - break; - - decOffset += decBytes; - write_bin(outFp, decPtr, decBytes); - - // Wraparound the ringbuffer offset - if(decOffset >= DEC_BUFFER_BYTES - MESSAGE_MAX_BYTES) - decOffset = 0; - } - } -} - - -// Compare 2 files content -// return 0 if identical -// return ByteNb>0 if different -size_t compare(FILE* f0, FILE* f1) -{ - size_t result = 1; - - for (;;) - { - char b0[65536]; - char b1[65536]; - const size_t r0 = fread(b0, 1, sizeof(b0), f0); - const size_t r1 = fread(b1, 1, sizeof(b1), f1); - - if ((r0==0) && (r1==0)) return 0; // success - - if (r0 != r1) - { - size_t smallest = r0; - if (r1 -#include -#include -#include - -enum { - BLOCK_BYTES = 1024 * 8, -// BLOCK_BYTES = 1024 * 64, -}; - - -size_t write_int(FILE* fp, int i) { - return fwrite(&i, sizeof(i), 1, fp); -} - -size_t write_bin(FILE* fp, const void* array, size_t arrayBytes) { - return fwrite(array, 1, arrayBytes, fp); -} - -size_t read_int(FILE* fp, int* i) { - return fread(i, sizeof(*i), 1, fp); -} - -size_t read_bin(FILE* fp, void* array, size_t arrayBytes) { - return fread(array, 1, arrayBytes, fp); -} - - -void test_compress(FILE* outFp, FILE* inpFp) -{ - Lizard_stream_t* lizardStream = Lizard_createStream_MinLevel(); - char inpBuf[2][BLOCK_BYTES]; - int inpBufIndex = 0; - - if (!lizardStream) return; - lizardStream = Lizard_resetStream_MinLevel(lizardStream); - if (!lizardStream) return; - - for(;;) { - char* const inpPtr = inpBuf[inpBufIndex]; - const int inpBytes = (int) read_bin(inpFp, inpPtr, BLOCK_BYTES); - if(0 == inpBytes) { - break; - } - - { - char cmpBuf[LIZARD_COMPRESSBOUND(BLOCK_BYTES)]; - const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, sizeof(cmpBuf)); - if(cmpBytes <= 0) { - break; - } - write_int(outFp, cmpBytes); - write_bin(outFp, cmpBuf, (size_t) cmpBytes); - } - - inpBufIndex = (inpBufIndex + 1) % 2; - } - - write_int(outFp, 0); - Lizard_freeStream(lizardStream); -} - - -void test_decompress(FILE* outFp, FILE* inpFp) -{ - Lizard_streamDecode_t lizardStreamDecode_body; - Lizard_streamDecode_t* lizardStreamDecode = &lizardStreamDecode_body; - - char decBuf[2][BLOCK_BYTES]; - int decBufIndex = 0; - - Lizard_setStreamDecode(lizardStreamDecode, NULL, 0); - - for(;;) { - char cmpBuf[LIZARD_COMPRESSBOUND(BLOCK_BYTES)]; - int cmpBytes = 0; - - { - const size_t readCount0 = read_int(inpFp, &cmpBytes); - if(readCount0 != 1 || cmpBytes <= 0) { - break; - } - - const size_t readCount1 = read_bin(inpFp, cmpBuf, (size_t) cmpBytes); - if(readCount1 != (size_t) cmpBytes) { - break; - } - } - - { - char* const decPtr = decBuf[decBufIndex]; - const int decBytes = Lizard_decompress_safe_continue( - lizardStreamDecode, cmpBuf, decPtr, cmpBytes, BLOCK_BYTES); - if(decBytes <= 0) { - break; - } - write_bin(outFp, decPtr, (size_t) decBytes); - } - - decBufIndex = (decBufIndex + 1) % 2; - } -} - - -int compare(FILE* fp0, FILE* fp1) -{ - int result = 0; - - while(0 == result) { - char b0[65536]; - char b1[65536]; - const size_t r0 = read_bin(fp0, b0, sizeof(b0)); - const size_t r1 = read_bin(fp1, b1, sizeof(b1)); - - result = (int) r0 - (int) r1; - - if(0 == r0 || 0 == r1) { - break; - } - if(0 == result) { - result = memcmp(b0, b1, r0); - } - } - - return result; -} - - -int main(int argc, char* argv[]) -{ - char inpFilename[256] = { 0 }; - char lizardFilename[256] = { 0 }; - char decFilename[256] = { 0 }; - - if(argc < 2) { - printf("Please specify input filename\n"); - return 0; - } - - snprintf(inpFilename, 256, "%s", argv[1]); - snprintf(lizardFilename, 256, "%s.lizs-%d", argv[1], BLOCK_BYTES); - snprintf(decFilename, 256, "%s.lizs-%d.dec", argv[1], BLOCK_BYTES); - - printf("inp = [%s]\n", inpFilename); - printf("lizard = [%s]\n", lizardFilename); - printf("dec = [%s]\n", decFilename); - - // compress - { - FILE* inpFp = fopen(inpFilename, "rb"); - FILE* outFp = fopen(lizardFilename, "wb"); - - printf("compress : %s -> %s\n", inpFilename, lizardFilename); - test_compress(outFp, inpFp); - printf("compress : done\n"); - - fclose(outFp); - fclose(inpFp); - } - - // decompress - { - FILE* inpFp = fopen(lizardFilename, "rb"); - FILE* outFp = fopen(decFilename, "wb"); - - printf("decompress : %s -> %s\n", lizardFilename, decFilename); - test_decompress(outFp, inpFp); - printf("decompress : done\n"); - - fclose(outFp); - fclose(inpFp); - } - - // verify - { - FILE* inpFp = fopen(inpFilename, "rb"); - FILE* decFp = fopen(decFilename, "rb"); - - printf("verify : %s <-> %s\n", inpFilename, decFilename); - const int cmp = compare(inpFp, decFp); - if(0 == cmp) { - printf("verify : OK\n"); - } else { - printf("verify : NG\n"); - } - - fclose(decFp); - fclose(inpFp); - } - - return 0; -} diff --git a/contrib/lizard/examples/blockStreaming_doubleBuffer.md b/contrib/lizard/examples/blockStreaming_doubleBuffer.md deleted file mode 100644 index c27afcfb98c..00000000000 --- a/contrib/lizard/examples/blockStreaming_doubleBuffer.md +++ /dev/null @@ -1,100 +0,0 @@ -# Lizard streaming API Example : Double Buffer -by *Takayuki Matsuoka* - -`blockStreaming_doubleBuffer.c` is Lizard Straming API example which implements double buffer (de)compression. - -Please note : - - - Firstly, read "Lizard Streaming API Basics". - - This is relatively advanced application example. - - Output file is not compatible with lizard_frame and platform dependent. - - -## What's the point of this example ? - - - Handle huge file in small amount of memory - - Always better compression ratio than Block API - - Uniform block size - - -## How the compression works - -First of all, allocate "Double Buffer" for input and Lizard compressed data buffer for output. -Double buffer has two pages, "first" page (Page#1) and "second" page (Page#2). - -``` - Double Buffer - - Page#1 Page#2 - +---------+---------+ - | Block#1 | | - +----+----+---------+ - | - v - {Out#1} - - - Prefix Dependency - +---------+ - | | - v | - +---------+----+----+ - | Block#1 | Block#2 | - +---------+----+----+ - | - v - {Out#2} - - - External Dictionary Mode - +---------+ - | | - | v - +----+----+---------+ - | Block#3 | Block#2 | - +----+----+---------+ - | - v - {Out#3} - - - Prefix Dependency - +---------+ - | | - v | - +---------+----+----+ - | Block#3 | Block#4 | - +---------+----+----+ - | - v - {Out#4} -``` - -Next, read first block to double buffer's first page. And compress it by `Lizard_compress_continue()`. -For the first time, Lizard doesn't know any previous dependencies, -so it just compress the line without dependencies and generates compressed block {Out#1} to Lizard compressed data buffer. -After that, write {Out#1} to the file. - -Next, read second block to double buffer's second page. And compress it. -In this time, Lizard can use dependency to Block#1 to improve compression ratio. -This dependency is called "Prefix mode". - -Next, read third block to double buffer's *first* page. And compress it. -Also this time, Lizard can use dependency to Block#2. -This dependency is called "External Dictonaly mode". - -Continue these procedure to the end of the file. - - -## How the decompression works - -Decompression will do reverse order. - - - Read first compressed block. - - Decompress it to the first page and write that page to the file. - - Read second compressed block. - - Decompress it to the second page and write that page to the file. - - Read third compressed block. - - Decompress it to the *first* page and write that page to the file. - -Continue these procedure to the end of the compressed file. diff --git a/contrib/lizard/examples/blockStreaming_lineByLine.c b/contrib/lizard/examples/blockStreaming_lineByLine.c deleted file mode 100644 index 9d239533501..00000000000 --- a/contrib/lizard/examples/blockStreaming_lineByLine.c +++ /dev/null @@ -1,210 +0,0 @@ -// Lizard streaming API example : line-by-line logfile compression -// Copyright : Takayuki Matsuoka - - -#ifdef _MSC_VER /* Visual Studio */ -# define _CRT_SECURE_NO_WARNINGS -# define snprintf sprintf_s -#endif -#include "lizard_common.h" -#include "lizard_decompress.h" -#include -#include -#include -#include - -static size_t write_uint16(FILE* fp, uint16_t i) -{ - return fwrite(&i, sizeof(i), 1, fp); -} - -static size_t write_bin(FILE* fp, const void* array, int arrayBytes) -{ - return fwrite(array, 1, arrayBytes, fp); -} - -static size_t read_uint16(FILE* fp, uint16_t* i) -{ - return fread(i, sizeof(*i), 1, fp); -} - -static size_t read_bin(FILE* fp, void* array, int arrayBytes) -{ - return fread(array, 1, arrayBytes, fp); -} - - -static void test_compress( - FILE* outFp, - FILE* inpFp, - size_t messageMaxBytes, - size_t ringBufferBytes) -{ - Lizard_stream_t* const lizardStream = Lizard_createStream_MinLevel(); - const size_t cmpBufBytes = LIZARD_COMPRESSBOUND(messageMaxBytes); - char* const cmpBuf = (char*) malloc(cmpBufBytes); - char* const inpBuf = (char*) malloc(ringBufferBytes); - int inpOffset = 0; - - for ( ; ; ) - { - char* const inpPtr = &inpBuf[inpOffset]; - -#if 0 - // Read random length data to the ring buffer. - const int randomLength = (rand() % messageMaxBytes) + 1; - const int inpBytes = (int) read_bin(inpFp, inpPtr, randomLength); - if (0 == inpBytes) break; -#else - // Read line to the ring buffer. - int inpBytes = 0; - if (!fgets(inpPtr, (int) messageMaxBytes, inpFp)) - break; - inpBytes = (int) strlen(inpPtr); -#endif - - { - const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, cmpBufBytes); - if (cmpBytes <= 0) break; - write_uint16(outFp, (uint16_t) cmpBytes); - write_bin(outFp, cmpBuf, cmpBytes); - - // Add and wraparound the ringbuffer offset - inpOffset += inpBytes; - if ((size_t)inpOffset >= ringBufferBytes - messageMaxBytes) inpOffset = 0; - } - } - write_uint16(outFp, 0); - - free(inpBuf); - free(cmpBuf); - Lizard_freeStream(lizardStream); -} - - -static void test_decompress( - FILE* outFp, - FILE* inpFp, - size_t messageMaxBytes, - size_t ringBufferBytes) -{ - Lizard_streamDecode_t* const lizardStreamDecode = Lizard_createStreamDecode(); - char* const cmpBuf = (char*) malloc(LIZARD_COMPRESSBOUND(messageMaxBytes)); - char* const decBuf = (char*) malloc(ringBufferBytes); - int decOffset = 0; - - for ( ; ; ) - { - uint16_t cmpBytes = 0; - - if (read_uint16(inpFp, &cmpBytes) != 1) break; - if (cmpBytes <= 0) break; - if (read_bin(inpFp, cmpBuf, cmpBytes) != cmpBytes) break; - - { - char* const decPtr = &decBuf[decOffset]; - const int decBytes = Lizard_decompress_safe_continue( - lizardStreamDecode, cmpBuf, decPtr, cmpBytes, (int) messageMaxBytes); - if (decBytes <= 0) break; - write_bin(outFp, decPtr, decBytes); - - // Add and wraparound the ringbuffer offset - decOffset += decBytes; - if ((size_t)decOffset >= ringBufferBytes - messageMaxBytes) decOffset = 0; - } - } - - free(decBuf); - free(cmpBuf); - Lizard_freeStreamDecode(lizardStreamDecode); -} - - -static int compare(FILE* f0, FILE* f1) -{ - int result = 0; - const size_t tempBufferBytes = 65536; - char* const b0 = (char*) malloc(tempBufferBytes); - char* const b1 = (char*) malloc(tempBufferBytes); - - while(0 == result) - { - const size_t r0 = fread(b0, 1, tempBufferBytes, f0); - const size_t r1 = fread(b1, 1, tempBufferBytes, f1); - - result = (int) r0 - (int) r1; - - if (0 == r0 || 0 == r1) break; - if (0 == result) result = memcmp(b0, b1, r0); - } - - free(b1); - free(b0); - return result; -} - - -int main(int argc, char* argv[]) -{ - enum { - MESSAGE_MAX_BYTES = 1024, - RING_BUFFER_BYTES = 1024 * 256 + MESSAGE_MAX_BYTES, - }; - - char inpFilename[256] = { 0 }; - char lizardFilename[256] = { 0 }; - char decFilename[256] = { 0 }; - - if (argc < 2) - { - printf("Please specify input filename\n"); - return 0; - } - - snprintf(inpFilename, 256, "%s", argv[1]); - snprintf(lizardFilename, 256, "%s.lizs", argv[1]); - snprintf(decFilename, 256, "%s.lizs.dec", argv[1]); - - printf("inp = [%s]\n", inpFilename); - printf("lizard = [%s]\n", lizardFilename); - printf("dec = [%s]\n", decFilename); - - // compress - { - FILE* inpFp = fopen(inpFilename, "rb"); - FILE* outFp = fopen(lizardFilename, "wb"); - - test_compress(outFp, inpFp, MESSAGE_MAX_BYTES, RING_BUFFER_BYTES); - - fclose(outFp); - fclose(inpFp); - } - - // decompress - { - FILE* inpFp = fopen(lizardFilename, "rb"); - FILE* outFp = fopen(decFilename, "wb"); - - test_decompress(outFp, inpFp, MESSAGE_MAX_BYTES, RING_BUFFER_BYTES); - - fclose(outFp); - fclose(inpFp); - } - - // verify - { - FILE* inpFp = fopen(inpFilename, "rb"); - FILE* decFp = fopen(decFilename, "rb"); - - const int cmp = compare(inpFp, decFp); - if (0 == cmp) - printf("Verify : OK\n"); - else - printf("Verify : NG\n"); - - fclose(decFp); - fclose(inpFp); - } - - return 0; -} diff --git a/contrib/lizard/examples/blockStreaming_lineByLine.md b/contrib/lizard/examples/blockStreaming_lineByLine.md deleted file mode 100644 index ac56b879c90..00000000000 --- a/contrib/lizard/examples/blockStreaming_lineByLine.md +++ /dev/null @@ -1,122 +0,0 @@ -# Lizard streaming API Example : Line by Line Text Compression -by *Takayuki Matsuoka* - -`blockStreaming_lineByLine.c` is Lizard Straming API example which implements line by line incremental (de)compression. - -Please note the following restrictions : - - - Firstly, read "Lizard Streaming API Basics". - - This is relatively advanced application example. - - Output file is not compatible with lizard_frame and platform dependent. - - -## What's the point of this example ? - - - Line by line incremental (de)compression. - - Handle huge file in small amount of memory - - Generally better compression ratio than Block API - - Non-uniform block size - - -## How the compression works - -First of all, allocate "Ring Buffer" for input and Lizard compressed data buffer for output. - -``` -(1) - Ring Buffer - - +--------+ - | Line#1 | - +---+----+ - | - v - {Out#1} - - -(2) - Prefix Mode Dependency - +----+ - | | - v | - +--------+-+------+ - | Line#1 | Line#2 | - +--------+---+----+ - | - v - {Out#2} - - -(3) - Prefix Prefix - +----+ +----+ - | | | | - v | v | - +--------+-+------+-+------+ - | Line#1 | Line#2 | Line#3 | - +--------+--------+---+----+ - | - v - {Out#3} - - -(4) - External Dictionary Mode - +----+ +----+ - | | | | - v | v | - ------+--------+-+------+-+--------+ - | .... | Line#X | Line#X+1 | - ------+--------+--------+-----+----+ - ^ | - | v - | {Out#X+1} - | - Reset - - -(5) - Prefix - +-----+ - | | - v | - ------+--------+--------+----------+--+-------+ - | .... | Line#X | Line#X+1 | Line#X+2 | - ------+--------+--------+----------+-----+----+ - ^ | - | v - | {Out#X+2} - | - Reset -``` - -Next (see (1)), read first line to ringbuffer and compress it by `Lizard_compress_continue()`. -For the first time, Lizard doesn't know any previous dependencies, -so it just compress the line without dependencies and generates compressed line {Out#1} to Lizard compressed data buffer. -After that, write {Out#1} to the file and forward ringbuffer offset. - -Do the same things to second line (see (2)). -But in this time, Lizard can use dependency to Line#1 to improve compression ratio. -This dependency is called "Prefix mode". - -Eventually, we'll reach end of ringbuffer at Line#X (see (4)). -This time, we should reset ringbuffer offset. -After resetting, at Line#X+1 pointer is not adjacent, but Lizard still maintain its memory. -This is called "External Dictionary Mode". - -In Line#X+2 (see (5)), finally Lizard forget almost all memories but still remains Line#X+1. -This is the same situation as Line#2. - -Continue these procedure to the end of text file. - - -## How the decompression works - -Decompression will do reverse order. - - - Read compressed line from the file to buffer. - - Decompress it to the ringbuffer. - - Output decompressed plain text line to the file. - - Forward ringbuffer offset. If offset exceedes end of the ringbuffer, reset it. - -Continue these procedure to the end of the compressed file. diff --git a/contrib/lizard/examples/blockStreaming_ringBuffer.c b/contrib/lizard/examples/blockStreaming_ringBuffer.c deleted file mode 100644 index 011cd8b665b..00000000000 --- a/contrib/lizard/examples/blockStreaming_ringBuffer.c +++ /dev/null @@ -1,202 +0,0 @@ -// Lizard streaming API example : ring buffer -// Based on sample code from Takayuki Matsuoka - - -/************************************** - * Compiler Options - **************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define _CRT_SECURE_NO_WARNINGS // for MSVC -# define snprintf sprintf_s -#endif -#ifdef __GNUC__ -# pragma GCC diagnostic ignored "-Wmissing-braces" /* GCC bug 53119 : doesn't accept { 0 } as initializer (https://gcc.gnu.org/bugzilla/show_bug.cgi?id=53119) */ -#endif - - -/************************************** - * Includes - **************************************/ -#include -#include -#include -#include -#include "lizard_common.h" -#include "lizard_decompress.h" - - -enum { - MESSAGE_MAX_BYTES = 1024, - RING_BUFFER_BYTES = 1024 * 8 + MESSAGE_MAX_BYTES, - DECODE_RING_BUFFER = RING_BUFFER_BYTES + MESSAGE_MAX_BYTES // Intentionally larger, to test unsynchronized ring buffers -}; - - -size_t write_int32(FILE* fp, int32_t i) { - return fwrite(&i, sizeof(i), 1, fp); -} - -size_t write_bin(FILE* fp, const void* array, int arrayBytes) { - return fwrite(array, 1, arrayBytes, fp); -} - -size_t read_int32(FILE* fp, int32_t* i) { - return fread(i, sizeof(*i), 1, fp); -} - -size_t read_bin(FILE* fp, void* array, int arrayBytes) { - return fread(array, 1, arrayBytes, fp); -} - - -void test_compress(FILE* outFp, FILE* inpFp) -{ - Lizard_stream_t* lizardStream = Lizard_createStream_MinLevel(); - if (!lizardStream) return; - - static char inpBuf[RING_BUFFER_BYTES]; - int inpOffset = 0; - - for(;;) { - // Read random length ([1,MESSAGE_MAX_BYTES]) data to the ring buffer. - char* const inpPtr = &inpBuf[inpOffset]; - const int randomLength = (rand() % MESSAGE_MAX_BYTES) + 1; - const int inpBytes = (int) read_bin(inpFp, inpPtr, randomLength); - if (0 == inpBytes) break; - - { - char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; - const int cmpBytes = Lizard_compress_continue(lizardStream, inpPtr, cmpBuf, inpBytes, Lizard_compressBound(inpBytes)); - if(cmpBytes <= 0) break; - write_int32(outFp, cmpBytes); - write_bin(outFp, cmpBuf, cmpBytes); - - inpOffset += inpBytes; - - // Wraparound the ringbuffer offset - if(inpOffset >= RING_BUFFER_BYTES - MESSAGE_MAX_BYTES) inpOffset = 0; - } - } - - write_int32(outFp, 0); - Lizard_freeStream(lizardStream); -} - - -void test_decompress(FILE* outFp, FILE* inpFp) -{ - static char decBuf[DECODE_RING_BUFFER]; - int decOffset = 0; - Lizard_streamDecode_t lizardStreamDecode_body = { 0 }; - Lizard_streamDecode_t* lizardStreamDecode = &lizardStreamDecode_body; - - for(;;) { - int cmpBytes = 0; - char cmpBuf[LIZARD_COMPRESSBOUND(MESSAGE_MAX_BYTES)]; - - { - const size_t r0 = read_int32(inpFp, &cmpBytes); - if(r0 != 1 || cmpBytes <= 0) break; - - const size_t r1 = read_bin(inpFp, cmpBuf, cmpBytes); - if(r1 != (size_t) cmpBytes) break; - } - - { - char* const decPtr = &decBuf[decOffset]; - const int decBytes = Lizard_decompress_safe_continue( - lizardStreamDecode, cmpBuf, decPtr, cmpBytes, MESSAGE_MAX_BYTES); - if(decBytes <= 0) break; - decOffset += decBytes; - write_bin(outFp, decPtr, decBytes); - - // Wraparound the ringbuffer offset - if(decOffset >= DECODE_RING_BUFFER - MESSAGE_MAX_BYTES) decOffset = 0; - } - } -} - - -int compare(FILE* f0, FILE* f1) -{ - int result = 0; - - while(0 == result) { - char b0[65536]; - char b1[65536]; - const size_t r0 = fread(b0, 1, sizeof(b0), f0); - const size_t r1 = fread(b1, 1, sizeof(b1), f1); - - result = (int) r0 - (int) r1; - - if(0 == r0 || 0 == r1) { - break; - } - if(0 == result) { - result = memcmp(b0, b1, r0); - } - } - - return result; -} - - -int main(int argc, char** argv) -{ - char inpFilename[256] = { 0 }; - char lizardFilename[256] = { 0 }; - char decFilename[256] = { 0 }; - - if(argc < 2) { - printf("Please specify input filename\n"); - return 0; - } - - snprintf(inpFilename, 256, "%s", argv[1]); - snprintf(lizardFilename, 256, "%s.lizs-%d", argv[1], 0); - snprintf(decFilename, 256, "%s.lizs-%d.dec", argv[1], 0); - - printf("inp = [%s]\n", inpFilename); - printf("lizard = [%s]\n", lizardFilename); - printf("dec = [%s]\n", decFilename); - - // compress - { - FILE* inpFp = fopen(inpFilename, "rb"); - FILE* outFp = fopen(lizardFilename, "wb"); - - test_compress(outFp, inpFp); - - fclose(outFp); - fclose(inpFp); - } - - // decompress - { - FILE* inpFp = fopen(lizardFilename, "rb"); - FILE* outFp = fopen(decFilename, "wb"); - - test_decompress(outFp, inpFp); - - fclose(outFp); - fclose(inpFp); - } - - // verify - { - FILE* inpFp = fopen(inpFilename, "rb"); - FILE* decFp = fopen(decFilename, "rb"); - - const int cmp = compare(inpFp, decFp); - if(0 == cmp) { - printf("Verify : OK\n"); - } else { - printf("Verify : NG\n"); - } - - fclose(decFp); - fclose(inpFp); - } - - return 0; -} diff --git a/contrib/lizard/examples/compress_functions.c b/contrib/lizard/examples/compress_functions.c deleted file mode 100644 index 36bc40e691a..00000000000 --- a/contrib/lizard/examples/compress_functions.c +++ /dev/null @@ -1,303 +0,0 @@ -/* - * compress_functions.c - * Copyright : Kyle Harper - * License : Follows same licensing as the lizard_compress.c/lizard_compress.h program at any given time. Currently, BSD 2. - * Description: A program to demonstrate the various compression functions involved in when using Lizard_compress_MinLevel(). The idea - * is to show how each step in the call stack can be used directly, if desired. There is also some benchmarking for - * each function to demonstrate the (probably lack of) performance difference when jumping the stack. - * (If you're new to lizard, please read simple_buffer.c to understand the fundamentals) - * - * The call stack (before theoretical compiler optimizations) for Lizard_compress_MinLevel is as follows: - * Lizard_compress_MinLevel - * Lizard_compress_fast - * Lizard_compress_extState_MinLevel - * Lizard_compress_generic - * - * Lizard_compress_MinLevel() - * This is the recommended function for compressing data. It will serve as the baseline for comparison. - * Lizard_compress_fast() - * Despite its name, it's not a "fast" version of compression. It simply decides if HEAPMODE is set and either - * allocates memory on the heap for a struct or creates the struct directly on the stack. Stack access is generally - * faster but this function itself isn't giving that advantage, it's just some logic for compile time. - * Lizard_compress_extState_MinLevel() - * This simply accepts all the pointers and values collected thus far and adds logic to determine how - * Lizard_compress_generic should be invoked; specifically: can the source fit into a single pass as determined by - * Lizard_64Klimit. - * Lizard_compress_generic() - * As the name suggests, this is the generic function that ultimately does most of the heavy lifting. Calling this - * directly can help avoid some test cases and branching which might be useful in some implementation-specific - * situations, but you really need to know what you're doing AND what you're asking lizard to do! You also need a - * wrapper function because this function isn't exposed with lizard_compress.h. - * - * The call stack for decompression functions is shallow. There are 2 options: - * Lizard_decompress_safe || Lizard_decompress_fast - * Lizard_decompress_generic - * - * Lizard_decompress_safe - * This is the recommended function for decompressing data. It is considered safe because the caller specifies - * both the size of the compresssed buffer to read as well as the maximum size of the output (decompressed) buffer - * instead of just the latter. - * Lizard_decompress_generic - * This is the generic function that both of the Lizard_decompress_* functions above end up calling. Calling this - * directly is not advised, period. Furthermore, it is a static inline function in lizard_compress.c, so there isn't a symbol - * exposed for anyone using lizard_compress.h to utilize. - * - * Special Note About Decompression: - * Using the Lizard_decompress_safe() function protects against malicious (user) input. - */ - -/* Since lizard compiles with c99 and not gnu/std99 we need to enable POSIX linking for time.h structs and functions. */ -#if __STDC_VERSION__ >= 199901L -#define _XOPEN_SOURCE 600 -#else -#define _XOPEN_SOURCE 500 -#endif -#define _POSIX_C_SOURCE 199309L - -/* Includes, for Power! */ -#include "lizard_compress.h" -#include "lizard_decompress.h" -#include /* for printf() */ -#include /* for exit() */ -#include /* for atoi() memcmp() */ -#include /* for uint_types */ -#include /* for PRIu64 */ -#include /* for clock_gettime() */ -#include /* for setlocale() */ - -/* We need to know what one billion is for clock timing. */ -#define BILLION 1000000000L - -/* Create a crude set of test IDs so we can switch on them later (Can't switch() on a char[] or char*). */ -#define ID__LIZARD_COMPRESS_DEFAULT 1 -#define ID__LIZARD_COMPRESS_GENERIC 4 -#define ID__LIZARD_DECOMPRESS_SAFE 5 - - - -/* - * Easy show-error-and-bail function. - */ -void run_screaming(const char *message, const int code) { - printf("%s\n", message); - exit(code); - return; -} - - -/* - * Centralize the usage function to keep main cleaner. - */ -void usage(const char *message) { - printf("Usage: ./argPerformanceTesting \n"); - run_screaming(message, 1); - return; -} - - - -/* - * Runs the benchmark for Lizard_compress_* based on function_id. - */ -uint64_t bench( - const char *known_good_dst, - const int function_id, - const int iterations, - const char *src, - char *dst, - const size_t src_size, - const size_t max_dst_size, - const size_t comp_size - ) { - uint64_t time_taken = 0; - int rv = 0; - const int warm_up = 5000; - struct timespec start, end; - Lizard_stream_t* state = Lizard_createStream_MinLevel(); - if (!state) return; - - // Select the right function to perform the benchmark on. We perform 5000 initial loops to warm the cache and ensure that dst - // remains matching to known_good_dst between successive calls. - switch(function_id) { - case ID__LIZARD_COMPRESS_DEFAULT: - printf("Starting benchmark for function: Lizard_compress_MinLevel()\n"); - for(int junk=0; junk 1) - iterations = atoi(argv[1]); - if (iterations < 1) - usage("Argument 1 (iterations) must be > 0."); - - // First we will create 2 sources (char *) of 2000 bytes each. One normal text, the other highly-compressible text. - const char *src = "Lorem ipsum dolor sit amet, consectetur adipiscing elit. Sed luctus purus et risus vulputate, et mollis orci ullamcorper. Nulla facilisi. Fusce in ligula sed purus varius aliquet interdum vitae justo. Proin quis diam velit. Nulla varius iaculis auctor. Cras volutpat, justo eu dictum pulvinar, elit sem porttitor metus, et imperdiet metus sapien et ante. Nullam nisi nulla, ornare eu tristique eu, dignissim vitae diam. Nulla sagittis porta libero, a accumsan felis sagittis scelerisque. Integer laoreet eleifend congue. Etiam rhoncus leo vel dolor fermentum, quis luctus nisl iaculis. Praesent a erat sapien. Aliquam semper mi in lorem ultrices ultricies. Lorem ipsum dolor sit amet, consectetur adipiscing elit. In feugiat risus sed enim ultrices, at sodales nulla tristique. Maecenas eget pellentesque justo, sed pellentesque lectus. Fusce sagittis sit amet elit vel varius. Donec sed ligula nec ligula vulputate rutrum sed ut lectus. Etiam congue pharetra leo vitae cursus. Morbi enim ante, porttitor ut varius vel, tincidunt quis justo. Nunc iaculis, risus id ultrices semper, metus est efficitur ligula, vel posuere risus nunc eget purus. Ut lorem turpis, condimentum at sem sed, porta aliquam turpis. In ut sapien a nulla dictum tincidunt quis sit amet lorem. Fusce at est egestas, luctus neque eu, consectetur tortor. Phasellus eleifend ultricies nulla ac lobortis. Morbi maximus quam cursus vehicula iaculis. Maecenas cursus vel justo ut rutrum. Curabitur magna orci, dignissim eget dapibus vitae, finibus id lacus. Praesent rhoncus mattis augue vitae bibendum. Praesent porta mauris non ultrices fermentum. Quisque vulputate ipsum in sodales pulvinar. Aliquam nec mollis felis. Donec vitae augue pulvinar, congue nisl sed, pretium purus. Fusce lobortis mi ac neque scelerisque semper. Pellentesque vel est vitae magna aliquet aliquet. Nam non dolor. Nulla facilisi. Class aptent taciti sociosqu ad litora torquent per conubia nostra, per inceptos himenaeos. Morbi ac lacinia felis metus."; - const char *hc_src = "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"; - // Set and derive sizes. Since we're using strings, use strlen() + 1 for \0. - const size_t src_size = strlen(src) + 1; - const size_t max_dst_size = Lizard_compressBound(src_size); - int bytes_returned = 0; - // Now build allocations for the data we'll be playing with. - char *dst = calloc(1, max_dst_size); - char *known_good_dst = calloc(1, max_dst_size); - char *known_good_hc_dst = calloc(1, max_dst_size); - if (dst == NULL || known_good_dst == NULL || known_good_hc_dst == NULL) - run_screaming("Couldn't allocate memory for the destination buffers. Sad :(", 1); - - // Create known-good buffers to verify our tests with other functions will produce the same results. - bytes_returned = Lizard_compress_MinLevel(src, known_good_dst, src_size, max_dst_size); - if (bytes_returned < 1) - run_screaming("Couldn't create a known-good destination buffer for comparison... this is bad.", 1); - const size_t src_comp_size = bytes_returned; - bytes_returned = Lizard_compress_MinLevel(hc_src, known_good_hc_dst, src_size, max_dst_size); - if (bytes_returned < 1) - run_screaming("Couldn't create a known-good (highly compressible) destination buffer for comparison... this is bad.", 1); - const size_t hc_src_comp_size = bytes_returned; - - - /* Lizard_compress_MinLevel() */ - // This is the default function so we don't need to demonstrate how to use it. See basics.c if you need more basal information. - - /* Lizard_compress_extState_MinLevel() */ - // Using this function directly requires that we build an Lizard_stream_t struct ourselves. We do NOT have to reset it ourselves. - memset(dst, 0, max_dst_size); - Lizard_stream_t* state = Lizard_createStream_MinLevel(); - if (!state) return; - bytes_returned = Lizard_compress_extState_MinLevel(state, src, dst, src_size, max_dst_size, 1); - if (bytes_returned < 1) - run_screaming("Failed to compress src using Lizard_compress_extState_MinLevel. echo $? for return code.", bytes_returned); - if (memcmp(dst, known_good_dst, bytes_returned) != 0) - run_screaming("According to memcmp(), the value we got in dst from Lizard_compress_extState_MinLevel doesn't match the known-good value. This is bad.", 1); - - /* Lizard_compress_generic */ - // When you can exactly control the inputs and options of your Lizard needs, you can use Lizard_compress_generic and fixed (const) - // values for the enum types such as dictionary and limitations. Any other direct-use is probably a bad idea. - // - // That said, the Lizard_compress_generic() function is 'static inline' and does not have a prototype in lizard_compress.h to expose a symbol - // for it. In other words: we can't access it directly. I don't want to submit a PR that modifies lizard_compress.c/h. Yann and others can - // do that if they feel it's worth expanding this example. - // - // I will, however, leave a skeleton of what would be required to use it directly: - /* - memset(dst, 0, max_dst_size); - // Lizard_stream_t state: is already declared above. We can reuse it BUT we have to reset the stream ourselves between each call. - Lizard_resetStream_MinLevel((Lizard_stream_t *)state); - // Since src size is small we know the following enums will be used: notLimited (0), byU16 (2), noDict (0), noDictIssue (0). - bytes_returned = Lizard_compress_generic(state, src, dst, src_size, max_dst_size, notLimited, byU16, noDict, noDictIssue, 1); - if (bytes_returned < 1) - run_screaming("Failed to compress src using Lizard_compress_generic. echo $? for return code.", bytes_returned); - if (memcmp(dst, known_good_dst, bytes_returned) != 0) - run_screaming("According to memcmp(), the value we got in dst from Lizard_compress_generic doesn't match the known-good value. This is bad.", 1); - */ - Lizard_freeStream(state); - - - /* Benchmarking */ - /* Now we'll run a few rudimentary benchmarks with each function to demonstrate differences in speed based on the function used. - * Remember, we cannot call Lizard_compress_generic() directly (yet) so it's disabled. - */ - // Suite A - Normal Compressibility - char *dst_d = calloc(1, src_size); - memset(dst, 0, max_dst_size); - printf("\nStarting suite A: Normal compressible text.\n"); - uint64_t time_taken__default = bench(known_good_dst, ID__LIZARD_COMPRESS_DEFAULT, iterations, src, dst, src_size, max_dst_size, src_comp_size); - //uint64_t time_taken__generic = bench(known_good_dst, ID__LIZARD_COMPRESS_GENERIC, iterations, src, dst, src_size, max_dst_size, src_comp_size); - uint64_t time_taken__decomp_safe = bench(src, ID__LIZARD_DECOMPRESS_SAFE, iterations, known_good_dst, dst_d, src_size, max_dst_size, src_comp_size); - // Suite B - Highly Compressible - memset(dst, 0, max_dst_size); - printf("\nStarting suite B: Highly compressible text.\n"); - uint64_t time_taken_hc__default = bench(known_good_hc_dst, ID__LIZARD_COMPRESS_DEFAULT, iterations, hc_src, dst, src_size, max_dst_size, hc_src_comp_size); - //uint64_t time_taken_hc__generic = bench(known_good_hc_dst, ID__LIZARD_COMPRESS_GENERIC, iterations, hc_src, dst, src_size, max_dst_size, hc_src_comp_size); - uint64_t time_taken_hc__decomp_safe = bench(hc_src, ID__LIZARD_DECOMPRESS_SAFE, iterations, known_good_hc_dst, dst_d, src_size, max_dst_size, hc_src_comp_size); - - // Report and leave. - setlocale(LC_ALL, ""); - const char *format = "|%-14s|%-30s|%'14.9f|%'16d|%'14d|%'13.2f%%|\n"; - const char *header_format = "|%-14s|%-30s|%14s|%16s|%14s|%14s|\n"; - const char *separator = "+--------------+------------------------------+--------------+----------------+--------------+--------------+\n"; - printf("\n"); - printf("%s", separator); - printf(header_format, "Source", "Function Benchmarked", "Total Seconds", "Iterations/sec", "ns/Iteration", "% of default"); - printf("%s", separator); - printf(format, "Normal Text", "Lizard_compress_MinLevel()", (double)time_taken__default / BILLION, (int)(iterations / ((double)time_taken__default /BILLION)), time_taken__default / iterations, (double)time_taken__default * 100 / time_taken__default); - printf(format, "Normal Text", "Lizard_compress_fast()", (double)time_taken__fast / BILLION, (int)(iterations / ((double)time_taken__fast /BILLION)), time_taken__fast / iterations, (double)time_taken__fast * 100 / time_taken__default); - printf(format, "Normal Text", "Lizard_compress_extState_MinLevel()", (double)time_taken__fast_extstate / BILLION, (int)(iterations / ((double)time_taken__fast_extstate /BILLION)), time_taken__fast_extstate / iterations, (double)time_taken__fast_extstate * 100 / time_taken__default); - //printf(format, "Normal Text", "Lizard_compress_generic()", (double)time_taken__generic / BILLION, (int)(iterations / ((double)time_taken__generic /BILLION)), time_taken__generic / iterations, (double)time_taken__generic * 100 / time_taken__default); - printf(format, "Normal Text", "Lizard_decompress_safe()", (double)time_taken__decomp_safe / BILLION, (int)(iterations / ((double)time_taken__decomp_safe /BILLION)), time_taken__decomp_safe / iterations, (double)time_taken__decomp_safe * 100 / time_taken__default); - printf(header_format, "", "", "", "", "", ""); - printf(format, "Compressible", "Lizard_compress_MinLevel()", (double)time_taken_hc__default / BILLION, (int)(iterations / ((double)time_taken_hc__default /BILLION)), time_taken_hc__default / iterations, (double)time_taken_hc__default * 100 / time_taken_hc__default); - printf(format, "Compressible", "Lizard_compress_fast()", (double)time_taken_hc__fast / BILLION, (int)(iterations / ((double)time_taken_hc__fast /BILLION)), time_taken_hc__fast / iterations, (double)time_taken_hc__fast * 100 / time_taken_hc__default); - printf(format, "Compressible", "Lizard_compress_extState_MinLevel()", (double)time_taken_hc__fast_extstate / BILLION, (int)(iterations / ((double)time_taken_hc__fast_extstate /BILLION)), time_taken_hc__fast_extstate / iterations, (double)time_taken_hc__fast_extstate * 100 / time_taken_hc__default); - //printf(format, "Compressible", "Lizard_compress_generic()", (double)time_taken_hc__generic / BILLION, (int)(iterations / ((double)time_taken_hc__generic /BILLION)), time_taken_hc__generic / iterations, (double)time_taken_hc__generic * 100 / time_taken_hc__default); - printf(format, "Compressible", "Lizard_decompress_safe()", (double)time_taken_hc__decomp_safe / BILLION, (int)(iterations / ((double)time_taken_hc__decomp_safe /BILLION)), time_taken_hc__decomp_safe / iterations, (double)time_taken_hc__decomp_safe * 100 / time_taken_hc__default); - printf("%s", separator); - printf("\n"); - printf("All done, ran %d iterations per test.\n", iterations); - return 0; -} diff --git a/contrib/lizard/examples/frameCompress.c b/contrib/lizard/examples/frameCompress.c deleted file mode 100644 index 37ea0aa4fa8..00000000000 --- a/contrib/lizard/examples/frameCompress.c +++ /dev/null @@ -1,169 +0,0 @@ -// Lizardframe API example : compress a file -// Based on sample code from Zbigniew Jędrzejewski-Szmek - -#include -#include -#include -#include - -#include - -#define BUF_SIZE (16*1024) -#define LIZARD_HEADER_SIZE 19 -#define LIZARD_FOOTER_SIZE 4 - -static const LizardF_preferences_t lizard_preferences = { - { LizardF_max256KB, LizardF_blockLinked, LizardF_noContentChecksum, LizardF_frame, 0, { 0, 0 } }, - 0, /* compression level */ - 0, /* autoflush */ - { 0, 0, 0, 0 }, /* reserved, must be set to 0 */ -}; - -static int compress_file(FILE *in, FILE *out, size_t *size_in, size_t *size_out) { - LizardF_errorCode_t r; - LizardF_compressionContext_t ctx; - char *src, *buf = NULL; - size_t size, n, k, count_in = 0, count_out, offset = 0, frame_size; - - r = LizardF_createCompressionContext(&ctx, LIZARDF_VERSION); - if (LizardF_isError(r)) { - printf("Failed to create context: error %zu", r); - return 1; - } - r = 1; - - src = malloc(BUF_SIZE); - if (!src) { - printf("Not enough memory"); - goto cleanup; - } - - frame_size = LizardF_compressBound(BUF_SIZE, &lizard_preferences); - size = frame_size + LIZARD_HEADER_SIZE + LIZARD_FOOTER_SIZE; - buf = malloc(size); - if (!buf) { - printf("Not enough memory"); - goto cleanup; - } - - n = offset = count_out = LizardF_compressBegin(ctx, buf, size, &lizard_preferences); - if (LizardF_isError(n)) { - printf("Failed to start compression: error %zu", n); - goto cleanup; - } - - printf("Buffer size is %zu bytes, header size %zu bytes\n", size, n); - - for (;;) { - k = fread(src, 1, BUF_SIZE, in); - if (k == 0) - break; - count_in += k; - - n = LizardF_compressUpdate(ctx, buf + offset, size - offset, src, k, NULL); - if (LizardF_isError(n)) { - printf("Compression failed: error %zu", n); - goto cleanup; - } - - offset += n; - count_out += n; - if (size - offset < frame_size + LIZARD_FOOTER_SIZE) { - printf("Writing %zu bytes\n", offset); - - k = fwrite(buf, 1, offset, out); - if (k < offset) { - if (ferror(out)) - printf("Write failed"); - else - printf("Short write"); - goto cleanup; - } - - offset = 0; - } - } - - n = LizardF_compressEnd(ctx, buf + offset, size - offset, NULL); - if (LizardF_isError(n)) { - printf("Failed to end compression: error %zu", n); - goto cleanup; - } - - offset += n; - count_out += n; - printf("Writing %zu bytes\n", offset); - - k = fwrite(buf, 1, offset, out); - if (k < offset) { - if (ferror(out)) - printf("Write failed"); - else - printf("Short write"); - goto cleanup; - } - - *size_in = count_in; - *size_out = count_out; - r = 0; - cleanup: - if (ctx) - LizardF_freeCompressionContext(ctx); - free(src); - free(buf); - return r; -} - -static int compress(const char *input, const char *output) { - char *tmp = NULL; - FILE *in = NULL, *out = NULL; - size_t size_in = 0, size_out = 0; - int r = 1; - - if (!output) { - size_t len = strlen(input); - - output = tmp = malloc(len + 5); - if (!tmp) { - printf("Not enough memory"); - return 1; - } - strcpy(tmp, input); - strcpy(tmp + len, ".liz"); - } - - in = fopen(input, "rb"); - if (!in) { - fprintf(stderr, "Failed to open input file %s: %s\n", input, strerror(errno)); - goto cleanup; - } - - out = fopen(output, "wb"); - if (!out) { - fprintf(stderr, "Failed to open output file %s: %s\n", output, strerror(errno)); - goto cleanup; - } - - r = compress_file(in, out, &size_in, &size_out); - if (r == 0) - printf("%s: %zu → %zu bytes, %.1f%%\n", - input, size_in, size_out, - (double)size_out / size_in * 100); - cleanup: - if (in) - fclose(in); - if (out) - fclose(out); - free(tmp); - return r; -} - - -int main(int argc, char **argv) { - if (argc < 2 || argc > 3) { - fprintf(stderr, "Syntax: %s \n", argv[0]); - return EXIT_FAILURE; - } - - return compress(argv[1], argv[2]); -} diff --git a/contrib/lizard/examples/printVersion.c b/contrib/lizard/examples/printVersion.c deleted file mode 100644 index 9818fdd98cf..00000000000 --- a/contrib/lizard/examples/printVersion.c +++ /dev/null @@ -1,13 +0,0 @@ -// Lizard trivial example : print Library version number -// Copyright : Takayuki Matsuoka & Yann Collet - - -#include -#include "lizard_compress.h" - -int main(int argc, char** argv) -{ - (void)argc; (void)argv; - printf("Hello World ! Lizard Library version = %d\n", Lizard_versionNumber()); - return 0; -} diff --git a/contrib/lizard/examples/simple_buffer.c b/contrib/lizard/examples/simple_buffer.c deleted file mode 100644 index 154d3b2cb76..00000000000 --- a/contrib/lizard/examples/simple_buffer.c +++ /dev/null @@ -1,91 +0,0 @@ -/* - * simple_buffer.c - * Copyright : Kyle Harper - * License : Follows same licensing as the lizard_compress.c/lizard_compress.h program at any given time. Currently, BSD 2. - * Description: Example program to demonstrate the basic usage of the compress/decompress functions within lizard_compress.c/lizard_compress.h. - * The functions you'll likely want are Lizard_compress_MinLevel and Lizard_decompress_safe. Both of these are documented in - * the lizard_compress.h header file; I recommend reading them. - */ - -/* Includes, for Power! */ -#include "lizard_compress.h" // This is all that is required to expose the prototypes for basic compression and decompression. -#include "lizard_decompress.h" -#include // For printf() -#include // For memcmp() -#include // For exit() - -/* - * Easy show-error-and-bail function. - */ -void run_screaming(const char *message, const int code) { - printf("%s\n", message); - exit(code); - return; -} - - -/* - * main - */ -int main(void) { - /* Introduction */ - // Below we will have a Compression and Decompression section to demonstrate. There are a few important notes before we start: - // 1) The return codes of Lizard_ functions are important. Read lizard_compress.h if you're unsure what a given code means. - // 2) Lizard uses char* pointers in all Lizard_ functions. This is baked into the API and probably not going to change. If your - // program uses pointers that are unsigned char*, void*, or otherwise different you may need to do some casting or set the - // right -W compiler flags to ignore those warnings (e.g.: -Wno-pointer-sign). - - /* Compression */ - // We'll store some text into a variable pointed to by *src to be compressed later. - const char *src = "Lorem ipsum dolor sit amet, consectetur adipiscing elit."; - // The compression function needs to know how many bytes of exist. Since we're using a string, we can use strlen() + 1 (for \0). - const size_t src_size = strlen(src) + 1; - // Lizard provides a function that will tell you the maximum size of compressed output based on input data via Lizard_compressBound(). - const size_t max_dst_size = Lizard_compressBound(src_size); - // We will use that size for our destination boundary when allocating space. - char *compressed_data = malloc(max_dst_size); - if (compressed_data == NULL) - run_screaming("Failed to allocate memory for *compressed_data.", 1); - // That's all the information and preparation Lizard needs to compress *src into *compressed_data. Invoke Lizard_compress_MinLevel now - // with our size values and pointers to our memory locations. Save the return value for error checking. - int return_value = 0; - return_value = Lizard_compress_MinLevel(src, compressed_data, src_size, max_dst_size); - // Check return_value to determine what happened. - if (return_value < 0) - run_screaming("A negative result from Lizard_compress_MinLevel indicates a failure trying to compress the data. See exit code (echo $?) for value returned.", return_value); - if (return_value == 0) - run_screaming("A result of 0 means compression worked, but was stopped because the destination buffer couldn't hold all the information.", 1); - if (return_value > 0) - printf("We successfully compressed some data!\n"); - // Not only does a positive return_value mean success, the value returned == the number of bytes required. You can use this to - // realloc() *compress_data to free up memory, if desired. We'll do so just to demonstrate the concept. - const size_t compressed_data_size = return_value; - compressed_data = (char *)realloc(compressed_data, compressed_data_size); - if (compressed_data == NULL) - run_screaming("Failed to re-alloc memory for compressed_data. Sad :(", 1); - - /* Decompression */ - // Now that we've successfully compressed the information from *src to *compressed_data, let's do the opposite! We'll create a - // *new_src location of size src_size since we know that value. - char *new_src = malloc(src_size); - if (new_src == NULL) - run_screaming("Failed to allocate memory for *new_src.", 1); - // The Lizard_decompress_safe function needs to know where the compressed data is, how many bytes long it is, where the new_src - // memory location is, and how large the new_src (uncompressed) output will be. Again, save the return_value. - return_value = Lizard_decompress_safe(compressed_data, new_src, compressed_data_size, src_size); - if (return_value < 0) - run_screaming("A negative result from Lizard_decompress_fast indicates a failure trying to decompress the data. See exit code (echo $?) for value returned.", return_value); - if (return_value == 0) - run_screaming("I'm not sure this function can ever return 0. Documentation in lizard_compress.h doesn't indicate so.", 1); - if (return_value > 0) - printf("We successfully decompressed some data!\n"); - // Not only does a positive return value mean success, the value returned == the number of bytes read from the compressed_data - // stream. I'm not sure there's ever a time you'll need to know this in most cases... - - /* Validation */ - // We should be able to compare our original *src with our *new_src and be byte-for-byte identical. - if (memcmp(src, new_src, src_size) != 0) - run_screaming("Validation failed. *src and *new_src are not identical.", 1); - printf("Validation done. The string we ended up with is:\n%s\n", new_src); - return 0; -} diff --git a/contrib/lizard/examples/streaming_api_basics.md b/contrib/lizard/examples/streaming_api_basics.md deleted file mode 100644 index 85c06803455..00000000000 --- a/contrib/lizard/examples/streaming_api_basics.md +++ /dev/null @@ -1,87 +0,0 @@ -# Lizard Streaming API Basics -by *Takayuki Matsuoka* -## Lizard API sets - -Lizard has the following API sets : - - - "Auto Framing" API (lizard_frame.h) : - This is most recommended API for usual application. - It guarantees interoperability with other Lizard framing format compliant tools/libraries - such as Lizard command line utility, node-lizard, etc. - - "Block" API : This is recommended for simple purpose. - It compress single raw memory block to Lizard memory block and vice versa. - - "Streaming" API : This is designed for complex thing. - For example, compress huge stream data in restricted memory environment. - -Basically, you should use "Auto Framing" API. -But if you want to write advanced application, it's time to use Block or Streaming APIs. - - -## What is difference between Block and Streaming API ? - -Block API (de)compresses single contiguous memory block. -In other words, Lizard library find redundancy from single contiguous memory block. -Streaming API does same thing but (de)compress multiple adjacent contiguous memory block. -So Lizard library could find more redundancy than Block API. - -The following figure shows difference between API and block sizes. -In these figures, original data is splitted to 4KiBytes contiguous chunks. - -``` -Original Data - +---------------+---------------+----+----+----+ - | 4KiB Chunk A | 4KiB Chunk B | C | D |... | - +---------------+---------------+----+----+----+ - -Example (1) : Block API, 4KiB Block - +---------------+---------------+----+----+----+ - | 4KiB Chunk A | 4KiB Chunk B | C | D |... | - +---------------+---------------+----+----+----+ - | Block #1 | Block #2 | #3 | #4 |... | - +---------------+---------------+----+----+----+ - - (No Dependency) - - -Example (2) : Block API, 8KiB Block - +---------------+---------------+----+----+----+ - | 4KiB Chunk A | 4KiB Chunk B | C | D |... | - +---------------+---------------+----+----+----+ - | Block #1 |Block #2 |... | - +--------------------+----------+-------+-+----+ - ^ | ^ | - | | | | - +--------------+ +----+ - Internal Dependency Internal Dependency - - -Example (3) : Streaming API, 4KiB Block - +---------------+---------------+-----+----+----+ - | 4KiB Chunk A | 4KiB Chunk B | C | D |... | - +---------------+---------------+-----+----+----+ - | Block #1 | Block #2 | #3 | #4 |... | - +---------------+----+----------+-+---+-+--+----+ - ^ | ^ | ^ | - | | | | | | - +--------------+ +--------+ +---+ - Dependency Dependency Dependency -``` - - - In example (1), there is no dependency. - All blocks are compressed independently. - - In example (2), naturally 8KiBytes block has internal dependency. - But still block #1 and #2 are compressed independently. - - In example (3), block #2 has dependency to #1, - also #3 has dependency to #2 and #1, #4 has #3, #2 and #1, and so on. - -Here, we can observe difference between example (2) and (3). -In (2), there's no dependency between chunk B and C, but (3) has dependency between B and C. -This dependency improves compression ratio. - - -## Restriction of Streaming API - -For the efficiency, Streaming API doesn't keep mirror copy of dependent (de)compressed memory. -This means users should keep these dependent (de)compressed memory explicitly. -Usually, "Dependent memory" is previous adjacent contiguous memory up to 64KiBytes. -Lizard will not access further memories. diff --git a/contrib/lizard/lib/.gitignore b/contrib/lizard/lib/.gitignore deleted file mode 100644 index 6eedf69343a..00000000000 --- a/contrib/lizard/lib/.gitignore +++ /dev/null @@ -1,3 +0,0 @@ -# make install artefact -liblizard.pc -test/ diff --git a/contrib/lizard/lib/LICENSE b/contrib/lizard/lib/LICENSE deleted file mode 100644 index b335b0bafcf..00000000000 --- a/contrib/lizard/lib/LICENSE +++ /dev/null @@ -1,25 +0,0 @@ -Lizard Library -Copyright (C) 2011-2016, Yann Collet. -Copyright (C) 2016-2017, Przemyslaw Skibinski -All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, this - list of conditions and the following disclaimer in the documentation and/or - other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON -ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/lizard/lib/README.md b/contrib/lizard/lib/README.md deleted file mode 100644 index 2775b984d63..00000000000 --- a/contrib/lizard/lib/README.md +++ /dev/null @@ -1,76 +0,0 @@ -Lizard - Library Files -================================ - -The __lib__ directory contains several directories. -Depending on target use case, it's enough to include only files from relevant directories. - - -#### API - -Lizard stable API is exposed within [lizard_compress.h](lizard_compress.h) and [lizard_decompress.h](lizard_decompress.h), -at the root of `lib` directory. - - -#### Compatibility issues - -The raw Lizard block compression format is detailed within [lizard_Block_format]. -To compress an arbitrarily long file or data stream, multiple blocks are required. -Organizing these blocks and providing a common header format to handle their content -is the purpose of the Frame format, defined in [lizard_Frame_format]. -`lizard` command line utility produces files or streams compatible with the Frame format. -(_Advanced stuff_ : It's possible to hide xxhash symbols into a local namespace. -This is what `liblizard` does, to avoid symbol duplication -in case a user program would link to several libraries containing xxhash symbols.) - -[lizard_Block_format]: ../doc/lizard_Block_format.md -[lizard_Frame_format]: ../doc/lizard_Frame_format.md - - -#### Various Lizard builds - -Files `lizard_common.h`, `lizard_compress*`, `lizard_parser_*.h`, `lizard_decompress*`, and `entropy\mem.h` are required in all circumstances. - -To compile: -- Lizard_raw only with levels 10...29 : use the `-DLIZARD_NO_HUFFMAN` compiler flag -- Lizard_raw with levels 10...49 : include also all files from `entropy` directory -- Lizard_frame with levels 10...49 : `lizard_frame*` and all files from `entropy` and `xxhash` directories - - -#### Advanced API - -A more complex `lizard_frame_static.h` is also provided. -It contains definitions which are not guaranteed to remain stable within future versions. -It must be used with static linking ***only***. - - -#### Using MinGW+MSYS to create DLL - -DLL can be created using MinGW+MSYS with the `make liblizard` command. -This command creates `dll\liblizard.dll` and the import library `dll\liblizard.lib`. -The import library is only required with Visual C++. -The header files `lizard.h`, `lizardhc.h`, `lizard_frame.h` and the dynamic library -`dll\liblizard.dll` are required to compile a project using gcc/MinGW. -The dynamic library has to be added to linking options. -It means that if a project that uses Lizard consists of a single `test-dll.c` -file it should be compiled with `liblizard.lib`. For example: -``` - gcc $(CFLAGS) -Iinclude/ test-dll.c -o test-dll dll\liblizard.dll -``` -The compiled executable will require Lizard DLL which is available at `dll\liblizard.dll`. - - -#### Miscellaneous - -Other files present in the directory are not source code. There are : - - - LICENSE : contains the BSD license text - - Makefile : script to compile or install lizard library (static or dynamic) - - liblizard.pc.in : for pkg-config (make install) - - README.md : this file - - -#### License - -All source material within __lib__ directory are BSD 2-Clause licensed. -See [LICENSE](LICENSE) for details. -The license is also repeated at the top of each source file. diff --git a/contrib/lizard/lib/dll/liblizard.def b/contrib/lizard/lib/dll/liblizard.def deleted file mode 100644 index 73ebc44570a..00000000000 --- a/contrib/lizard/lib/dll/liblizard.def +++ /dev/null @@ -1,19 +0,0 @@ -LIBRARY liblizard.dll -EXPORTS - Lizard_compress - Lizard_compressBound - Lizard_compress_continue - Lizard_compress_extState - Lizard_createStream - Lizard_createStreamDecode - Lizard_decompress_safe - Lizard_decompress_safe_continue - Lizard_decompress_safe_partial - Lizard_decompress_safe_usingDict - Lizard_freeStream - Lizard_freeStreamDecode - Lizard_loadDict - Lizard_resetStream - Lizard_saveDict - Lizard_setStreamDecode - Lizard_sizeofState diff --git a/contrib/lizard/lib/entropy/README.md b/contrib/lizard/lib/entropy/README.md deleted file mode 100644 index 5b18f20b063..00000000000 --- a/contrib/lizard/lib/entropy/README.md +++ /dev/null @@ -1,38 +0,0 @@ -New Generation Entropy library -============================== - -The __lib__ directory contains several files, but you don't necessarily want them all. -Here is a detailed list, to help you decide which one you need : - - -#### Compulsory files - -These files are required in all circumstances : -- __error_public.h__ : error list as enum -- __error_private.h__ : error management -- __mem.h__ : low level memory access routines -- __bitstream.h__ : generic read/write bitstream common to all entropy codecs -- __entropy_common.c__ : common functions needed for both compression and decompression - - -#### Finite State Entropy - -This is the base codec required by other ones. -It implements a tANS variant, similar to arithmetic in compression performance, but much faster. Compression and decompression can be compiled independently. -- __fse.h__ : exposes interfaces -- __fse_compress.c__ : implements compression codec -- __fse_decompress.c__ : implements decompression codec - - -#### FSE 16-bits symbols version - -This codec is able to encode alphabets of size > 256, using 2 bytes per symbol. It requires the base FSE codec to compile properly. Compression and decompression are merged in the same file. -- __fseU16.c__ implements the codec, while __fseU16.h__ exposes its interfaces. - - -#### Huffman codec - -This is the fast huffman codec. It requires the base FSE codec to compress its headers. Compression and decompression can be compiled independently. -- __huf.h__ : exposes interfaces. -- __huf_compress.c__ : implements compression codec -- __huf_decompress.c__ : implements decompression codec diff --git a/contrib/lizard/lib/entropy/bitstream.h b/contrib/lizard/lib/entropy/bitstream.h deleted file mode 100644 index 849a9e6018a..00000000000 --- a/contrib/lizard/lib/entropy/bitstream.h +++ /dev/null @@ -1,414 +0,0 @@ -/* ****************************************************************** - bitstream - Part of FSE library - header file (to include) - Copyright (C) 2013-2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Source repository : https://github.com/Cyan4973/FiniteStateEntropy -****************************************************************** */ -#ifndef BITSTREAM_H_MODULE -#define BITSTREAM_H_MODULE - -#if defined (__cplusplus) -extern "C" { -#endif - - -/* -* This API consists of small unitary functions, which must be inlined for best performance. -* Since link-time-optimization is not available for all compilers, -* these functions are defined into a .h to be included. -*/ - -/*-**************************************** -* Dependencies -******************************************/ -#include "mem.h" /* unaligned access routines */ -#include "error_private.h" /* error codes and messages */ - - -/*========================================= -* Target specific -=========================================*/ -#if defined(__BMI__) && defined(__GNUC__) -# include /* support for bextr (experimental) */ -#endif - - -/*-****************************************** -* bitStream encoding API (write forward) -********************************************/ -/* bitStream can mix input from multiple sources. -* A critical property of these streams is that they encode and decode in **reverse** direction. -* So the first bit sequence you add will be the last to be read, like a LIFO stack. -*/ -typedef struct -{ - size_t bitContainer; - int bitPos; - char* startPtr; - char* ptr; - char* endPtr; -} BIT_CStream_t; - -MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* dstBuffer, size_t dstCapacity); -MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits); -MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC); -MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC); - -/* Start with initCStream, providing the size of buffer to write into. -* bitStream will never write outside of this buffer. -* `dstCapacity` must be >= sizeof(bitD->bitContainer), otherwise @return will be an error code. -* -* bits are first added to a local register. -* Local register is size_t, hence 64-bits on 64-bits systems, or 32-bits on 32-bits systems. -* Writing data into memory is an explicit operation, performed by the flushBits function. -* Hence keep track how many bits are potentially stored into local register to avoid register overflow. -* After a flushBits, a maximum of 7 bits might still be stored into local register. -* -* Avoid storing elements of more than 24 bits if you want compatibility with 32-bits bitstream readers. -* -* Last operation is to close the bitStream. -* The function returns the final size of CStream in bytes. -* If data couldn't fit into `dstBuffer`, it will return a 0 ( == not storable) -*/ - - -/*-******************************************** -* bitStream decoding API (read backward) -**********************************************/ -typedef struct -{ - size_t bitContainer; - unsigned bitsConsumed; - const char* ptr; - const char* start; -} BIT_DStream_t; - -typedef enum { BIT_DStream_unfinished = 0, - BIT_DStream_endOfBuffer = 1, - BIT_DStream_completed = 2, - BIT_DStream_overflow = 3 } BIT_DStream_status; /* result of BIT_reloadDStream() */ - /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */ - -MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize); -MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits); -MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD); -MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD); - - -/* Start by invoking BIT_initDStream(). -* A chunk of the bitStream is then stored into a local register. -* Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t). -* You can then retrieve bitFields stored into the local register, **in reverse order**. -* Local register is explicitly reloaded from memory by the BIT_reloadDStream() method. -* A reload guarantee a minimum of ((8*sizeof(bitD->bitContainer))-7) bits when its result is BIT_DStream_unfinished. -* Otherwise, it can be less than that, so proceed accordingly. -* Checking if DStream has reached its end can be performed with BIT_endOfDStream(). -*/ - - -/*-**************************************** -* unsafe API -******************************************/ -MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC, size_t value, unsigned nbBits); -/* faster, but works only if value is "clean", meaning all high bits above nbBits are 0 */ - -MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC); -/* unsafe version; does not check buffer overflow */ - -MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits); -/* faster, but works only if nbBits >= 1 */ - - - -/*-************************************************************** -* Internal functions -****************************************************************/ -MEM_STATIC unsigned BIT_highbit32 (register U32 val) -{ -# if defined(_MSC_VER) /* Visual */ - unsigned long r=0; - _BitScanReverse ( &r, val ); - return (unsigned) r; -# elif defined(__GNUC__) && (__GNUC__ >= 3) /* Use GCC Intrinsic */ - return 31 - __builtin_clz (val); -# else /* Software version */ - static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 }; - U32 v = val; - v |= v >> 1; - v |= v >> 2; - v |= v >> 4; - v |= v >> 8; - v |= v >> 16; - return DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27]; -# endif -} - -/*===== Local Constants =====*/ -static const unsigned BIT_mask[] = { 0, 1, 3, 7, 0xF, 0x1F, 0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF, 0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0x1FFFF, 0x3FFFF, 0x7FFFF, 0xFFFFF, 0x1FFFFF, 0x3FFFFF, 0x7FFFFF, 0xFFFFFF, 0x1FFFFFF, 0x3FFFFFF }; /* up to 26 bits */ - - -/*-************************************************************** -* bitStream encoding -****************************************************************/ -/*! BIT_initCStream() : - * `dstCapacity` must be > sizeof(void*) - * @return : 0 if success, - otherwise an error code (can be tested using ERR_isError() ) */ -MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* startPtr, size_t dstCapacity) -{ - bitC->bitContainer = 0; - bitC->bitPos = 0; - bitC->startPtr = (char*)startPtr; - bitC->ptr = bitC->startPtr; - bitC->endPtr = bitC->startPtr + dstCapacity - sizeof(bitC->ptr); - if (dstCapacity <= sizeof(bitC->ptr)) return ERROR(dstSize_tooSmall); - return 0; -} - -/*! BIT_addBits() : - can add up to 26 bits into `bitC`. - Does not check for register overflow ! */ -MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits) -{ - bitC->bitContainer |= (value & BIT_mask[nbBits]) << bitC->bitPos; - bitC->bitPos += nbBits; -} - -/*! BIT_addBitsFast() : - * works only if `value` is _clean_, meaning all high bits above nbBits are 0 */ -MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC, size_t value, unsigned nbBits) -{ - bitC->bitContainer |= value << bitC->bitPos; - bitC->bitPos += nbBits; -} - -/*! BIT_flushBitsFast() : - * unsafe version; does not check buffer overflow */ -MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC) -{ - size_t const nbBytes = bitC->bitPos >> 3; - MEM_writeLEST(bitC->ptr, bitC->bitContainer); - bitC->ptr += nbBytes; - bitC->bitPos &= 7; - bitC->bitContainer >>= nbBytes*8; /* if bitPos >= sizeof(bitContainer)*8 --> undefined behavior */ -} - -/*! BIT_flushBits() : - * safe version; check for buffer overflow, and prevents it. - * note : does not signal buffer overflow. This will be revealed later on using BIT_closeCStream() */ -MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC) -{ - size_t const nbBytes = bitC->bitPos >> 3; - MEM_writeLEST(bitC->ptr, bitC->bitContainer); - bitC->ptr += nbBytes; - if (bitC->ptr > bitC->endPtr) bitC->ptr = bitC->endPtr; - bitC->bitPos &= 7; - bitC->bitContainer >>= nbBytes*8; /* if bitPos >= sizeof(bitContainer)*8 --> undefined behavior */ -} - -/*! BIT_closeCStream() : - * @return : size of CStream, in bytes, - or 0 if it could not fit into dstBuffer */ -MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC) -{ - BIT_addBitsFast(bitC, 1, 1); /* endMark */ - BIT_flushBits(bitC); - - if (bitC->ptr >= bitC->endPtr) return 0; /* doesn't fit within authorized budget : cancel */ - - return (bitC->ptr - bitC->startPtr) + (bitC->bitPos > 0); -} - - -/*-******************************************************** -* bitStream decoding -**********************************************************/ -/*! BIT_initDStream() : -* Initialize a BIT_DStream_t. -* `bitD` : a pointer to an already allocated BIT_DStream_t structure. -* `srcSize` must be the *exact* size of the bitStream, in bytes. -* @return : size of stream (== srcSize) or an errorCode if a problem is detected -*/ -MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize) -{ - if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); } - - if (srcSize >= sizeof(bitD->bitContainer)) { /* normal case */ - bitD->start = (const char*)srcBuffer; - bitD->ptr = (const char*)srcBuffer + srcSize - sizeof(bitD->bitContainer); - bitD->bitContainer = MEM_readLEST(bitD->ptr); - { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1]; - bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; /* ensures bitsConsumed is always set */ - if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ } - } else { - bitD->start = (const char*)srcBuffer; - bitD->ptr = bitD->start; - bitD->bitContainer = *(const BYTE*)(bitD->start); - switch(srcSize) - { - case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16); // fallthrough - case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24); // fallthrough - case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32); // fallthrough - case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24; // fallthrough - case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16; // fallthrough - case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) << 8; // fallthrough - default:; - } - { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1]; - bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; - if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ } - bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8; - } - - return srcSize; -} - -MEM_STATIC size_t BIT_getUpperBits(size_t bitContainer, U32 const start) -{ - return bitContainer >> start; -} - -MEM_STATIC size_t BIT_getMiddleBits(size_t bitContainer, U32 const start, U32 const nbBits) -{ -#if defined(__BMI__) && defined(__GNUC__) && __GNUC__*1000+__GNUC_MINOR__ >= 4008 /* experimental */ -# if defined(__x86_64__) - if (sizeof(bitContainer)==8) - return _bextr_u64(bitContainer, start, nbBits); - else -# endif - return _bextr_u32(bitContainer, start, nbBits); -#else - return (bitContainer >> start) & BIT_mask[nbBits]; -#endif -} - -MEM_STATIC size_t BIT_getLowerBits(size_t bitContainer, U32 const nbBits) -{ - return bitContainer & BIT_mask[nbBits]; -} - -/*! BIT_lookBits() : - * Provides next n bits from local register. - * local register is not modified. - * On 32-bits, maxNbBits==24. - * On 64-bits, maxNbBits==56. - * @return : value extracted - */ - MEM_STATIC size_t BIT_lookBits(const BIT_DStream_t* bitD, U32 nbBits) -{ -#if defined(__BMI__) && defined(__GNUC__) /* experimental; fails if bitD->bitsConsumed + nbBits > sizeof(bitD->bitContainer)*8 */ - return BIT_getMiddleBits(bitD->bitContainer, (sizeof(bitD->bitContainer)*8) - bitD->bitsConsumed - nbBits, nbBits); -#else - U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1; - return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask); -#endif -} - -/*! BIT_lookBitsFast() : -* unsafe version; only works only if nbBits >= 1 */ -MEM_STATIC size_t BIT_lookBitsFast(const BIT_DStream_t* bitD, U32 nbBits) -{ - U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1; - return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask); -} - -MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits) -{ - bitD->bitsConsumed += nbBits; -} - -/*! BIT_readBits() : - * Read (consume) next n bits from local register and update. - * Pay attention to not read more than nbBits contained into local register. - * @return : extracted value. - */ -MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits) -{ - size_t const value = BIT_lookBits(bitD, nbBits); - BIT_skipBits(bitD, nbBits); - return value; -} - -/*! BIT_readBitsFast() : -* unsafe version; only works only if nbBits >= 1 */ -MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits) -{ - size_t const value = BIT_lookBitsFast(bitD, nbBits); - BIT_skipBits(bitD, nbBits); - return value; -} - -/*! BIT_reloadDStream() : -* Refill `bitD` from buffer previously set in BIT_initDStream() . -* This function is safe, it guarantees it will not read beyond src buffer. -* @return : status of `BIT_DStream_t` internal register. - if status == BIT_DStream_unfinished, internal register is filled with >= (sizeof(bitD->bitContainer)*8 - 7) bits */ -MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD) -{ - if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8)) /* should not happen => corruption detected */ - return BIT_DStream_overflow; - - if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer)) { - bitD->ptr -= bitD->bitsConsumed >> 3; - bitD->bitsConsumed &= 7; - bitD->bitContainer = MEM_readLEST(bitD->ptr); - return BIT_DStream_unfinished; - } - if (bitD->ptr == bitD->start) { - if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer; - return BIT_DStream_completed; - } - { U32 nbBytes = bitD->bitsConsumed >> 3; - BIT_DStream_status result = BIT_DStream_unfinished; - if (bitD->ptr - nbBytes < bitD->start) { - nbBytes = (U32)(bitD->ptr - bitD->start); /* ptr > start */ - result = BIT_DStream_endOfBuffer; - } - bitD->ptr -= nbBytes; - bitD->bitsConsumed -= nbBytes*8; - bitD->bitContainer = MEM_readLEST(bitD->ptr); /* reminder : srcSize > sizeof(bitD) */ - return result; - } -} - -/*! BIT_endOfDStream() : -* @return Tells if DStream has exactly reached its end (all bits consumed). -*/ -MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream) -{ - return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8)); -} - -#if defined (__cplusplus) -} -#endif - -#endif /* BITSTREAM_H_MODULE */ diff --git a/contrib/lizard/lib/entropy/entropy_common.c b/contrib/lizard/lib/entropy/entropy_common.c deleted file mode 100644 index 72bc398da38..00000000000 --- a/contrib/lizard/lib/entropy/entropy_common.c +++ /dev/null @@ -1,231 +0,0 @@ -/* - Common functions of New Generation Entropy library - Copyright (C) 2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy - - Public forum : https://groups.google.com/forum/#!forum/lz4c -*************************************************************************** */ - -/* ************************************* -* Dependencies -***************************************/ -#include "mem.h" -#include "error_private.h" /* ERR_*, ERROR */ -#define FSE_STATIC_LINKING_ONLY /* FSE_MIN_TABLELOG */ -#include "fse.h" -#define HUF_STATIC_LINKING_ONLY /* HUF_TABLELOG_ABSOLUTEMAX */ -#include "huf.h" - - -/*-**************************************** -* Version -******************************************/ -unsigned FSE_versionNumber(void) { return FSE_VERSION_NUMBER; } - - -/*-**************************************** -* FSE Error Management -******************************************/ -unsigned FSE_isError(size_t code) { return ERR_isError(code); } - -const char* FSE_getErrorName(size_t code) { return ERR_getErrorName(code); } - - -/* ************************************************************** -* HUF Error Management -****************************************************************/ -unsigned HUF_isError(size_t code) { return ERR_isError(code); } - -const char* HUF_getErrorName(size_t code) { return ERR_getErrorName(code); } - - -/*-************************************************************** -* FSE NCount encoding-decoding -****************************************************************/ -size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr, - const void* headerBuffer, size_t hbSize) -{ - const BYTE* const istart = (const BYTE*) headerBuffer; - const BYTE* const iend = istart + hbSize; - const BYTE* ip = istart; - int nbBits; - int remaining; - int threshold; - U32 bitStream; - int bitCount; - unsigned charnum = 0; - int previous0 = 0; - - if (hbSize < 4) return ERROR(srcSize_wrong); - bitStream = MEM_readLE32(ip); - nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */ - if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge); - bitStream >>= 4; - bitCount = 4; - *tableLogPtr = nbBits; - remaining = (1<1) & (charnum<=*maxSVPtr)) { - if (previous0) { - unsigned n0 = charnum; - while ((bitStream & 0xFFFF) == 0xFFFF) { - n0 += 24; - if (ip < iend-5) { - ip += 2; - bitStream = MEM_readLE32(ip) >> bitCount; - } else { - bitStream >>= 16; - bitCount += 16; - } } - while ((bitStream & 3) == 3) { - n0 += 3; - bitStream >>= 2; - bitCount += 2; - } - n0 += bitStream & 3; - bitCount += 2; - if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall); - while (charnum < n0) normalizedCounter[charnum++] = 0; - if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) { - ip += bitCount>>3; - bitCount &= 7; - bitStream = MEM_readLE32(ip) >> bitCount; - } else { - bitStream >>= 2; - } } - { int const max = (2*threshold-1) - remaining; - int count; - - if ((bitStream & (threshold-1)) < (U32)max) { - count = bitStream & (threshold-1); - bitCount += nbBits-1; - } else { - count = bitStream & (2*threshold-1); - if (count >= threshold) count -= max; - bitCount += nbBits; - } - - count--; /* extra accuracy */ - remaining -= count < 0 ? -count : count; /* -1 means +1 */ - normalizedCounter[charnum++] = (short)count; - previous0 = !count; - while (remaining < threshold) { - nbBits--; - threshold >>= 1; - } - - if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) { - ip += bitCount>>3; - bitCount &= 7; - } else { - bitCount -= (int)(8 * (iend - 4 - ip)); - ip = iend - 4; - } - bitStream = MEM_readLE32(ip) >> (bitCount & 31); - } } /* while ((remaining>1) & (charnum<=*maxSVPtr)) */ - if (remaining != 1) return ERROR(corruption_detected); - if (bitCount > 32) return ERROR(corruption_detected); - *maxSVPtr = charnum-1; - - ip += (bitCount+7)>>3; - return ip-istart; -} - - -/*! HUF_readStats() : - Read compact Huffman tree, saved by HUF_writeCTable(). - `huffWeight` is destination buffer. - `rankStats` is assumed to be a table of at least HUF_TABLELOG_MAX U32. - @return : size read from `src` , or an error Code . - Note : Needed by HUF_readCTable() and HUF_readDTableX?() . -*/ -size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats, - U32* nbSymbolsPtr, U32* tableLogPtr, - const void* src, size_t srcSize) -{ - U32 weightTotal; - const BYTE* ip = (const BYTE*) src; - size_t iSize; - size_t oSize; - - if (!srcSize) return ERROR(srcSize_wrong); - iSize = ip[0]; - /* memset(huffWeight, 0, hwSize); *//* is not necessary, even though some analyzer complain ... */ - - if (iSize >= 128) { /* special header */ - oSize = iSize - 127; - iSize = ((oSize+1)/2); - if (iSize+1 > srcSize) return ERROR(srcSize_wrong); - if (oSize >= hwSize) return ERROR(corruption_detected); - ip += 1; - { U32 n; - for (n=0; n> 4; - huffWeight[n+1] = ip[n/2] & 15; - } } } - else { /* header compressed with FSE (normal case) */ - FSE_DTable fseWorkspace[FSE_DTABLE_SIZE_U32(6)]; /* 6 is max possible tableLog for HUF header (maybe even 5, to be tested) */ - if (iSize+1 > srcSize) return ERROR(srcSize_wrong); - oSize = FSE_decompress_wksp(huffWeight, hwSize-1, ip+1, iSize, fseWorkspace, 6); /* max (hwSize-1) values decoded, as last one is implied */ - if (FSE_isError(oSize)) return oSize; - } - - /* collect weight stats */ - memset(rankStats, 0, (HUF_TABLELOG_MAX + 1) * sizeof(U32)); - weightTotal = 0; - { U32 n; for (n=0; n= HUF_TABLELOG_MAX) return ERROR(corruption_detected); - rankStats[huffWeight[n]]++; - weightTotal += (1 << huffWeight[n]) >> 1; - } } - if (weightTotal == 0) return ERROR(corruption_detected); - - /* get last non-null symbol weight (implied, total must be 2^n) */ - { U32 const tableLog = BIT_highbit32(weightTotal) + 1; - if (tableLog > HUF_TABLELOG_MAX) return ERROR(corruption_detected); - *tableLogPtr = tableLog; - /* determine last weight */ - { U32 const total = 1 << tableLog; - U32 const rest = total - weightTotal; - U32 const verif = 1 << BIT_highbit32(rest); - U32 const lastWeight = BIT_highbit32(rest) + 1; - if (verif != rest) return ERROR(corruption_detected); /* last value must be a clean power of 2 */ - huffWeight[oSize] = (BYTE)lastWeight; - rankStats[lastWeight]++; - } } - - /* check tree construction validity */ - if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected); /* by construction : at least 2 elts of rank 1, must be even */ - - /* results */ - *nbSymbolsPtr = (U32)(oSize+1); - return iSize+1; -} diff --git a/contrib/lizard/lib/entropy/error_private.h b/contrib/lizard/lib/entropy/error_private.h deleted file mode 100644 index 1340c16bf25..00000000000 --- a/contrib/lizard/lib/entropy/error_private.h +++ /dev/null @@ -1,115 +0,0 @@ -/* ****************************************************************** - Error codes and messages - Copyright (C) 2013-2016, Yann Collet - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Homepage : http://www.zstd.net -****************************************************************** */ -/* Note : this module is expected to remain private, do not expose it */ - -#ifndef ERROR_H_MODULE -#define ERROR_H_MODULE - -#if defined (__cplusplus) -extern "C" { -#endif - - -/* **************************************** -* Dependencies -******************************************/ -#include /* size_t */ -#include "error_public.h" /* enum list */ - - -/* **************************************** -* Compiler-specific -******************************************/ -#if defined(__GNUC__) -# define ERR_STATIC static __attribute__((unused)) -#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) -# define ERR_STATIC static inline -#elif defined(_MSC_VER) -# define ERR_STATIC static __inline -#else -# define ERR_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */ -#endif - - -/*-**************************************** -* Customization (error_public.h) -******************************************/ -typedef FSE_ErrorCode ERR_enum; -#define PREFIX(name) FSE_error_##name - - -/*-**************************************** -* Error codes handling -******************************************/ -#ifdef ERROR -# undef ERROR /* reported already defined on VS 2015 (Rich Geldreich) */ -#endif -#define ERROR(name) ((size_t)-PREFIX(name)) - -ERR_STATIC unsigned ERR_isError(size_t code) { return (code > ERROR(maxCode)); } - -ERR_STATIC ERR_enum ERR_getErrorCode(size_t code) { if (!ERR_isError(code)) return (ERR_enum)0; return (ERR_enum) (0-code); } - - -/*-**************************************** -* Error Strings -******************************************/ - -ERR_STATIC const char* ERR_getErrorString(ERR_enum code) -{ - static const char* notErrorCode = "Unspecified error code"; - switch( code ) - { - case PREFIX(no_error): return "No error detected"; - case PREFIX(GENERIC): return "Error (generic)"; - case PREFIX(dstSize_tooSmall): return "Destination buffer is too small"; - case PREFIX(srcSize_wrong): return "Src size incorrect"; - case PREFIX(corruption_detected): return "Corrupted block detected"; - case PREFIX(tableLog_tooLarge): return "tableLog requires too much memory : unsupported"; - case PREFIX(maxSymbolValue_tooLarge): return "Unsupported max Symbol Value : too large"; - case PREFIX(maxSymbolValue_tooSmall): return "Specified maxSymbolValue is too small"; - case PREFIX(maxCode): - default: return notErrorCode; - } -} - -ERR_STATIC const char* ERR_getErrorName(size_t code) -{ - return ERR_getErrorString(ERR_getErrorCode(code)); -} - -#if defined (__cplusplus) -} -#endif - -#endif /* ERROR_H_MODULE */ diff --git a/contrib/lizard/lib/entropy/error_public.h b/contrib/lizard/lib/entropy/error_public.h deleted file mode 100644 index a8524719889..00000000000 --- a/contrib/lizard/lib/entropy/error_public.h +++ /dev/null @@ -1,64 +0,0 @@ -/* ****************************************************************** - Error codes list - Copyright (C) 2016, Yann Collet - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Source repository : https://github.com/Cyan4973/FiniteStateEntropy - - Public forum : https://groups.google.com/forum/#!forum/lz4c -****************************************************************** */ -#ifndef ERROR_PUBLIC_H_MODULE -#define ERROR_PUBLIC_H_MODULE - -#if defined (__cplusplus) -extern "C" { -#endif - - -/* **************************************** -* error codes list -******************************************/ -typedef enum { - FSE_error_no_error, - FSE_error_GENERIC, - FSE_error_dstSize_tooSmall, - FSE_error_srcSize_wrong, - FSE_error_corruption_detected, - FSE_error_tableLog_tooLarge, - FSE_error_maxSymbolValue_tooLarge, - FSE_error_maxSymbolValue_tooSmall, - FSE_error_maxCode -} FSE_ErrorCode; - -/* note : compare with size_t function results using FSE_getError() */ - - -#if defined (__cplusplus) -} -#endif - -#endif /* ERROR_PUBLIC_H_MODULE */ diff --git a/contrib/lizard/lib/entropy/fse.h b/contrib/lizard/lib/entropy/fse.h deleted file mode 100644 index baac3903267..00000000000 --- a/contrib/lizard/lib/entropy/fse.h +++ /dev/null @@ -1,694 +0,0 @@ -/* ****************************************************************** - FSE : Finite State Entropy codec - Public Prototypes declaration - Copyright (C) 2013-2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Source repository : https://github.com/Cyan4973/FiniteStateEntropy -****************************************************************** */ -#ifndef FSE_H -#define FSE_H - -#if defined (__cplusplus) -extern "C" { -#endif - - -/*-***************************************** -* Dependencies -******************************************/ -#include /* size_t, ptrdiff_t */ - - -/*-***************************************** -* FSE_PUBLIC_API : control library symbols visibility -******************************************/ -#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4) -# define FSE_PUBLIC_API __attribute__ ((visibility ("default"))) -#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) /* Visual expected */ -# define FSE_PUBLIC_API __declspec(dllexport) -#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1) -# define FSE_PUBLIC_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ -#else -# define FSE_PUBLIC_API -#endif - -/*------ Version ------*/ -#define FSE_VERSION_MAJOR 0 -#define FSE_VERSION_MINOR 9 -#define FSE_VERSION_RELEASE 0 - -#define FSE_LIB_VERSION FSE_VERSION_MAJOR.FSE_VERSION_MINOR.FSE_VERSION_RELEASE -#define FSE_QUOTE(str) #str -#define FSE_EXPAND_AND_QUOTE(str) FSE_QUOTE(str) -#define FSE_VERSION_STRING FSE_EXPAND_AND_QUOTE(FSE_LIB_VERSION) - -#define FSE_VERSION_NUMBER (FSE_VERSION_MAJOR *100*100 + FSE_VERSION_MINOR *100 + FSE_VERSION_RELEASE) -FSE_PUBLIC_API unsigned FSE_versionNumber(void); /**< library version number; to be used when checking dll version */ - -/*-**************************************** -* FSE simple functions -******************************************/ -/*! FSE_compress() : - Compress content of buffer 'src', of size 'srcSize', into destination buffer 'dst'. - 'dst' buffer must be already allocated. Compression runs faster is dstCapacity >= FSE_compressBound(srcSize). - @return : size of compressed data (<= dstCapacity). - Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!! - if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression instead. - if FSE_isError(return), compression failed (more details using FSE_getErrorName()) -*/ -FSE_PUBLIC_API size_t FSE_compress(void* dst, size_t dstCapacity, - const void* src, size_t srcSize); - -/*! FSE_decompress(): - Decompress FSE data from buffer 'cSrc', of size 'cSrcSize', - into already allocated destination buffer 'dst', of size 'dstCapacity'. - @return : size of regenerated data (<= maxDstSize), - or an error code, which can be tested using FSE_isError() . - - ** Important ** : FSE_decompress() does not decompress non-compressible nor RLE data !!! - Why ? : making this distinction requires a header. - Header management is intentionally delegated to the user layer, which can better manage special cases. -*/ -FSE_PUBLIC_API size_t FSE_decompress(void* dst, size_t dstCapacity, - const void* cSrc, size_t cSrcSize); - - -/*-***************************************** -* Tool functions -******************************************/ -FSE_PUBLIC_API size_t FSE_compressBound(size_t size); /* maximum compressed size */ - -/* Error Management */ -FSE_PUBLIC_API unsigned FSE_isError(size_t code); /* tells if a return value is an error code */ -FSE_PUBLIC_API const char* FSE_getErrorName(size_t code); /* provides error code string (useful for debugging) */ - - -/*-***************************************** -* FSE advanced functions -******************************************/ -/*! FSE_compress2() : - Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog' - Both parameters can be defined as '0' to mean : use default value - @return : size of compressed data - Special values : if return == 0, srcData is not compressible => Nothing is stored within cSrc !!! - if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression. - if FSE_isError(return), it's an error code. -*/ -FSE_PUBLIC_API size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); - - -/*-***************************************** -* FSE detailed API -******************************************/ -/*! -FSE_compress() does the following: -1. count symbol occurrence from source[] into table count[] -2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog) -3. save normalized counters to memory buffer using writeNCount() -4. build encoding table 'CTable' from normalized counters -5. encode the data stream using encoding table 'CTable' - -FSE_decompress() does the following: -1. read normalized counters with readNCount() -2. build decoding table 'DTable' from normalized counters -3. decode the data stream using decoding table 'DTable' - -The following API allows targeting specific sub-functions for advanced tasks. -For example, it's possible to compress several blocks using the same 'CTable', -or to save and provide normalized distribution using external method. -*/ - -/* *** COMPRESSION *** */ - -/*! FSE_count(): - Provides the precise count of each byte within a table 'count'. - 'count' is a table of unsigned int, of minimum size (*maxSymbolValuePtr+1). - *maxSymbolValuePtr will be updated if detected smaller than initial value. - @return : the count of the most frequent symbol (which is not identified). - if return == srcSize, there is only one symbol. - Can also return an error code, which can be tested with FSE_isError(). */ -FSE_PUBLIC_API size_t FSE_count(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize); - -/*! FSE_optimalTableLog(): - dynamically downsize 'tableLog' when conditions are met. - It saves CPU time, by using smaller tables, while preserving or even improving compression ratio. - @return : recommended tableLog (necessarily <= 'maxTableLog') */ -FSE_PUBLIC_API unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue); - -/*! FSE_normalizeCount(): - normalize counts so that sum(count[]) == Power_of_2 (2^tableLog) - 'normalizedCounter' is a table of short, of minimum size (maxSymbolValue+1). - @return : tableLog, - or an errorCode, which can be tested using FSE_isError() */ -FSE_PUBLIC_API size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t srcSize, unsigned maxSymbolValue); - -/*! FSE_NCountWriteBound(): - Provides the maximum possible size of an FSE normalized table, given 'maxSymbolValue' and 'tableLog'. - Typically useful for allocation purpose. */ -FSE_PUBLIC_API size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog); - -/*! FSE_writeNCount(): - Compactly save 'normalizedCounter' into 'buffer'. - @return : size of the compressed table, - or an errorCode, which can be tested using FSE_isError(). */ -FSE_PUBLIC_API size_t FSE_writeNCount (void* buffer, size_t bufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); - - -/*! Constructor and Destructor of FSE_CTable. - Note that FSE_CTable size depends on 'tableLog' and 'maxSymbolValue' */ -typedef unsigned FSE_CTable; /* don't allocate that. It's only meant to be more restrictive than void* */ -FSE_PUBLIC_API FSE_CTable* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue); -FSE_PUBLIC_API void FSE_freeCTable (FSE_CTable* ct); - -/*! FSE_buildCTable(): - Builds `ct`, which must be already allocated, using FSE_createCTable(). - @return : 0, or an errorCode, which can be tested using FSE_isError() */ -FSE_PUBLIC_API size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); - -/*! FSE_compress_usingCTable(): - Compress `src` using `ct` into `dst` which must be already allocated. - @return : size of compressed data (<= `dstCapacity`), - or 0 if compressed data could not fit into `dst`, - or an errorCode, which can be tested using FSE_isError() */ -FSE_PUBLIC_API size_t FSE_compress_usingCTable (void* dst, size_t dstCapacity, const void* src, size_t srcSize, const FSE_CTable* ct); - -/*! -Tutorial : ----------- -The first step is to count all symbols. FSE_count() does this job very fast. -Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have 'maxSymbolValuePtr[0]+1' cells. -'src' is a table of bytes of size 'srcSize'. All values within 'src' MUST be <= maxSymbolValuePtr[0] -maxSymbolValuePtr[0] will be updated, with its real value (necessarily <= original value) -FSE_count() will return the number of occurrence of the most frequent symbol. -This can be used to know if there is a single symbol within 'src', and to quickly evaluate its compressibility. -If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()). - -The next step is to normalize the frequencies. -FSE_normalizeCount() will ensure that sum of frequencies is == 2 ^'tableLog'. -It also guarantees a minimum of 1 to any Symbol with frequency >= 1. -You can use 'tableLog'==0 to mean "use default tableLog value". -If you are unsure of which tableLog value to use, you can ask FSE_optimalTableLog(), -which will provide the optimal valid tableLog given sourceSize, maxSymbolValue, and a user-defined maximum (0 means "default"). - -The result of FSE_normalizeCount() will be saved into a table, -called 'normalizedCounter', which is a table of signed short. -'normalizedCounter' must be already allocated, and have at least 'maxSymbolValue+1' cells. -The return value is tableLog if everything proceeded as expected. -It is 0 if there is a single symbol within distribution. -If there is an error (ex: invalid tableLog value), the function will return an ErrorCode (which can be tested using FSE_isError()). - -'normalizedCounter' can be saved in a compact manner to a memory area using FSE_writeNCount(). -'buffer' must be already allocated. -For guaranteed success, buffer size must be at least FSE_headerBound(). -The result of the function is the number of bytes written into 'buffer'. -If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError(); ex : buffer size too small). - -'normalizedCounter' can then be used to create the compression table 'CTable'. -The space required by 'CTable' must be already allocated, using FSE_createCTable(). -You can then use FSE_buildCTable() to fill 'CTable'. -If there is an error, both functions will return an ErrorCode (which can be tested using FSE_isError()). - -'CTable' can then be used to compress 'src', with FSE_compress_usingCTable(). -Similar to FSE_count(), the convention is that 'src' is assumed to be a table of char of size 'srcSize' -The function returns the size of compressed data (without header), necessarily <= `dstCapacity`. -If it returns '0', compressed data could not fit into 'dst'. -If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()). -*/ - - -/* *** DECOMPRESSION *** */ - -/*! FSE_readNCount(): - Read compactly saved 'normalizedCounter' from 'rBuffer'. - @return : size read from 'rBuffer', - or an errorCode, which can be tested using FSE_isError(). - maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */ -FSE_PUBLIC_API size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize); - -/*! Constructor and Destructor of FSE_DTable. - Note that its size depends on 'tableLog' */ -typedef unsigned FSE_DTable; /* don't allocate that. It's just a way to be more restrictive than void* */ -FSE_PUBLIC_API FSE_DTable* FSE_createDTable(unsigned tableLog); -FSE_PUBLIC_API void FSE_freeDTable(FSE_DTable* dt); - -/*! FSE_buildDTable(): - Builds 'dt', which must be already allocated, using FSE_createDTable(). - return : 0, or an errorCode, which can be tested using FSE_isError() */ -FSE_PUBLIC_API size_t FSE_buildDTable (FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); - -/*! FSE_decompress_usingDTable(): - Decompress compressed source `cSrc` of size `cSrcSize` using `dt` - into `dst` which must be already allocated. - @return : size of regenerated data (necessarily <= `dstCapacity`), - or an errorCode, which can be tested using FSE_isError() */ -FSE_PUBLIC_API size_t FSE_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt); - -/*! -Tutorial : ----------- -(Note : these functions only decompress FSE-compressed blocks. - If block is uncompressed, use memcpy() instead - If block is a single repeated byte, use memset() instead ) - -The first step is to obtain the normalized frequencies of symbols. -This can be performed by FSE_readNCount() if it was saved using FSE_writeNCount(). -'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short. -In practice, that means it's necessary to know 'maxSymbolValue' beforehand, -or size the table to handle worst case situations (typically 256). -FSE_readNCount() will provide 'tableLog' and 'maxSymbolValue'. -The result of FSE_readNCount() is the number of bytes read from 'rBuffer'. -Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that. -If there is an error, the function will return an error code, which can be tested using FSE_isError(). - -The next step is to build the decompression tables 'FSE_DTable' from 'normalizedCounter'. -This is performed by the function FSE_buildDTable(). -The space required by 'FSE_DTable' must be already allocated using FSE_createDTable(). -If there is an error, the function will return an error code, which can be tested using FSE_isError(). - -`FSE_DTable` can then be used to decompress `cSrc`, with FSE_decompress_usingDTable(). -`cSrcSize` must be strictly correct, otherwise decompression will fail. -FSE_decompress_usingDTable() result will tell how many bytes were regenerated (<=`dstCapacity`). -If there is an error, the function will return an error code, which can be tested using FSE_isError(). (ex: dst buffer too small) -*/ - - -#ifdef FSE_STATIC_LINKING_ONLY - -/* *** Dependency *** */ -#include "bitstream.h" - - -/* ***************************************** -* Static allocation -*******************************************/ -/* FSE buffer bounds */ -#define FSE_NCOUNTBOUND 512 -#define FSE_BLOCKBOUND(size) (size + (size>>7)) -#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size)) /* Macro version, useful for static allocation */ - -/* It is possible to statically allocate FSE CTable/DTable as a table of FSE_CTable/FSE_DTable using below macros */ -#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2)) -#define FSE_DTABLE_SIZE_U32(maxTableLog) (1 + (1<= `1024` unsigned - */ -size_t FSE_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr, - const void* source, size_t sourceSize, unsigned* workSpace); - -/** FSE_countFast() : - * same as FSE_count(), but blindly trusts that all byte values within src are <= *maxSymbolValuePtr - */ -size_t FSE_countFast(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize); - -/* FSE_countFast_wksp() : - * Same as FSE_countFast(), but using an externally provided scratch buffer. - * `workSpace` must be a table of minimum `1024` unsigned - */ -size_t FSE_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize, unsigned* workSpace); - -/*! FSE_count_simple - * Same as FSE_countFast(), but does not use any additional memory (not even on stack). - * This function is unsafe, and will segfault if any value within `src` is `> *maxSymbolValuePtr` (presuming it's also the size of `count`). -*/ -size_t FSE_count_simple(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize); - - - -unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus); -/**< same as FSE_optimalTableLog(), which used `minus==2` */ - -/* FSE_compress_wksp() : - * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`). - * FSE_WKSP_SIZE_U32() provides the minimum size required for `workSpace` as a table of FSE_CTable. - */ -#define FSE_WKSP_SIZE_U32(maxTableLog, maxSymbolValue) ( FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) + (1<<((maxTableLog>2)?(maxTableLog-2):0)) ) -size_t FSE_compress_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); - -size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits); -/**< build a fake FSE_CTable, designed for a flat distribution, where each symbol uses nbBits */ - -size_t FSE_buildCTable_rle (FSE_CTable* ct, unsigned char symbolValue); -/**< build a fake FSE_CTable, designed to compress always the same symbolValue */ - -/* FSE_buildCTable_wksp() : - * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`). - * `wkspSize` must be >= `(1<= BIT_DStream_completed - -When it's done, verify decompression is fully completed, by checking both DStream and the relevant states. -Checking if DStream has reached its end is performed by : - BIT_endOfDStream(&DStream); -Check also the states. There might be some symbols left there, if some high probability ones (>50%) are possible. - FSE_endOfDState(&DState); -*/ - - -/* ***************************************** -* FSE unsafe API -*******************************************/ -static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD); -/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */ - - -/* ***************************************** -* Implementation of inlined functions -*******************************************/ -typedef struct { - int deltaFindState; - U32 deltaNbBits; -} FSE_symbolCompressionTransform; /* total 8 bytes */ - -MEM_STATIC void FSE_initCState(FSE_CState_t* statePtr, const FSE_CTable* ct) -{ - const void* ptr = ct; - const U16* u16ptr = (const U16*) ptr; - const U32 tableLog = MEM_read16(ptr); - statePtr->value = (ptrdiff_t)1<stateTable = u16ptr+2; - statePtr->symbolTT = ((const U32*)ct + 1 + (tableLog ? (1<<(tableLog-1)) : 1)); - statePtr->stateLog = tableLog; -} - - -/*! FSE_initCState2() : -* Same as FSE_initCState(), but the first symbol to include (which will be the last to be read) -* uses the smallest state value possible, saving the cost of this symbol */ -MEM_STATIC void FSE_initCState2(FSE_CState_t* statePtr, const FSE_CTable* ct, U32 symbol) -{ - FSE_initCState(statePtr, ct); - { const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol]; - const U16* stateTable = (const U16*)(statePtr->stateTable); - U32 nbBitsOut = (U32)((symbolTT.deltaNbBits + (1<<15)) >> 16); - statePtr->value = (nbBitsOut << 16) - symbolTT.deltaNbBits; - statePtr->value = stateTable[(statePtr->value >> nbBitsOut) + symbolTT.deltaFindState]; - } -} - -MEM_STATIC void FSE_encodeSymbol(BIT_CStream_t* bitC, FSE_CState_t* statePtr, U32 symbol) -{ - const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol]; - const U16* const stateTable = (const U16*)(statePtr->stateTable); - U32 nbBitsOut = (U32)((statePtr->value + symbolTT.deltaNbBits) >> 16); - BIT_addBits(bitC, statePtr->value, nbBitsOut); - statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT.deltaFindState]; -} - -MEM_STATIC void FSE_flushCState(BIT_CStream_t* bitC, const FSE_CState_t* statePtr) -{ - BIT_addBits(bitC, statePtr->value, statePtr->stateLog); - BIT_flushBits(bitC); -} - - -/* ====== Decompression ====== */ - -typedef struct { - U16 tableLog; - U16 fastMode; -} FSE_DTableHeader; /* sizeof U32 */ - -typedef struct -{ - unsigned short newState; - unsigned char symbol; - unsigned char nbBits; -} FSE_decode_t; /* size == U32 */ - -MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt) -{ - const void* ptr = dt; - const FSE_DTableHeader* const DTableH = (const FSE_DTableHeader*)ptr; - DStatePtr->state = BIT_readBits(bitD, DTableH->tableLog); - BIT_reloadDStream(bitD); - DStatePtr->table = dt + 1; -} - -MEM_STATIC BYTE FSE_peekSymbol(const FSE_DState_t* DStatePtr) -{ - FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; - return DInfo.symbol; -} - -MEM_STATIC void FSE_updateState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD) -{ - FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; - U32 const nbBits = DInfo.nbBits; - size_t const lowBits = BIT_readBits(bitD, nbBits); - DStatePtr->state = DInfo.newState + lowBits; -} - -MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD) -{ - FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; - U32 const nbBits = DInfo.nbBits; - BYTE const symbol = DInfo.symbol; - size_t const lowBits = BIT_readBits(bitD, nbBits); - - DStatePtr->state = DInfo.newState + lowBits; - return symbol; -} - -/*! FSE_decodeSymbolFast() : - unsafe, only works if no symbol has a probability > 50% */ -MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD) -{ - FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; - U32 const nbBits = DInfo.nbBits; - BYTE const symbol = DInfo.symbol; - size_t const lowBits = BIT_readBitsFast(bitD, nbBits); - - DStatePtr->state = DInfo.newState + lowBits; - return symbol; -} - -MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr) -{ - return DStatePtr->state == 0; -} - - - -#ifndef FSE_COMMONDEFS_ONLY - -/* ************************************************************** -* Tuning parameters -****************************************************************/ -/*!MEMORY_USAGE : -* Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.) -* Increasing memory usage improves compression ratio -* Reduced memory usage can improve speed, due to cache effect -* Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */ -#ifndef FSE_MAX_MEMORY_USAGE -# define FSE_MAX_MEMORY_USAGE 14 -#endif -#ifndef FSE_DEFAULT_MEMORY_USAGE -# define FSE_DEFAULT_MEMORY_USAGE 13 -#endif - -/*!FSE_MAX_SYMBOL_VALUE : -* Maximum symbol value authorized. -* Required for proper stack allocation */ -#ifndef FSE_MAX_SYMBOL_VALUE -# define FSE_MAX_SYMBOL_VALUE 255 -#endif - -/* ************************************************************** -* template functions type & suffix -****************************************************************/ -#define FSE_FUNCTION_TYPE BYTE -#define FSE_FUNCTION_EXTENSION -#define FSE_DECODE_TYPE FSE_decode_t - - -#endif /* !FSE_COMMONDEFS_ONLY */ - - -/* *************************************************************** -* Constants -*****************************************************************/ -#define FSE_MAX_TABLELOG (FSE_MAX_MEMORY_USAGE-2) -#define FSE_MAX_TABLESIZE (1U< FSE_TABLELOG_ABSOLUTE_MAX -# error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported" -#endif - -#define FSE_TABLESTEP(tableSize) ((tableSize>>1) + (tableSize>>3) + 3) - - -#endif /* FSE_STATIC_LINKING_ONLY */ - - -#if defined (__cplusplus) -} -#endif - -#endif /* FSE_H */ diff --git a/contrib/lizard/lib/entropy/fse_compress.c b/contrib/lizard/lib/entropy/fse_compress.c deleted file mode 100644 index 337b7a6ffac..00000000000 --- a/contrib/lizard/lib/entropy/fse_compress.c +++ /dev/null @@ -1,848 +0,0 @@ -/* ****************************************************************** - FSE : Finite State Entropy encoder - Copyright (C) 2013-2015, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy - - Public forum : https://groups.google.com/forum/#!forum/lz4c -****************************************************************** */ - -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# include /* For Visual 2005 */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - - -/* ************************************************************** -* Includes -****************************************************************/ -#include /* malloc, free, qsort */ -#include /* memcpy, memset */ -#include /* printf (debug) */ -#include "bitstream.h" -#define FSE_STATIC_LINKING_ONLY -#include "fse.h" - - -/* ************************************************************** -* Error Management -****************************************************************/ -#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ - - -/* ************************************************************** -* Templates -****************************************************************/ -/* - designed to be included - for type-specific functions (template emulation in C) - Objective is to write these functions only once, for improved maintenance -*/ - -/* safety checks */ -#ifndef FSE_FUNCTION_EXTENSION -# error "FSE_FUNCTION_EXTENSION must be defined" -#endif -#ifndef FSE_FUNCTION_TYPE -# error "FSE_FUNCTION_TYPE must be defined" -#endif - -/* Function names */ -#define FSE_CAT(X,Y) X##Y -#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y) -#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y) - - -/* Function templates */ - -/* FSE_buildCTable_wksp() : - * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`). - * wkspSize should be sized to handle worst case situation, which is `1<>1 : 1) ; - FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT); - U32 const step = FSE_TABLESTEP(tableSize); - U32 cumul[FSE_MAX_SYMBOL_VALUE+2]; - - FSE_FUNCTION_TYPE* const tableSymbol = (FSE_FUNCTION_TYPE*)workSpace; - U32 highThreshold = tableSize-1; - - /* CTable header */ - if (((size_t)1 << tableLog) * sizeof(FSE_FUNCTION_TYPE) > wkspSize) return ERROR(tableLog_tooLarge); - tableU16[-2] = (U16) tableLog; - tableU16[-1] = (U16) maxSymbolValue; - - /* For explanations on how to distribute symbol values over the table : - * http://fastcompression.blogspot.fr/2014/02/fse-distributing-symbol-values.html */ - - /* symbol start positions */ - { U32 u; - cumul[0] = 0; - for (u=1; u<=maxSymbolValue+1; u++) { - if (normalizedCounter[u-1]==-1) { /* Low proba symbol */ - cumul[u] = cumul[u-1] + 1; - tableSymbol[highThreshold--] = (FSE_FUNCTION_TYPE)(u-1); - } else { - cumul[u] = cumul[u-1] + normalizedCounter[u-1]; - } } - cumul[maxSymbolValue+1] = tableSize+1; - } - - /* Spread symbols */ - { U32 position = 0; - U32 symbol; - for (symbol=0; symbol<=maxSymbolValue; symbol++) { - int nbOccurences; - for (nbOccurences=0; nbOccurences highThreshold) position = (position + step) & tableMask; /* Low proba area */ - } } - - if (position!=0) return ERROR(GENERIC); /* Must have gone through all positions */ - } - - /* Build table */ - { U32 u; for (u=0; u> 3) + 3; - return maxSymbolValue ? maxHeaderSize : FSE_NCOUNTBOUND; /* maxSymbolValue==0 ? use default */ -} - -static size_t FSE_writeNCount_generic (void* header, size_t headerBufferSize, - const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog, - unsigned writeIsSafe) -{ - BYTE* const ostart = (BYTE*) header; - BYTE* out = ostart; - BYTE* const oend = ostart + headerBufferSize; - int nbBits; - const int tableSize = 1 << tableLog; - int remaining; - int threshold; - U32 bitStream; - int bitCount; - unsigned charnum = 0; - int previous0 = 0; - - bitStream = 0; - bitCount = 0; - /* Table Size */ - bitStream += (tableLog-FSE_MIN_TABLELOG) << bitCount; - bitCount += 4; - - /* Init */ - remaining = tableSize+1; /* +1 for extra accuracy */ - threshold = tableSize; - nbBits = tableLog+1; - - while (remaining>1) { /* stops at 1 */ - if (previous0) { - unsigned start = charnum; - while (!normalizedCounter[charnum]) charnum++; - while (charnum >= start+24) { - start+=24; - bitStream += 0xFFFFU << bitCount; - if ((!writeIsSafe) && (out > oend-2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ - out[0] = (BYTE) bitStream; - out[1] = (BYTE)(bitStream>>8); - out+=2; - bitStream>>=16; - } - while (charnum >= start+3) { - start+=3; - bitStream += 3 << bitCount; - bitCount += 2; - } - bitStream += (charnum-start) << bitCount; - bitCount += 2; - if (bitCount>16) { - if ((!writeIsSafe) && (out > oend - 2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ - out[0] = (BYTE)bitStream; - out[1] = (BYTE)(bitStream>>8); - out += 2; - bitStream >>= 16; - bitCount -= 16; - } } - { int count = normalizedCounter[charnum++]; - int const max = (2*threshold-1)-remaining; - remaining -= count < 0 ? -count : count; - count++; /* +1 for extra accuracy */ - if (count>=threshold) count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */ - bitStream += count << bitCount; - bitCount += nbBits; - bitCount -= (count>=1; - } - if (bitCount>16) { - if ((!writeIsSafe) && (out > oend - 2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ - out[0] = (BYTE)bitStream; - out[1] = (BYTE)(bitStream>>8); - out += 2; - bitStream >>= 16; - bitCount -= 16; - } } - - /* flush remaining bitStream */ - if ((!writeIsSafe) && (out > oend - 2)) return ERROR(dstSize_tooSmall); /* Buffer overflow */ - out[0] = (BYTE)bitStream; - out[1] = (BYTE)(bitStream>>8); - out+= (bitCount+7) /8; - - if (charnum > maxSymbolValue + 1) return ERROR(GENERIC); - - return (out-ostart); -} - - -size_t FSE_writeNCount (void* buffer, size_t bufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog) -{ - if (tableLog > FSE_MAX_TABLELOG) return ERROR(GENERIC); /* Unsupported */ - if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC); /* Unsupported */ - - if (bufferSize < FSE_NCountWriteBound(maxSymbolValue, tableLog)) - return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 0); - - return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 1); -} - - - -/*-************************************************************** -* Counting histogram -****************************************************************/ -/*! FSE_count_simple - This function counts byte values within `src`, and store the histogram into table `count`. - It doesn't use any additional memory. - But this function is unsafe : it doesn't check that all values within `src` can fit into `count`. - For this reason, prefer using a table `count` with 256 elements. - @return : count of most numerous element -*/ -size_t FSE_count_simple(unsigned* count, unsigned* maxSymbolValuePtr, - const void* src, size_t srcSize) -{ - const BYTE* ip = (const BYTE*)src; - const BYTE* const end = ip + srcSize; - unsigned maxSymbolValue = *maxSymbolValuePtr; - unsigned max=0; - - memset(count, 0, (maxSymbolValue+1)*sizeof(*count)); - if (srcSize==0) { *maxSymbolValuePtr = 0; return 0; } - - while (ip max) max = count[s]; } - - return (size_t)max; -} - - -/* FSE_count_parallel_wksp() : - * Same as FSE_count_parallel(), but using an externally provided scratch buffer. - * `workSpace` size must be a minimum of `1024 * sizeof(unsigned)`` */ -static size_t FSE_count_parallel_wksp( - unsigned* count, unsigned* maxSymbolValuePtr, - const void* source, size_t sourceSize, - unsigned checkMax, unsigned* const workSpace) -{ - const BYTE* ip = (const BYTE*)source; - const BYTE* const iend = ip+sourceSize; - unsigned maxSymbolValue = *maxSymbolValuePtr; - unsigned max=0; - U32* const Counting1 = workSpace; - U32* const Counting2 = Counting1 + 256; - U32* const Counting3 = Counting2 + 256; - U32* const Counting4 = Counting3 + 256; - - memset(Counting1, 0, 4*256*sizeof(unsigned)); - - /* safety checks */ - if (!sourceSize) { - memset(count, 0, maxSymbolValue + 1); - *maxSymbolValuePtr = 0; - return 0; - } - if (!maxSymbolValue) maxSymbolValue = 255; /* 0 == default */ - - /* by stripes of 16 bytes */ - { U32 cached = MEM_read32(ip); ip += 4; - while (ip < iend-15) { - U32 c = cached; cached = MEM_read32(ip); ip += 4; - Counting1[(BYTE) c ]++; - Counting2[(BYTE)(c>>8) ]++; - Counting3[(BYTE)(c>>16)]++; - Counting4[ c>>24 ]++; - c = cached; cached = MEM_read32(ip); ip += 4; - Counting1[(BYTE) c ]++; - Counting2[(BYTE)(c>>8) ]++; - Counting3[(BYTE)(c>>16)]++; - Counting4[ c>>24 ]++; - c = cached; cached = MEM_read32(ip); ip += 4; - Counting1[(BYTE) c ]++; - Counting2[(BYTE)(c>>8) ]++; - Counting3[(BYTE)(c>>16)]++; - Counting4[ c>>24 ]++; - c = cached; cached = MEM_read32(ip); ip += 4; - Counting1[(BYTE) c ]++; - Counting2[(BYTE)(c>>8) ]++; - Counting3[(BYTE)(c>>16)]++; - Counting4[ c>>24 ]++; - } - ip-=4; - } - - /* finish last symbols */ - while (ipmaxSymbolValue; s--) { - Counting1[s] += Counting2[s] + Counting3[s] + Counting4[s]; - if (Counting1[s]) return ERROR(maxSymbolValue_tooSmall); - } } - - { U32 s; for (s=0; s<=maxSymbolValue; s++) { - count[s] = Counting1[s] + Counting2[s] + Counting3[s] + Counting4[s]; - if (count[s] > max) max = count[s]; - } } - - while (!count[maxSymbolValue]) maxSymbolValue--; - *maxSymbolValuePtr = maxSymbolValue; - return (size_t)max; -} - -/* FSE_countFast_wksp() : - * Same as FSE_countFast(), but using an externally provided scratch buffer. - * `workSpace` size must be table of >= `1024` unsigned */ -size_t FSE_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr, - const void* source, size_t sourceSize, unsigned* workSpace) -{ - if (sourceSize < 1500) return FSE_count_simple(count, maxSymbolValuePtr, source, sourceSize); - return FSE_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, 0, workSpace); -} - -/* fast variant (unsafe : won't check if src contains values beyond count[] limit) */ -size_t FSE_countFast(unsigned* count, unsigned* maxSymbolValuePtr, - const void* source, size_t sourceSize) -{ - unsigned tmpCounters[1024]; - return FSE_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, tmpCounters); -} - -/* FSE_count_wksp() : - * Same as FSE_count(), but using an externally provided scratch buffer. - * `workSpace` size must be table of >= `1024` unsigned */ -size_t FSE_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr, - const void* source, size_t sourceSize, unsigned* workSpace) -{ - if (*maxSymbolValuePtr < 255) - return FSE_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, 1, workSpace); - *maxSymbolValuePtr = 255; - return FSE_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, workSpace); -} - -size_t FSE_count(unsigned* count, unsigned* maxSymbolValuePtr, - const void* src, size_t srcSize) -{ - unsigned tmpCounters[1024]; - return FSE_count_wksp(count, maxSymbolValuePtr, src, srcSize, tmpCounters); -} - - - -/*-************************************************************** -* FSE Compression Code -****************************************************************/ -/*! FSE_sizeof_CTable() : - FSE_CTable is a variable size structure which contains : - `U16 tableLog;` - `U16 maxSymbolValue;` - `U16 nextStateNumber[1 << tableLog];` // This size is variable - `FSE_symbolCompressionTransform symbolTT[maxSymbolValue+1];` // This size is variable -Allocation is manual (C standard does not support variable-size structures). -*/ -size_t FSE_sizeof_CTable (unsigned maxSymbolValue, unsigned tableLog) -{ - if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); - return FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32); -} - -FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog) -{ - size_t size; - if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX; - size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32); - return (FSE_CTable*)malloc(size); -} - -void FSE_freeCTable (FSE_CTable* ct) { free(ct); } - -/* provides the minimum logSize to safely represent a distribution */ -static unsigned FSE_minTableLog(size_t srcSize, unsigned maxSymbolValue) -{ - U32 minBitsSrc = BIT_highbit32((U32)(srcSize - 1)) + 1; - U32 minBitsSymbols = BIT_highbit32(maxSymbolValue) + 2; - U32 minBits = minBitsSrc < minBitsSymbols ? minBitsSrc : minBitsSymbols; - return minBits; -} - -unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus) -{ - U32 maxBitsSrc = BIT_highbit32((U32)(srcSize - 1)) - minus; - U32 tableLog = maxTableLog; - U32 minBits = FSE_minTableLog(srcSize, maxSymbolValue); - if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG; - if (maxBitsSrc < tableLog) tableLog = maxBitsSrc; /* Accuracy can be reduced */ - if (minBits > tableLog) tableLog = minBits; /* Need a minimum to safely represent all symbol values */ - if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG; - if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG; - return tableLog; -} - -unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue) -{ - return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 2); -} - - -/* Secondary normalization method. - To be used when primary method fails. */ - -static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count, size_t total, U32 maxSymbolValue) -{ - U32 s; - U32 distributed = 0; - U32 ToDistribute; - - /* Init */ - U32 const lowThreshold = (U32)(total >> tableLog); - U32 lowOne = (U32)((total * 3) >> (tableLog + 1)); - - for (s=0; s<=maxSymbolValue; s++) { - if (count[s] == 0) { - norm[s]=0; - continue; - } - if (count[s] <= lowThreshold) { - norm[s] = -1; - distributed++; - total -= count[s]; - continue; - } - if (count[s] <= lowOne) { - norm[s] = 1; - distributed++; - total -= count[s]; - continue; - } - norm[s]=-2; - } - ToDistribute = (1 << tableLog) - distributed; - - if ((total / ToDistribute) > lowOne) { - /* risk of rounding to zero */ - lowOne = (U32)((total * 3) / (ToDistribute * 2)); - for (s=0; s<=maxSymbolValue; s++) { - if ((norm[s] == -2) && (count[s] <= lowOne)) { - norm[s] = 1; - distributed++; - total -= count[s]; - continue; - } } - ToDistribute = (1 << tableLog) - distributed; - } - - if (distributed == maxSymbolValue+1) { - /* all values are pretty poor; - probably incompressible data (should have already been detected); - find max, then give all remaining points to max */ - U32 maxV = 0, maxC = 0; - for (s=0; s<=maxSymbolValue; s++) - if (count[s] > maxC) maxV=s, maxC=count[s]; - norm[maxV] += (short)ToDistribute; - return 0; - } - - { U64 const vStepLog = 62 - tableLog; - U64 const mid = (1ULL << (vStepLog-1)) - 1; - U64 const rStep = ((((U64)1<> vStepLog); - U32 const sEnd = (U32)(end >> vStepLog); - U32 const weight = sEnd - sStart; - if (weight < 1) - return ERROR(GENERIC); - norm[s] = (short)weight; - tmpTotal = end; - } } } - - return 0; -} - - -size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog, - const unsigned* count, size_t total, - unsigned maxSymbolValue) -{ - /* Sanity checks */ - if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG; - if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC); /* Unsupported size */ - if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); /* Unsupported size */ - if (tableLog < FSE_minTableLog(total, maxSymbolValue)) return ERROR(GENERIC); /* Too small tableLog, compression potentially impossible */ - - { U32 const rtbTable[] = { 0, 473195, 504333, 520860, 550000, 700000, 750000, 830000 }; - U64 const scale = 62 - tableLog; - U64 const step = ((U64)1<<62) / total; /* <== here, one division ! */ - U64 const vStep = 1ULL<<(scale-20); - int stillToDistribute = 1<> tableLog); - - for (s=0; s<=maxSymbolValue; s++) { - if (count[s] == total) return 0; /* rle special case */ - if (count[s] == 0) { normalizedCounter[s]=0; continue; } - if (count[s] <= lowThreshold) { - normalizedCounter[s] = -1; - stillToDistribute--; - } else { - short proba = (short)((count[s]*step) >> scale); - if (proba<8) { - U64 restToBeat = vStep * rtbTable[proba]; - proba += (count[s]*step) - ((U64)proba< restToBeat; - } - if (proba > largestP) largestP=proba, largest=s; - normalizedCounter[s] = proba; - stillToDistribute -= proba; - } } - if (-stillToDistribute >= (normalizedCounter[largest] >> 1)) { - /* corner case, need another normalization method */ - size_t const errorCode = FSE_normalizeM2(normalizedCounter, tableLog, count, total, maxSymbolValue); - if (FSE_isError(errorCode)) return errorCode; - } - else normalizedCounter[largest] += (short)stillToDistribute; - } - -#if 0 - { /* Print Table (debug) */ - U32 s; - U32 nTotal = 0; - for (s=0; s<=maxSymbolValue; s++) - printf("%3i: %4i \n", s, normalizedCounter[s]); - for (s=0; s<=maxSymbolValue; s++) - nTotal += abs(normalizedCounter[s]); - if (nTotal != (1U<>1); /* assumption : tableLog >= 1 */ - FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT); - unsigned s; - - /* Sanity checks */ - if (nbBits < 1) return ERROR(GENERIC); /* min size */ - - /* header */ - tableU16[-2] = (U16) nbBits; - tableU16[-1] = (U16) maxSymbolValue; - - /* Build table */ - for (s=0; s FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) { /* test bit 2 */ - FSE_encodeSymbol(&bitC, &CState2, *--ip); - FSE_encodeSymbol(&bitC, &CState1, *--ip); - FSE_FLUSHBITS(&bitC); - } - - /* 2 or 4 encoding per loop */ - while ( ip>istart ) { - - FSE_encodeSymbol(&bitC, &CState2, *--ip); - - if (sizeof(bitC.bitContainer)*8 < FSE_MAX_TABLELOG*2+7 ) /* this test must be static */ - FSE_FLUSHBITS(&bitC); - - FSE_encodeSymbol(&bitC, &CState1, *--ip); - - if (sizeof(bitC.bitContainer)*8 > FSE_MAX_TABLELOG*4+7 ) { /* this test must be static */ - FSE_encodeSymbol(&bitC, &CState2, *--ip); - FSE_encodeSymbol(&bitC, &CState1, *--ip); - } - - FSE_FLUSHBITS(&bitC); - } - - FSE_flushCState(&bitC, &CState2); - FSE_flushCState(&bitC, &CState1); - return BIT_closeCStream(&bitC); -} - -size_t FSE_compress_usingCTable (void* dst, size_t dstSize, - const void* src, size_t srcSize, - const FSE_CTable* ct) -{ - unsigned const fast = (dstSize >= FSE_BLOCKBOUND(srcSize)); - - if (fast) - return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 1); - else - return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 0); -} - - -size_t FSE_compressBound(size_t size) { return FSE_COMPRESSBOUND(size); } - -#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return f -#define CHECK_F(f) { CHECK_V_F(_var_err__, f); } - -/* FSE_compress_wksp() : - * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`). - * `wkspSize` size must be `(1< not compressible */ - if (maxCount < (srcSize >> 7)) return 0; /* Heuristic : not compressible enough */ - } - - tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue); - CHECK_F( FSE_normalizeCount(norm, tableLog, count, srcSize, maxSymbolValue) ); - - /* Write table description header */ - { CHECK_V_F(nc_err, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) ); - op += nc_err; - } - - /* Compress */ - CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, scratchBufferSize) ); - { CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, src, srcSize, CTable) ); - if (cSize == 0) return 0; /* not enough space for compressed data */ - op += cSize; - } - - /* check compressibility */ - if ( (size_t)(op-ostart) >= srcSize-1 ) return 0; - - return op-ostart; -} - -typedef struct { - FSE_CTable CTable_max[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)]; - BYTE scratchBuffer[1 << FSE_MAX_TABLELOG]; -} fseWkspMax_t; - -size_t FSE_compress2 (void* dst, size_t dstCapacity, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog) -{ - fseWkspMax_t scratchBuffer; - FSE_STATIC_ASSERT(sizeof(scratchBuffer) >= FSE_WKSP_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)); /* compilation failures here means scratchBuffer is not large enough */ - if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); - return FSE_compress_wksp(dst, dstCapacity, src, srcSize, maxSymbolValue, tableLog, &scratchBuffer, sizeof(scratchBuffer)); -} - -size_t FSE_compress (void* dst, size_t dstCapacity, const void* src, size_t srcSize) -{ - return FSE_compress2(dst, dstCapacity, src, srcSize, FSE_MAX_SYMBOL_VALUE, FSE_DEFAULT_TABLELOG); -} - - -#endif /* FSE_COMMONDEFS_ONLY */ diff --git a/contrib/lizard/lib/entropy/fse_decompress.c b/contrib/lizard/lib/entropy/fse_decompress.c deleted file mode 100644 index 1479a5e8217..00000000000 --- a/contrib/lizard/lib/entropy/fse_decompress.c +++ /dev/null @@ -1,329 +0,0 @@ -/* ****************************************************************** - FSE : Finite State Entropy decoder - Copyright (C) 2013-2015, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy - - Public forum : https://groups.google.com/forum/#!forum/lz4c -****************************************************************** */ - - -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# include /* For Visual 2005 */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - - -/* ************************************************************** -* Includes -****************************************************************/ -#include /* malloc, free, qsort */ -#include /* memcpy, memset */ -#include /* printf (debug) */ -#include "bitstream.h" -#define FSE_STATIC_LINKING_ONLY -#include "fse.h" - - -/* ************************************************************** -* Error Management -****************************************************************/ -#define FSE_isError ERR_isError -#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ - -/* check and forward error code */ -#define CHECK_F(f) { size_t const e = f; if (FSE_isError(e)) return e; } - - -/* ************************************************************** -* Templates -****************************************************************/ -/* - designed to be included - for type-specific functions (template emulation in C) - Objective is to write these functions only once, for improved maintenance -*/ - -/* safety checks */ -#ifndef FSE_FUNCTION_EXTENSION -# error "FSE_FUNCTION_EXTENSION must be defined" -#endif -#ifndef FSE_FUNCTION_TYPE -# error "FSE_FUNCTION_TYPE must be defined" -#endif - -/* Function names */ -#define FSE_CAT(X,Y) X##Y -#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y) -#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y) - - -/* Function templates */ -FSE_DTable* FSE_createDTable (unsigned tableLog) -{ - if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX; - return (FSE_DTable*)malloc( FSE_DTABLE_SIZE_U32(tableLog) * sizeof (U32) ); -} - -void FSE_freeDTable (FSE_DTable* dt) -{ - free(dt); -} - -size_t FSE_buildDTable(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog) -{ - void* const tdPtr = dt+1; /* because *dt is unsigned, 32-bits aligned on 32-bits */ - FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (tdPtr); - U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1]; - - U32 const maxSV1 = maxSymbolValue + 1; - U32 const tableSize = 1 << tableLog; - U32 highThreshold = tableSize-1; - - /* Sanity Checks */ - if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge); - if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge); - - /* Init, lay down lowprob symbols */ - { FSE_DTableHeader DTableH; - DTableH.tableLog = (U16)tableLog; - DTableH.fastMode = 1; - { S16 const largeLimit= (S16)(1 << (tableLog-1)); - U32 s; - for (s=0; s= largeLimit) DTableH.fastMode=0; - symbolNext[s] = normalizedCounter[s]; - } } } - memcpy(dt, &DTableH, sizeof(DTableH)); - } - - /* Spread symbols */ - { U32 const tableMask = tableSize-1; - U32 const step = FSE_TABLESTEP(tableSize); - U32 s, position = 0; - for (s=0; s highThreshold) position = (position + step) & tableMask; /* lowprob area */ - } } - if (position!=0) return ERROR(GENERIC); /* position must reach all cells once, otherwise normalizedCounter is incorrect */ - } - - /* Build Decoding table */ - { U32 u; - for (u=0; utableLog = 0; - DTableH->fastMode = 0; - - cell->newState = 0; - cell->symbol = symbolValue; - cell->nbBits = 0; - - return 0; -} - - -size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits) -{ - void* ptr = dt; - FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr; - void* dPtr = dt + 1; - FSE_decode_t* const dinfo = (FSE_decode_t*)dPtr; - const unsigned tableSize = 1 << nbBits; - const unsigned tableMask = tableSize - 1; - const unsigned maxSV1 = tableMask+1; - unsigned s; - - /* Sanity checks */ - if (nbBits < 1) return ERROR(GENERIC); /* min size */ - - /* Build Decoding Table */ - DTableH->tableLog = (U16)nbBits; - DTableH->fastMode = 1; - for (s=0; s sizeof(bitD.bitContainer)*8) /* This test must be static */ - BIT_reloadDStream(&bitD); - - op[1] = FSE_GETSYMBOL(&state2); - - if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */ - { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } } - - op[2] = FSE_GETSYMBOL(&state1); - - if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8) /* This test must be static */ - BIT_reloadDStream(&bitD); - - op[3] = FSE_GETSYMBOL(&state2); - } - - /* tail */ - /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */ - while (1) { - if (op>(omax-2)) return ERROR(dstSize_tooSmall); - *op++ = FSE_GETSYMBOL(&state1); - if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) { - *op++ = FSE_GETSYMBOL(&state2); - break; - } - - if (op>(omax-2)) return ERROR(dstSize_tooSmall); - *op++ = FSE_GETSYMBOL(&state2); - if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) { - *op++ = FSE_GETSYMBOL(&state1); - break; - } } - - return op-ostart; -} - - -size_t FSE_decompress_usingDTable(void* dst, size_t originalSize, - const void* cSrc, size_t cSrcSize, - const FSE_DTable* dt) -{ - const void* ptr = dt; - const FSE_DTableHeader* DTableH = (const FSE_DTableHeader*)ptr; - const U32 fastMode = DTableH->fastMode; - - /* select fast mode (static) */ - if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1); - return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0); -} - - -size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog) -{ - const BYTE* const istart = (const BYTE*)cSrc; - const BYTE* ip = istart; - short counting[FSE_MAX_SYMBOL_VALUE+1]; - unsigned tableLog; - unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE; - - /* normal FSE decoding mode */ - size_t const NCountLength = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize); - if (FSE_isError(NCountLength)) return NCountLength; - //if (NCountLength >= cSrcSize) return ERROR(srcSize_wrong); /* too small input size; supposed to be already checked in NCountLength, only remaining case : NCountLength==cSrcSize */ - if (tableLog > maxLog) return ERROR(tableLog_tooLarge); - ip += NCountLength; - cSrcSize -= NCountLength; - - CHECK_F( FSE_buildDTable (workSpace, counting, maxSymbolValue, tableLog) ); - - return FSE_decompress_usingDTable (dst, dstCapacity, ip, cSrcSize, workSpace); /* always return, even if it is an error code */ -} - - -typedef FSE_DTable DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)]; - -size_t FSE_decompress(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize) -{ - DTable_max_t dt; /* Static analyzer seems unable to understand this table will be properly initialized later */ - return FSE_decompress_wksp(dst, dstCapacity, cSrc, cSrcSize, dt, FSE_MAX_TABLELOG); -} - - - -#endif /* FSE_COMMONDEFS_ONLY */ diff --git a/contrib/lizard/lib/entropy/huf.h b/contrib/lizard/lib/entropy/huf.h deleted file mode 100644 index 48c7f831f55..00000000000 --- a/contrib/lizard/lib/entropy/huf.h +++ /dev/null @@ -1,250 +0,0 @@ -/* ****************************************************************** - Huffman coder, part of New Generation Entropy library - header file - Copyright (C) 2013-2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Source repository : https://github.com/Cyan4973/FiniteStateEntropy -****************************************************************** */ -#ifndef HUF_H_298734234 -#define HUF_H_298734234 - -#if defined (__cplusplus) -extern "C" { -#endif - - -/* *** Dependencies *** */ -#include /* size_t */ - - -/*-*** PUBLIC_API : control library symbols visibility *** */ -#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4) -# define HUF_PUBLIC_API __attribute__ ((visibility ("default"))) -#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) /* Visual expected */ -# define HUF_PUBLIC_API __declspec(dllexport) -#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1) -# define HUF_PUBLIC_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ -#else -# define HUF_PUBLIC_API -#endif - - -/* *** simple functions *** */ -/** -HUF_compress() : - Compress content from buffer 'src', of size 'srcSize', into buffer 'dst'. - 'dst' buffer must be already allocated. - Compression runs faster if `dstCapacity` >= HUF_compressBound(srcSize). - `srcSize` must be <= `HUF_BLOCKSIZE_MAX` == 128 KB. - @return : size of compressed data (<= `dstCapacity`). - Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!! - if return == 1, srcData is a single repeated byte symbol (RLE compression). - if HUF_isError(return), compression failed (more details using HUF_getErrorName()) -*/ -HUF_PUBLIC_API size_t HUF_compress(void* dst, size_t dstCapacity, - const void* src, size_t srcSize); - -/** -HUF_decompress() : - Decompress HUF data from buffer 'cSrc', of size 'cSrcSize', - into already allocated buffer 'dst', of minimum size 'dstSize'. - `originalSize` : **must** be the ***exact*** size of original (uncompressed) data. - Note : in contrast with FSE, HUF_decompress can regenerate - RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data, - because it knows size to regenerate. - @return : size of regenerated data (== originalSize), - or an error code, which can be tested using HUF_isError() -*/ -HUF_PUBLIC_API size_t HUF_decompress(void* dst, size_t originalSize, - const void* cSrc, size_t cSrcSize); - - -/* *** Tool functions *** */ -#define HUF_BLOCKSIZE_MAX (128 * 1024) /**< maximum input size for a single block compressed with HUF_compress */ -HUF_PUBLIC_API size_t HUF_compressBound(size_t size); /**< maximum compressed size (worst case) */ - -/* Error Management */ -HUF_PUBLIC_API unsigned HUF_isError(size_t code); /**< tells if a return value is an error code */ -HUF_PUBLIC_API const char* HUF_getErrorName(size_t code); /**< provides error code string (useful for debugging) */ - - -/* *** Advanced function *** */ - -/** HUF_compress2() : - * Same as HUF_compress(), but offers direct control over `maxSymbolValue` and `tableLog` . - * `tableLog` must be `<= HUF_TABLELOG_MAX` . */ -HUF_PUBLIC_API size_t HUF_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); - -/** HUF_compress4X_wksp() : -* Same as HUF_compress2(), but uses externally allocated `workSpace`, which must be a table of >= 1024 unsigned */ -HUF_PUBLIC_API size_t HUF_compress4X_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); /**< `workSpace` must be a table of at least 1024 unsigned */ - - - -#ifdef HUF_STATIC_LINKING_ONLY - -/* *** Dependencies *** */ -#include "mem.h" /* U32 */ - - -/* *** Constants *** */ -#define HUF_TABLELOG_ABSOLUTEMAX 15 /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */ -#define HUF_TABLELOG_MAX 12 /* max configured tableLog (for static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */ -#define HUF_TABLELOG_DEFAULT 11 /* tableLog by default, when not specified */ -#define HUF_SYMBOLVALUE_MAX 255 -#if (HUF_TABLELOG_MAX > HUF_TABLELOG_ABSOLUTEMAX) -# error "HUF_TABLELOG_MAX is too large !" -#endif - - -/* **************************************** -* Static allocation -******************************************/ -/* HUF buffer bounds */ -#define HUF_CTABLEBOUND 129 -#define HUF_BLOCKBOUND(size) (size + (size>>8) + 8) /* only true if incompressible pre-filtered with fast heuristic */ -#define HUF_COMPRESSBOUND(size) (HUF_CTABLEBOUND + HUF_BLOCKBOUND(size)) /* Macro version, useful for static allocation */ - -/* static allocation of HUF's Compression Table */ -#define HUF_CREATE_STATIC_CTABLE(name, maxSymbolValue) \ - U32 name##hb[maxSymbolValue+1]; \ - void* name##hv = &(name##hb); \ - HUF_CElt* name = (HUF_CElt*)(name##hv) /* no final ; */ - -/* static allocation of HUF's DTable */ -typedef U32 HUF_DTable; -#define HUF_DTABLE_SIZE(maxTableLog) (1 + (1<<(maxTableLog))) -#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \ - HUF_DTable DTable[HUF_DTABLE_SIZE((maxTableLog)-1)] = { ((U32)((maxTableLog)-1) * 0x01000001) } -#define HUF_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \ - HUF_DTable DTable[HUF_DTABLE_SIZE(maxTableLog)] = { ((U32)(maxTableLog) * 0x01000001) } - - -/* **************************************** -* Advanced decompression functions -******************************************/ -size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */ -size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */ - -size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< decodes RLE and uncompressed */ -size_t HUF_decompress4X_hufOnly(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< considers RLE and uncompressed as errors */ -size_t HUF_decompress4X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */ -size_t HUF_decompress4X4_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */ - - -/* **************************************** -* HUF detailed API -******************************************/ -/*! -HUF_compress() does the following: -1. count symbol occurrence from source[] into table count[] using FSE_count() -2. (optional) refine tableLog using HUF_optimalTableLog() -3. build Huffman table from count using HUF_buildCTable() -4. save Huffman table to memory buffer using HUF_writeCTable() -5. encode the data stream using HUF_compress4X_usingCTable() - -The following API allows targeting specific sub-functions for advanced tasks. -For example, it's possible to compress several blocks using the same 'CTable', -or to save and regenerate 'CTable' using external methods. -*/ -/* FSE_count() : find it within "fse.h" */ -unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue); -typedef struct HUF_CElt_s HUF_CElt; /* incomplete type */ -size_t HUF_buildCTable (HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue, unsigned maxNbBits); -size_t HUF_writeCTable (void* dst, size_t maxDstSize, const HUF_CElt* CTable, unsigned maxSymbolValue, unsigned huffLog); -size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable); - - -/** HUF_buildCTable_wksp() : - * Same as HUF_buildCTable(), but using externally allocated scratch buffer. - * `workSpace` must be aligned on 4-bytes boundaries, and be at least as large as a table of 1024 unsigned. - */ -size_t HUF_buildCTable_wksp (HUF_CElt* tree, const U32* count, U32 maxSymbolValue, U32 maxNbBits, void* workSpace, size_t wkspSize); - -/*! HUF_readStats() : - Read compact Huffman tree, saved by HUF_writeCTable(). - `huffWeight` is destination buffer. - @return : size read from `src` , or an error Code . - Note : Needed by HUF_readCTable() and HUF_readDTableXn() . */ -size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats, - U32* nbSymbolsPtr, U32* tableLogPtr, - const void* src, size_t srcSize); - -/** HUF_readCTable() : -* Loading a CTable saved with HUF_writeCTable() */ -size_t HUF_readCTable (HUF_CElt* CTable, unsigned maxSymbolValue, const void* src, size_t srcSize); - - -/* -HUF_decompress() does the following: -1. select the decompression algorithm (X2, X4) based on pre-computed heuristics -2. build Huffman table from save, using HUF_readDTableXn() -3. decode 1 or 4 segments in parallel using HUF_decompressSXn_usingDTable -*/ - -/** HUF_selectDecoder() : -* Tells which decoder is likely to decode faster, -* based on a set of pre-determined metrics. -* @return : 0==HUF_decompress4X2, 1==HUF_decompress4X4 . -* Assumption : 0 < cSrcSize < dstSize <= 128 KB */ -U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize); - -size_t HUF_readDTableX2 (HUF_DTable* DTable, const void* src, size_t srcSize); -size_t HUF_readDTableX4 (HUF_DTable* DTable, const void* src, size_t srcSize); - -size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); -size_t HUF_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); -size_t HUF_decompress4X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); - - -/* single stream variants */ - -size_t HUF_compress1X (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); -size_t HUF_compress1X_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize); /**< `workSpace` must be a table of at least 1024 unsigned */ -size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable); - -size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /* single-symbol decoder */ -size_t HUF_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /* double-symbol decoder */ - -size_t HUF_decompress1X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); -size_t HUF_decompress1X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< single-symbol decoder */ -size_t HUF_decompress1X4_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< double-symbols decoder */ - -size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); /**< automatic selection of sing or double symbol decoder, based on DTable */ -size_t HUF_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); -size_t HUF_decompress1X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable); - -#endif /* HUF_STATIC_LINKING_ONLY */ - - -#if defined (__cplusplus) -} -#endif - -#endif /* HUF_H_298734234 */ diff --git a/contrib/lizard/lib/entropy/huf_compress.c b/contrib/lizard/lib/entropy/huf_compress.c deleted file mode 100644 index a47a1a2c761..00000000000 --- a/contrib/lizard/lib/entropy/huf_compress.c +++ /dev/null @@ -1,612 +0,0 @@ -/* ****************************************************************** - Huffman encoder, part of New Generation Entropy library - Copyright (C) 2013-2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy - - Public forum : https://groups.google.com/forum/#!forum/lz4c -****************************************************************** */ - -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -#endif - - -/* ************************************************************** -* Includes -****************************************************************/ -#include /* memcpy, memset */ -#include /* printf (debug) */ -#include "bitstream.h" -#define FSE_STATIC_LINKING_ONLY /* FSE_optimalTableLog_internal */ -#include "fse.h" /* header compression */ -#define HUF_STATIC_LINKING_ONLY -#include "huf.h" - - -/* ************************************************************** -* Error Management -****************************************************************/ -#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ -#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return f -#define CHECK_F(f) { CHECK_V_F(_var_err__, f); } - - -/* ************************************************************** -* Utils -****************************************************************/ -unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue) -{ - return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 1); -} - - -/* ******************************************************* -* HUF : Huffman block compression -*********************************************************/ -/* HUF_compressWeights() : - * Same as FSE_compress(), but dedicated to huff0's weights compression. - * The use case needs much less stack memory. - * Note : all elements within weightTable are supposed to be <= HUF_TABLELOG_MAX. - */ -#define MAX_FSE_TABLELOG_FOR_HUFF_HEADER 6 -size_t HUF_compressWeights (void* dst, size_t dstSize, const void* weightTable, size_t wtSize) -{ - BYTE* const ostart = (BYTE*) dst; - BYTE* op = ostart; - BYTE* const oend = ostart + dstSize; - - U32 maxSymbolValue = HUF_TABLELOG_MAX; - U32 tableLog = MAX_FSE_TABLELOG_FOR_HUFF_HEADER; - - FSE_CTable CTable[FSE_CTABLE_SIZE_U32(MAX_FSE_TABLELOG_FOR_HUFF_HEADER, HUF_TABLELOG_MAX)]; - BYTE scratchBuffer[1< not compressible */ - } - - tableLog = FSE_optimalTableLog(tableLog, wtSize, maxSymbolValue); - CHECK_F( FSE_normalizeCount(norm, tableLog, count, wtSize, maxSymbolValue) ); - - /* Write table description header */ - { CHECK_V_F(hSize, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) ); - op += hSize; - } - - /* Compress */ - CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, sizeof(scratchBuffer)) ); - { CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, weightTable, wtSize, CTable) ); - if (cSize == 0) return 0; /* not enough space for compressed data */ - op += cSize; - } - - return op-ostart; -} - - -struct HUF_CElt_s { - U16 val; - BYTE nbBits; -}; /* typedef'd to HUF_CElt within "huf.h" */ - -/*! HUF_writeCTable() : - `CTable` : huffman tree to save, using huf representation. - @return : size of saved CTable */ -size_t HUF_writeCTable (void* dst, size_t maxDstSize, - const HUF_CElt* CTable, U32 maxSymbolValue, U32 huffLog) -{ - BYTE bitsToWeight[HUF_TABLELOG_MAX + 1]; /* precomputed conversion table */ - BYTE huffWeight[HUF_SYMBOLVALUE_MAX]; - BYTE* op = (BYTE*)dst; - U32 n; - - /* check conditions */ - if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge); - - /* convert to weight */ - bitsToWeight[0] = 0; - for (n=1; n1) & (hSize < maxSymbolValue/2)) { /* FSE compressed */ - op[0] = (BYTE)hSize; - return hSize+1; - } } - - /* write raw values as 4-bits (max : 15) */ - if (maxSymbolValue > (256-128)) return ERROR(GENERIC); /* should not happen : likely means source cannot be compressed */ - if (((maxSymbolValue+1)/2) + 1 > maxDstSize) return ERROR(dstSize_tooSmall); /* not enough space within dst buffer */ - op[0] = (BYTE)(128 /*special case*/ + (maxSymbolValue-1)); - huffWeight[maxSymbolValue] = 0; /* to be sure it doesn't cause msan issue in final combination */ - for (n=0; n HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge); - if (nbSymbols > maxSymbolValue+1) return ERROR(maxSymbolValue_tooSmall); - - /* Prepare base value per rank */ - { U32 n, nextRankStart = 0; - for (n=1; n<=tableLog; n++) { - U32 current = nextRankStart; - nextRankStart += (rankVal[n] << (n-1)); - rankVal[n] = current; - } } - - /* fill nbBits */ - { U32 n; for (n=0; nn=tableLog+1 */ - U16 valPerRank[HUF_TABLELOG_MAX+2] = {0}; - { U32 n; for (n=0; n0; n--) { /* start at n=tablelog <-> w=1 */ - valPerRank[n] = min; /* get starting value within each rank */ - min += nbPerRank[n]; - min >>= 1; - } } - /* assign value within rank, symbol order */ - { U32 n; for (n=0; n<=maxSymbolValue; n++) CTable[n].val = valPerRank[CTable[n].nbBits]++; } - } - - return readSize; -} - - -typedef struct nodeElt_s { - U32 count; - U16 parent; - BYTE byte; - BYTE nbBits; -} nodeElt; - -static U32 HUF_setMaxHeight(nodeElt* huffNode, U32 lastNonNull, U32 maxNbBits) -{ - const U32 largestBits = huffNode[lastNonNull].nbBits; - if (largestBits <= maxNbBits) return largestBits; /* early exit : no elt > maxNbBits */ - - /* there are several too large elements (at least >= 2) */ - { int totalCost = 0; - const U32 baseCost = 1 << (largestBits - maxNbBits); - U32 n = lastNonNull; - - while (huffNode[n].nbBits > maxNbBits) { - totalCost += baseCost - (1 << (largestBits - huffNode[n].nbBits)); - huffNode[n].nbBits = (BYTE)maxNbBits; - n --; - } /* n stops at huffNode[n].nbBits <= maxNbBits */ - while (huffNode[n].nbBits == maxNbBits) n--; /* n end at index of smallest symbol using < maxNbBits */ - - /* renorm totalCost */ - totalCost >>= (largestBits - maxNbBits); /* note : totalCost is necessarily a multiple of baseCost */ - - /* repay normalized cost */ - { U32 const noSymbol = 0xF0F0F0F0; - U32 rankLast[HUF_TABLELOG_MAX+2]; - int pos; - - /* Get pos of last (smallest) symbol per rank */ - memset(rankLast, 0xF0, sizeof(rankLast)); - { U32 currentNbBits = maxNbBits; - for (pos=n ; pos >= 0; pos--) { - if (huffNode[pos].nbBits >= currentNbBits) continue; - currentNbBits = huffNode[pos].nbBits; /* < maxNbBits */ - rankLast[maxNbBits-currentNbBits] = pos; - } } - - while (totalCost > 0) { - U32 nBitsToDecrease = BIT_highbit32(totalCost) + 1; - for ( ; nBitsToDecrease > 1; nBitsToDecrease--) { - U32 highPos = rankLast[nBitsToDecrease]; - U32 lowPos = rankLast[nBitsToDecrease-1]; - if (highPos == noSymbol) continue; - if (lowPos == noSymbol) break; - { U32 const highTotal = huffNode[highPos].count; - U32 const lowTotal = 2 * huffNode[lowPos].count; - if (highTotal <= lowTotal) break; - } } - /* only triggered when no more rank 1 symbol left => find closest one (note : there is necessarily at least one !) */ - while ((nBitsToDecrease<=HUF_TABLELOG_MAX) && (rankLast[nBitsToDecrease] == noSymbol)) /* HUF_MAX_TABLELOG test just to please gcc 5+; but it should not be necessary */ - nBitsToDecrease ++; - totalCost -= 1 << (nBitsToDecrease-1); - if (rankLast[nBitsToDecrease-1] == noSymbol) - rankLast[nBitsToDecrease-1] = rankLast[nBitsToDecrease]; /* this rank is no longer empty */ - huffNode[rankLast[nBitsToDecrease]].nbBits ++; - if (rankLast[nBitsToDecrease] == 0) /* special case, reached largest symbol */ - rankLast[nBitsToDecrease] = noSymbol; - else { - rankLast[nBitsToDecrease]--; - if (huffNode[rankLast[nBitsToDecrease]].nbBits != maxNbBits-nBitsToDecrease) - rankLast[nBitsToDecrease] = noSymbol; /* this rank is now empty */ - } } /* while (totalCost > 0) */ - - while (totalCost < 0) { /* Sometimes, cost correction overshoot */ - if (rankLast[1] == noSymbol) { /* special case : no rank 1 symbol (using maxNbBits-1); let's create one from largest rank 0 (using maxNbBits) */ - while (huffNode[n].nbBits == maxNbBits) n--; - huffNode[n+1].nbBits--; - rankLast[1] = n+1; - totalCost++; - continue; - } - huffNode[ rankLast[1] + 1 ].nbBits--; - rankLast[1]++; - totalCost ++; - } } } /* there are several too large elements (at least >= 2) */ - - return maxNbBits; -} - - -typedef struct { - U32 base; - U32 current; -} rankPos; - -static void HUF_sort(nodeElt* huffNode, const U32* count, U32 maxSymbolValue) -{ - rankPos rank[32]; - U32 n; - - memset(rank, 0, sizeof(rank)); - for (n=0; n<=maxSymbolValue; n++) { - U32 r = BIT_highbit32(count[n] + 1); - rank[r].base ++; - } - for (n=30; n>0; n--) rank[n-1].base += rank[n].base; - for (n=0; n<32; n++) rank[n].current = rank[n].base; - for (n=0; n<=maxSymbolValue; n++) { - U32 const c = count[n]; - U32 const r = BIT_highbit32(c+1) + 1; - U32 pos = rank[r].current++; - while ((pos > rank[r].base) && (c > huffNode[pos-1].count)) huffNode[pos]=huffNode[pos-1], pos--; - huffNode[pos].count = c; - huffNode[pos].byte = (BYTE)n; - } -} - - -/** HUF_buildCTable_wksp() : - * Same as HUF_buildCTable(), but using externally allocated scratch buffer. - * `workSpace` must be aligned on 4-bytes boundaries, and be at least as large as a table of 1024 unsigned. - */ -#define STARTNODE (HUF_SYMBOLVALUE_MAX+1) -typedef nodeElt huffNodeTable[2*HUF_SYMBOLVALUE_MAX+1 +1]; -size_t HUF_buildCTable_wksp (HUF_CElt* tree, const U32* count, U32 maxSymbolValue, U32 maxNbBits, void* workSpace, size_t wkspSize) -{ - nodeElt* const huffNode0 = (nodeElt*)workSpace; - nodeElt* const huffNode = huffNode0+1; - U32 n, nonNullRank; - int lowS, lowN; - U16 nodeNb = STARTNODE; - U32 nodeRoot; - - /* safety checks */ - if (wkspSize < sizeof(huffNodeTable)) return ERROR(GENERIC); /* workSpace is not large enough */ - if (maxNbBits == 0) maxNbBits = HUF_TABLELOG_DEFAULT; - if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(GENERIC); - memset(huffNode0, 0, sizeof(huffNodeTable)); - - /* sort, decreasing order */ - HUF_sort(huffNode, count, maxSymbolValue); - - /* init for parents */ - nonNullRank = maxSymbolValue; - while(huffNode[nonNullRank].count == 0) nonNullRank--; - lowS = nonNullRank; nodeRoot = nodeNb + lowS - 1; lowN = nodeNb; - huffNode[nodeNb].count = huffNode[lowS].count + huffNode[lowS-1].count; - huffNode[lowS].parent = huffNode[lowS-1].parent = nodeNb; - nodeNb++; lowS-=2; - for (n=nodeNb; n<=nodeRoot; n++) huffNode[n].count = (U32)(1U<<30); - huffNode0[0].count = (U32)(1U<<31); /* fake entry, strong barrier */ - - /* create parents */ - while (nodeNb <= nodeRoot) { - U32 n1 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++; - U32 n2 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++; - huffNode[nodeNb].count = huffNode[n1].count + huffNode[n2].count; - huffNode[n1].parent = huffNode[n2].parent = nodeNb; - nodeNb++; - } - - /* distribute weights (unlimited tree height) */ - huffNode[nodeRoot].nbBits = 0; - for (n=nodeRoot-1; n>=STARTNODE; n--) - huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1; - for (n=0; n<=nonNullRank; n++) - huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1; - - /* enforce maxTableLog */ - maxNbBits = HUF_setMaxHeight(huffNode, nonNullRank, maxNbBits); - - /* fill result into tree (val, nbBits) */ - { U16 nbPerRank[HUF_TABLELOG_MAX+1] = {0}; - U16 valPerRank[HUF_TABLELOG_MAX+1] = {0}; - if (maxNbBits > HUF_TABLELOG_MAX) return ERROR(GENERIC); /* check fit into table */ - for (n=0; n<=nonNullRank; n++) - nbPerRank[huffNode[n].nbBits]++; - /* determine stating value per rank */ - { U16 min = 0; - for (n=maxNbBits; n>0; n--) { - valPerRank[n] = min; /* get starting value within each rank */ - min += nbPerRank[n]; - min >>= 1; - } } - for (n=0; n<=maxSymbolValue; n++) - tree[huffNode[n].byte].nbBits = huffNode[n].nbBits; /* push nbBits per symbol, symbol order */ - for (n=0; n<=maxSymbolValue; n++) - tree[n].val = valPerRank[tree[n].nbBits]++; /* assign value within rank, symbol order */ - } - - return maxNbBits; -} - -/** HUF_buildCTable() : - * Note : count is used before tree is written, so they can safely overlap - */ -size_t HUF_buildCTable (HUF_CElt* tree, const U32* count, U32 maxSymbolValue, U32 maxNbBits) -{ - huffNodeTable nodeTable; - return HUF_buildCTable_wksp(tree, count, maxSymbolValue, maxNbBits, nodeTable, sizeof(nodeTable)); -} - -static void HUF_encodeSymbol(BIT_CStream_t* bitCPtr, U32 symbol, const HUF_CElt* CTable) -{ - BIT_addBitsFast(bitCPtr, CTable[symbol].val, CTable[symbol].nbBits); -} - -size_t HUF_compressBound(size_t size) { return HUF_COMPRESSBOUND(size); } - -#define HUF_FLUSHBITS(s) (fast ? BIT_flushBitsFast(s) : BIT_flushBits(s)) - -#define HUF_FLUSHBITS_1(stream) \ - if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*2+7) HUF_FLUSHBITS(stream) - -#define HUF_FLUSHBITS_2(stream) \ - if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*4+7) HUF_FLUSHBITS(stream) - -size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable) -{ - const BYTE* ip = (const BYTE*) src; - BYTE* const ostart = (BYTE*)dst; - BYTE* const oend = ostart + dstSize; - BYTE* op = ostart; - size_t n; - const unsigned fast = (dstSize >= HUF_BLOCKBOUND(srcSize)); - BIT_CStream_t bitC; - - /* init */ - if (dstSize < 8) return 0; /* not enough space to compress */ - { size_t const initErr = BIT_initCStream(&bitC, op, oend-op); - if (HUF_isError(initErr)) return 0; } - - n = srcSize & ~3; /* join to mod 4 */ - switch (srcSize & 3) - { - case 3 : HUF_encodeSymbol(&bitC, ip[n+ 2], CTable); - HUF_FLUSHBITS_2(&bitC); - // fallthrough - case 2 : HUF_encodeSymbol(&bitC, ip[n+ 1], CTable); - HUF_FLUSHBITS_1(&bitC); - // fallthrough - case 1 : HUF_encodeSymbol(&bitC, ip[n+ 0], CTable); - HUF_FLUSHBITS(&bitC); - // fallthrough - case 0 : - default: ; - } - - for (; n>0; n-=4) { /* note : n&3==0 at this stage */ - HUF_encodeSymbol(&bitC, ip[n- 1], CTable); - HUF_FLUSHBITS_1(&bitC); - HUF_encodeSymbol(&bitC, ip[n- 2], CTable); - HUF_FLUSHBITS_2(&bitC); - HUF_encodeSymbol(&bitC, ip[n- 3], CTable); - HUF_FLUSHBITS_1(&bitC); - HUF_encodeSymbol(&bitC, ip[n- 4], CTable); - HUF_FLUSHBITS(&bitC); - } - - return BIT_closeCStream(&bitC); -} - - -size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable) -{ - size_t const segmentSize = (srcSize+3)/4; /* first 3 segments */ - const BYTE* ip = (const BYTE*) src; - const BYTE* const iend = ip + srcSize; - BYTE* const ostart = (BYTE*) dst; - BYTE* const oend = ostart + dstSize; - BYTE* op = ostart; - - if (dstSize < 6 + 1 + 1 + 1 + 8) return 0; /* minimum space to compress successfully */ - if (srcSize < 12) return 0; /* no saving possible : too small input */ - op += 6; /* jumpTable */ - - { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, segmentSize, CTable) ); - if (cSize==0) return 0; - MEM_writeLE16(ostart, (U16)cSize); - op += cSize; - } - - ip += segmentSize; - { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, segmentSize, CTable) ); - if (cSize==0) return 0; - MEM_writeLE16(ostart+2, (U16)cSize); - op += cSize; - } - - ip += segmentSize; - { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, segmentSize, CTable) ); - if (cSize==0) return 0; - MEM_writeLE16(ostart+4, (U16)cSize); - op += cSize; - } - - ip += segmentSize; - { CHECK_V_F(cSize, HUF_compress1X_usingCTable(op, oend-op, ip, iend-ip, CTable) ); - if (cSize==0) return 0; - op += cSize; - } - - return op-ostart; -} - - -/* `workSpace` must a table of at least 1024 unsigned */ -static size_t HUF_compress_internal ( - void* dst, size_t dstSize, - const void* src, size_t srcSize, - unsigned maxSymbolValue, unsigned huffLog, - unsigned singleStream, - void* workSpace, size_t wkspSize) -{ - BYTE* const ostart = (BYTE*)dst; - BYTE* const oend = ostart + dstSize; - BYTE* op = ostart; - - union { - U32 count[HUF_SYMBOLVALUE_MAX+1]; - HUF_CElt CTable[HUF_SYMBOLVALUE_MAX+1]; - } table; /* `count` can overlap with `CTable`; saves 1 KB */ - - /* checks & inits */ - if (wkspSize < sizeof(huffNodeTable)) return ERROR(GENERIC); - if (!srcSize) return 0; /* Uncompressed (note : 1 means rle, so first byte must be correct) */ - if (!dstSize) return 0; /* cannot fit within dst budget */ - if (srcSize > HUF_BLOCKSIZE_MAX) return ERROR(srcSize_wrong); /* current block size limit */ - if (huffLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge); - if (!maxSymbolValue) maxSymbolValue = HUF_SYMBOLVALUE_MAX; - if (!huffLog) huffLog = HUF_TABLELOG_DEFAULT; - - /* Scan input and build symbol stats */ - { CHECK_V_F(largest, FSE_count_wksp (table.count, &maxSymbolValue, (const BYTE*)src, srcSize, (U32*)workSpace) ); - if (largest == srcSize) { *ostart = ((const BYTE*)src)[0]; return 1; } /* single symbol, rle */ - if (largest <= (srcSize >> 7)+1) return 0; /* Fast heuristic : not compressible enough */ - } - - /* Build Huffman Tree */ - huffLog = HUF_optimalTableLog(huffLog, srcSize, maxSymbolValue); - { CHECK_V_F(maxBits, HUF_buildCTable_wksp (table.CTable, table.count, maxSymbolValue, huffLog, workSpace, wkspSize) ); - huffLog = (U32)maxBits; - } - - /* Write table description header */ - { CHECK_V_F(hSize, HUF_writeCTable (op, dstSize, table.CTable, maxSymbolValue, huffLog) ); - if (hSize + 12 >= srcSize) return 0; /* not useful to try compression */ - op += hSize; - } - - /* Compress */ - { size_t const cSize = (singleStream) ? - HUF_compress1X_usingCTable(op, oend - op, src, srcSize, table.CTable) : /* single segment */ - HUF_compress4X_usingCTable(op, oend - op, src, srcSize, table.CTable); - if (HUF_isError(cSize)) return cSize; - if (cSize==0) return 0; /* uncompressible */ - op += cSize; - } - - /* check compressibility */ - if ((size_t)(op-ostart) >= srcSize-1) - return 0; - - return op-ostart; -} - - -size_t HUF_compress1X_wksp (void* dst, size_t dstSize, - const void* src, size_t srcSize, - unsigned maxSymbolValue, unsigned huffLog, - void* workSpace, size_t wkspSize) -{ - return HUF_compress_internal(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, 1 /* single stream */, workSpace, wkspSize); -} - -size_t HUF_compress1X (void* dst, size_t dstSize, - const void* src, size_t srcSize, - unsigned maxSymbolValue, unsigned huffLog) -{ - unsigned workSpace[1024]; - return HUF_compress1X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace)); -} - -size_t HUF_compress4X_wksp (void* dst, size_t dstSize, - const void* src, size_t srcSize, - unsigned maxSymbolValue, unsigned huffLog, - void* workSpace, size_t wkspSize) -{ - return HUF_compress_internal(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, 0 /* 4 streams */, workSpace, wkspSize); -} - -size_t HUF_compress2 (void* dst, size_t dstSize, - const void* src, size_t srcSize, - unsigned maxSymbolValue, unsigned huffLog) -{ - unsigned workSpace[1024]; - return HUF_compress4X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace)); -} - -size_t HUF_compress (void* dst, size_t maxDstSize, const void* src, size_t srcSize) -{ - return HUF_compress2(dst, maxDstSize, src, (U32)srcSize, 255, HUF_TABLELOG_DEFAULT); -} diff --git a/contrib/lizard/lib/entropy/huf_decompress.c b/contrib/lizard/lib/entropy/huf_decompress.c deleted file mode 100644 index a342dfb1e4a..00000000000 --- a/contrib/lizard/lib/entropy/huf_decompress.c +++ /dev/null @@ -1,885 +0,0 @@ -/* ****************************************************************** - Huffman decoder, part of New Generation Entropy library - Copyright (C) 2013-2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy - - Public forum : https://groups.google.com/forum/#!forum/lz4c -****************************************************************** */ - -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) -/* inline is defined */ -#elif defined(_MSC_VER) || defined(__GNUC__) -# define inline __inline -#else -# define inline /* disable inline */ -#endif - -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -#endif - - -/* ************************************************************** -* Dependencies -****************************************************************/ -#include /* memcpy, memset */ -#include "bitstream.h" /* BIT_* */ -#include "fse.h" /* header compression */ -#define HUF_STATIC_LINKING_ONLY -#include "huf.h" - - -/* ************************************************************** -* Error Management -****************************************************************/ -#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ - - -/*-***************************/ -/* generic DTableDesc */ -/*-***************************/ - -typedef struct { BYTE maxTableLog; BYTE tableType; BYTE tableLog; BYTE reserved; } DTableDesc; - -static DTableDesc HUF_getDTableDesc(const HUF_DTable* table) -{ - DTableDesc dtd; - memcpy(&dtd, table, sizeof(dtd)); - return dtd; -} - - -/*-***************************/ -/* single-symbol decoding */ -/*-***************************/ - -typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX2; /* single-symbol decoding */ - -size_t HUF_readDTableX2 (HUF_DTable* DTable, const void* src, size_t srcSize) -{ - BYTE huffWeight[HUF_SYMBOLVALUE_MAX + 1]; - U32 rankVal[HUF_TABLELOG_ABSOLUTEMAX + 1]; /* large enough for values from 0 to 16 */ - U32 tableLog = 0; - U32 nbSymbols = 0; - size_t iSize; - void* const dtPtr = DTable + 1; - HUF_DEltX2* const dt = (HUF_DEltX2*)dtPtr; - - HUF_STATIC_ASSERT(sizeof(DTableDesc) == sizeof(HUF_DTable)); - /* memset(huffWeight, 0, sizeof(huffWeight)); */ /* is not necessary, even though some analyzer complain ... */ - - iSize = HUF_readStats(huffWeight, HUF_SYMBOLVALUE_MAX + 1, rankVal, &nbSymbols, &tableLog, src, srcSize); - if (HUF_isError(iSize)) return iSize; - - /* Table header */ - { DTableDesc dtd = HUF_getDTableDesc(DTable); - if (tableLog > (U32)(dtd.maxTableLog+1)) return ERROR(tableLog_tooLarge); /* DTable too small, huffman tree cannot fit in */ - dtd.tableType = 0; - dtd.tableLog = (BYTE)tableLog; - memcpy(DTable, &dtd, sizeof(dtd)); - } - - /* Prepare ranks */ - { U32 n, nextRankStart = 0; - for (n=1; n> 1; - U32 i; - HUF_DEltX2 D; - D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w); - for (i = rankVal[w]; i < rankVal[w] + length; i++) - dt[i] = D; - rankVal[w] += length; - } } - - return iSize; -} - - -static BYTE HUF_decodeSymbolX2(BIT_DStream_t* Dstream, const HUF_DEltX2* dt, const U32 dtLog) -{ - size_t const val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */ - BYTE const c = dt[val].byte; - BIT_skipBits(Dstream, dt[val].nbBits); - return c; -} - -#define HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \ - *ptr++ = HUF_decodeSymbolX2(DStreamPtr, dt, dtLog) - -#define HUF_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \ - if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \ - HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) - -#define HUF_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \ - if (MEM_64bits()) \ - HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) - -static inline size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog) -{ - BYTE* const pStart = p; - - /* up to 4 symbols at a time */ - while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-4)) { - HUF_DECODE_SYMBOLX2_2(p, bitDPtr); - HUF_DECODE_SYMBOLX2_1(p, bitDPtr); - HUF_DECODE_SYMBOLX2_2(p, bitDPtr); - HUF_DECODE_SYMBOLX2_0(p, bitDPtr); - } - - /* closer to the end */ - while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd)) - HUF_DECODE_SYMBOLX2_0(p, bitDPtr); - - /* no more data to retrieve from bitstream, hence no need to reload */ - while (p < pEnd) - HUF_DECODE_SYMBOLX2_0(p, bitDPtr); - - return pEnd-pStart; -} - -static size_t HUF_decompress1X2_usingDTable_internal( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - BYTE* op = (BYTE*)dst; - BYTE* const oend = op + dstSize; - const void* dtPtr = DTable + 1; - const HUF_DEltX2* const dt = (const HUF_DEltX2*)dtPtr; - BIT_DStream_t bitD; - DTableDesc const dtd = HUF_getDTableDesc(DTable); - U32 const dtLog = dtd.tableLog; - - { size_t const errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize); - if (HUF_isError(errorCode)) return errorCode; } - - HUF_decodeStreamX2(op, &bitD, oend, dt, dtLog); - - /* check */ - if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected); - - return dstSize; -} - -size_t HUF_decompress1X2_usingDTable( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - DTableDesc dtd = HUF_getDTableDesc(DTable); - if (dtd.tableType != 0) return ERROR(GENERIC); - return HUF_decompress1X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); -} - -size_t HUF_decompress1X2_DCtx (HUF_DTable* DCtx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - const BYTE* ip = (const BYTE*) cSrc; - - size_t const hSize = HUF_readDTableX2 (DCtx, cSrc, cSrcSize); - if (HUF_isError(hSize)) return hSize; - if (hSize >= cSrcSize) return ERROR(srcSize_wrong); - ip += hSize; cSrcSize -= hSize; - - return HUF_decompress1X2_usingDTable_internal (dst, dstSize, ip, cSrcSize, DCtx); -} - -size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_TABLELOG_MAX); - return HUF_decompress1X2_DCtx (DTable, dst, dstSize, cSrc, cSrcSize); -} - - -static size_t HUF_decompress4X2_usingDTable_internal( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - /* Check */ - if (cSrcSize < 10) return ERROR(corruption_detected); /* strict minimum : jump table + 1 byte per stream */ - - { const BYTE* const istart = (const BYTE*) cSrc; - BYTE* const ostart = (BYTE*) dst; - BYTE* const oend = ostart + dstSize; - const void* const dtPtr = DTable + 1; - const HUF_DEltX2* const dt = (const HUF_DEltX2*)dtPtr; - - /* Init */ - BIT_DStream_t bitD1; - BIT_DStream_t bitD2; - BIT_DStream_t bitD3; - BIT_DStream_t bitD4; - size_t const length1 = MEM_readLE16(istart); - size_t const length2 = MEM_readLE16(istart+2); - size_t const length3 = MEM_readLE16(istart+4); - size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6); - const BYTE* const istart1 = istart + 6; /* jumpTable */ - const BYTE* const istart2 = istart1 + length1; - const BYTE* const istart3 = istart2 + length2; - const BYTE* const istart4 = istart3 + length3; - const size_t segmentSize = (dstSize+3) / 4; - BYTE* const opStart2 = ostart + segmentSize; - BYTE* const opStart3 = opStart2 + segmentSize; - BYTE* const opStart4 = opStart3 + segmentSize; - BYTE* op1 = ostart; - BYTE* op2 = opStart2; - BYTE* op3 = opStart3; - BYTE* op4 = opStart4; - U32 endSignal; - DTableDesc const dtd = HUF_getDTableDesc(DTable); - U32 const dtLog = dtd.tableLog; - - if (length4 > cSrcSize) return ERROR(corruption_detected); /* overflow */ - { size_t const errorCode = BIT_initDStream(&bitD1, istart1, length1); - if (HUF_isError(errorCode)) return errorCode; } - { size_t const errorCode = BIT_initDStream(&bitD2, istart2, length2); - if (HUF_isError(errorCode)) return errorCode; } - { size_t const errorCode = BIT_initDStream(&bitD3, istart3, length3); - if (HUF_isError(errorCode)) return errorCode; } - { size_t const errorCode = BIT_initDStream(&bitD4, istart4, length4); - if (HUF_isError(errorCode)) return errorCode; } - - /* 16-32 symbols per loop (4-8 symbols per stream) */ - endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); - for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; ) { - HUF_DECODE_SYMBOLX2_2(op1, &bitD1); - HUF_DECODE_SYMBOLX2_2(op2, &bitD2); - HUF_DECODE_SYMBOLX2_2(op3, &bitD3); - HUF_DECODE_SYMBOLX2_2(op4, &bitD4); - HUF_DECODE_SYMBOLX2_1(op1, &bitD1); - HUF_DECODE_SYMBOLX2_1(op2, &bitD2); - HUF_DECODE_SYMBOLX2_1(op3, &bitD3); - HUF_DECODE_SYMBOLX2_1(op4, &bitD4); - HUF_DECODE_SYMBOLX2_2(op1, &bitD1); - HUF_DECODE_SYMBOLX2_2(op2, &bitD2); - HUF_DECODE_SYMBOLX2_2(op3, &bitD3); - HUF_DECODE_SYMBOLX2_2(op4, &bitD4); - HUF_DECODE_SYMBOLX2_0(op1, &bitD1); - HUF_DECODE_SYMBOLX2_0(op2, &bitD2); - HUF_DECODE_SYMBOLX2_0(op3, &bitD3); - HUF_DECODE_SYMBOLX2_0(op4, &bitD4); - endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); - } - - /* check corruption */ - if (op1 > opStart2) return ERROR(corruption_detected); - if (op2 > opStart3) return ERROR(corruption_detected); - if (op3 > opStart4) return ERROR(corruption_detected); - /* note : op4 supposed already verified within main loop */ - - /* finish bitStreams one by one */ - HUF_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog); - HUF_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog); - HUF_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog); - HUF_decodeStreamX2(op4, &bitD4, oend, dt, dtLog); - - /* check */ - endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4); - if (!endSignal) return ERROR(corruption_detected); - - /* decoded size */ - return dstSize; - } -} - - -size_t HUF_decompress4X2_usingDTable( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - DTableDesc dtd = HUF_getDTableDesc(DTable); - if (dtd.tableType != 0) return ERROR(GENERIC); - return HUF_decompress4X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); -} - - -size_t HUF_decompress4X2_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - const BYTE* ip = (const BYTE*) cSrc; - - size_t const hSize = HUF_readDTableX2 (dctx, cSrc, cSrcSize); - if (HUF_isError(hSize)) return hSize; - if (hSize >= cSrcSize) return ERROR(srcSize_wrong); - ip += hSize; cSrcSize -= hSize; - - return HUF_decompress4X2_usingDTable_internal (dst, dstSize, ip, cSrcSize, dctx); -} - -size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_TABLELOG_MAX); - return HUF_decompress4X2_DCtx(DTable, dst, dstSize, cSrc, cSrcSize); -} - - -/* *************************/ -/* double-symbols decoding */ -/* *************************/ -typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUF_DEltX4; /* double-symbols decoding */ - -typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t; - -/* HUF_fillDTableX4Level2() : - * `rankValOrigin` must be a table of at least (HUF_TABLELOG_MAX + 1) U32 */ -static void HUF_fillDTableX4Level2(HUF_DEltX4* DTable, U32 sizeLog, const U32 consumed, - const U32* rankValOrigin, const int minWeight, - const sortedSymbol_t* sortedSymbols, const U32 sortedListSize, - U32 nbBitsBaseline, U16 baseSeq) -{ - HUF_DEltX4 DElt; - U32 rankVal[HUF_TABLELOG_MAX + 1]; - - /* get pre-calculated rankVal */ - memcpy(rankVal, rankValOrigin, sizeof(rankVal)); - - /* fill skipped values */ - if (minWeight>1) { - U32 i, skipSize = rankVal[minWeight]; - MEM_writeLE16(&(DElt.sequence), baseSeq); - DElt.nbBits = (BYTE)(consumed); - DElt.length = 1; - for (i = 0; i < skipSize; i++) - DTable[i] = DElt; - } - - /* fill DTable */ - { U32 s; for (s=0; s= 1 */ - - rankVal[weight] += length; - } } -} - -typedef U32 rankVal_t[HUF_TABLELOG_MAX][HUF_TABLELOG_MAX + 1]; - -static void HUF_fillDTableX4(HUF_DEltX4* DTable, const U32 targetLog, - const sortedSymbol_t* sortedList, const U32 sortedListSize, - const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight, - const U32 nbBitsBaseline) -{ - U32 rankVal[HUF_TABLELOG_MAX + 1]; - const int scaleLog = nbBitsBaseline - targetLog; /* note : targetLog >= srcLog, hence scaleLog <= 1 */ - const U32 minBits = nbBitsBaseline - maxWeight; - U32 s; - - memcpy(rankVal, rankValOrigin, sizeof(rankVal)); - - /* fill DTable */ - for (s=0; s= minBits) { /* enough room for a second symbol */ - U32 sortedRank; - int minWeight = nbBits + scaleLog; - if (minWeight < 1) minWeight = 1; - sortedRank = rankStart[minWeight]; - HUF_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits, - rankValOrigin[nbBits], minWeight, - sortedList+sortedRank, sortedListSize-sortedRank, - nbBitsBaseline, symbol); - } else { - HUF_DEltX4 DElt; - MEM_writeLE16(&(DElt.sequence), symbol); - DElt.nbBits = (BYTE)(nbBits); - DElt.length = 1; - { U32 const end = start + length; - U32 u; - for (u = start; u < end; u++) DTable[u] = DElt; - } } - rankVal[weight] += length; - } -} - -size_t HUF_readDTableX4 (HUF_DTable* DTable, const void* src, size_t srcSize) -{ - BYTE weightList[HUF_SYMBOLVALUE_MAX + 1]; - sortedSymbol_t sortedSymbol[HUF_SYMBOLVALUE_MAX + 1]; - U32 rankStats[HUF_TABLELOG_MAX + 1] = { 0 }; - U32 rankStart0[HUF_TABLELOG_MAX + 2] = { 0 }; - U32* const rankStart = rankStart0+1; - rankVal_t rankVal; - U32 tableLog, maxW, sizeOfSort, nbSymbols; - DTableDesc dtd = HUF_getDTableDesc(DTable); - U32 const maxTableLog = dtd.maxTableLog; - size_t iSize; - void* dtPtr = DTable+1; /* force compiler to avoid strict-aliasing */ - HUF_DEltX4* const dt = (HUF_DEltX4*)dtPtr; - - HUF_STATIC_ASSERT(sizeof(HUF_DEltX4) == sizeof(HUF_DTable)); /* if compilation fails here, assertion is false */ - if (maxTableLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge); - /* memset(weightList, 0, sizeof(weightList)); */ /* is not necessary, even though some analyzer complain ... */ - - iSize = HUF_readStats(weightList, HUF_SYMBOLVALUE_MAX + 1, rankStats, &nbSymbols, &tableLog, src, srcSize); - if (HUF_isError(iSize)) return iSize; - - /* check result */ - if (tableLog > maxTableLog) return ERROR(tableLog_tooLarge); /* DTable can't fit code depth */ - - /* find maxWeight */ - for (maxW = tableLog; rankStats[maxW]==0; maxW--) {} /* necessarily finds a solution before 0 */ - - /* Get start index of each weight */ - { U32 w, nextRankStart = 0; - for (w=1; w> consumed; - } } } } - - HUF_fillDTableX4(dt, maxTableLog, - sortedSymbol, sizeOfSort, - rankStart0, rankVal, maxW, - tableLog+1); - - dtd.tableLog = (BYTE)maxTableLog; - dtd.tableType = 1; - memcpy(DTable, &dtd, sizeof(dtd)); - return iSize; -} - - -static U32 HUF_decodeSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog) -{ - size_t const val = BIT_lookBitsFast(DStream, dtLog); /* note : dtLog >= 1 */ - memcpy(op, dt+val, 2); - BIT_skipBits(DStream, dt[val].nbBits); - return dt[val].length; -} - -static U32 HUF_decodeLastSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog) -{ - size_t const val = BIT_lookBitsFast(DStream, dtLog); /* note : dtLog >= 1 */ - memcpy(op, dt+val, 1); - if (dt[val].length==1) BIT_skipBits(DStream, dt[val].nbBits); - else { - if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8)) { - BIT_skipBits(DStream, dt[val].nbBits); - if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8)) - DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8); /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */ - } } - return 1; -} - - -#define HUF_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \ - ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) - -#define HUF_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \ - if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \ - ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) - -#define HUF_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \ - if (MEM_64bits()) \ - ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) - -static inline size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog) -{ - BYTE* const pStart = p; - - /* up to 8 symbols at a time */ - while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd-(sizeof(bitDPtr->bitContainer)-1))) { - HUF_DECODE_SYMBOLX4_2(p, bitDPtr); - HUF_DECODE_SYMBOLX4_1(p, bitDPtr); - HUF_DECODE_SYMBOLX4_2(p, bitDPtr); - HUF_DECODE_SYMBOLX4_0(p, bitDPtr); - } - - /* closer to end : up to 2 symbols at a time */ - while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p <= pEnd-2)) - HUF_DECODE_SYMBOLX4_0(p, bitDPtr); - - while (p <= pEnd-2) - HUF_DECODE_SYMBOLX4_0(p, bitDPtr); /* no need to reload : reached the end of DStream */ - - if (p < pEnd) - p += HUF_decodeLastSymbolX4(p, bitDPtr, dt, dtLog); - - return p-pStart; -} - - -static size_t HUF_decompress1X4_usingDTable_internal( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - BIT_DStream_t bitD; - - /* Init */ - { size_t const errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize); - if (HUF_isError(errorCode)) return errorCode; - } - - /* decode */ - { BYTE* const ostart = (BYTE*) dst; - BYTE* const oend = ostart + dstSize; - const void* const dtPtr = DTable+1; /* force compiler to not use strict-aliasing */ - const HUF_DEltX4* const dt = (const HUF_DEltX4*)dtPtr; - DTableDesc const dtd = HUF_getDTableDesc(DTable); - HUF_decodeStreamX4(ostart, &bitD, oend, dt, dtd.tableLog); - } - - /* check */ - if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected); - - /* decoded size */ - return dstSize; -} - -size_t HUF_decompress1X4_usingDTable( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - DTableDesc dtd = HUF_getDTableDesc(DTable); - if (dtd.tableType != 1) return ERROR(GENERIC); - return HUF_decompress1X4_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); -} - -size_t HUF_decompress1X4_DCtx (HUF_DTable* DCtx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - const BYTE* ip = (const BYTE*) cSrc; - - size_t const hSize = HUF_readDTableX4 (DCtx, cSrc, cSrcSize); - if (HUF_isError(hSize)) return hSize; - if (hSize >= cSrcSize) return ERROR(srcSize_wrong); - ip += hSize; cSrcSize -= hSize; - - return HUF_decompress1X4_usingDTable_internal (dst, dstSize, ip, cSrcSize, DCtx); -} - -size_t HUF_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_TABLELOG_MAX); - return HUF_decompress1X4_DCtx(DTable, dst, dstSize, cSrc, cSrcSize); -} - -static size_t HUF_decompress4X4_usingDTable_internal( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - if (cSrcSize < 10) return ERROR(corruption_detected); /* strict minimum : jump table + 1 byte per stream */ - - { const BYTE* const istart = (const BYTE*) cSrc; - BYTE* const ostart = (BYTE*) dst; - BYTE* const oend = ostart + dstSize; - const void* const dtPtr = DTable+1; - const HUF_DEltX4* const dt = (const HUF_DEltX4*)dtPtr; - - /* Init */ - BIT_DStream_t bitD1; - BIT_DStream_t bitD2; - BIT_DStream_t bitD3; - BIT_DStream_t bitD4; - size_t const length1 = MEM_readLE16(istart); - size_t const length2 = MEM_readLE16(istart+2); - size_t const length3 = MEM_readLE16(istart+4); - size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6); - const BYTE* const istart1 = istart + 6; /* jumpTable */ - const BYTE* const istart2 = istart1 + length1; - const BYTE* const istart3 = istart2 + length2; - const BYTE* const istart4 = istart3 + length3; - size_t const segmentSize = (dstSize+3) / 4; - BYTE* const opStart2 = ostart + segmentSize; - BYTE* const opStart3 = opStart2 + segmentSize; - BYTE* const opStart4 = opStart3 + segmentSize; - BYTE* op1 = ostart; - BYTE* op2 = opStart2; - BYTE* op3 = opStart3; - BYTE* op4 = opStart4; - U32 endSignal; - DTableDesc const dtd = HUF_getDTableDesc(DTable); - U32 const dtLog = dtd.tableLog; - - if (length4 > cSrcSize) return ERROR(corruption_detected); /* overflow */ - { size_t const errorCode = BIT_initDStream(&bitD1, istart1, length1); - if (HUF_isError(errorCode)) return errorCode; } - { size_t const errorCode = BIT_initDStream(&bitD2, istart2, length2); - if (HUF_isError(errorCode)) return errorCode; } - { size_t const errorCode = BIT_initDStream(&bitD3, istart3, length3); - if (HUF_isError(errorCode)) return errorCode; } - { size_t const errorCode = BIT_initDStream(&bitD4, istart4, length4); - if (HUF_isError(errorCode)) return errorCode; } - - /* 16-32 symbols per loop (4-8 symbols per stream) */ - endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); - for ( ; (endSignal==BIT_DStream_unfinished) & (op4<(oend-(sizeof(bitD4.bitContainer)-1))) ; ) { - HUF_DECODE_SYMBOLX4_2(op1, &bitD1); - HUF_DECODE_SYMBOLX4_2(op2, &bitD2); - HUF_DECODE_SYMBOLX4_2(op3, &bitD3); - HUF_DECODE_SYMBOLX4_2(op4, &bitD4); - HUF_DECODE_SYMBOLX4_1(op1, &bitD1); - HUF_DECODE_SYMBOLX4_1(op2, &bitD2); - HUF_DECODE_SYMBOLX4_1(op3, &bitD3); - HUF_DECODE_SYMBOLX4_1(op4, &bitD4); - HUF_DECODE_SYMBOLX4_2(op1, &bitD1); - HUF_DECODE_SYMBOLX4_2(op2, &bitD2); - HUF_DECODE_SYMBOLX4_2(op3, &bitD3); - HUF_DECODE_SYMBOLX4_2(op4, &bitD4); - HUF_DECODE_SYMBOLX4_0(op1, &bitD1); - HUF_DECODE_SYMBOLX4_0(op2, &bitD2); - HUF_DECODE_SYMBOLX4_0(op3, &bitD3); - HUF_DECODE_SYMBOLX4_0(op4, &bitD4); - - endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4); - } - - /* check corruption */ - if (op1 > opStart2) return ERROR(corruption_detected); - if (op2 > opStart3) return ERROR(corruption_detected); - if (op3 > opStart4) return ERROR(corruption_detected); - /* note : op4 already verified within main loop */ - - /* finish bitStreams one by one */ - HUF_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog); - HUF_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog); - HUF_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog); - HUF_decodeStreamX4(op4, &bitD4, oend, dt, dtLog); - - /* check */ - { U32 const endCheck = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4); - if (!endCheck) return ERROR(corruption_detected); } - - /* decoded size */ - return dstSize; - } -} - - -size_t HUF_decompress4X4_usingDTable( - void* dst, size_t dstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - DTableDesc dtd = HUF_getDTableDesc(DTable); - if (dtd.tableType != 1) return ERROR(GENERIC); - return HUF_decompress4X4_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable); -} - - -size_t HUF_decompress4X4_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - const BYTE* ip = (const BYTE*) cSrc; - - size_t hSize = HUF_readDTableX4 (dctx, cSrc, cSrcSize); - if (HUF_isError(hSize)) return hSize; - if (hSize >= cSrcSize) return ERROR(srcSize_wrong); - ip += hSize; cSrcSize -= hSize; - - return HUF_decompress4X4_usingDTable_internal(dst, dstSize, ip, cSrcSize, dctx); -} - -size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_TABLELOG_MAX); - return HUF_decompress4X4_DCtx(DTable, dst, dstSize, cSrc, cSrcSize); -} - - -/* ********************************/ -/* Generic decompression selector */ -/* ********************************/ - -size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - DTableDesc const dtd = HUF_getDTableDesc(DTable); - return dtd.tableType ? HUF_decompress1X4_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable) : - HUF_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable); -} - -size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize, - const void* cSrc, size_t cSrcSize, - const HUF_DTable* DTable) -{ - DTableDesc const dtd = HUF_getDTableDesc(DTable); - return dtd.tableType ? HUF_decompress4X4_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable) : - HUF_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable); -} - - -typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t; -static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] = -{ - /* single, double, quad */ - {{0,0}, {1,1}, {2,2}}, /* Q==0 : impossible */ - {{0,0}, {1,1}, {2,2}}, /* Q==1 : impossible */ - {{ 38,130}, {1313, 74}, {2151, 38}}, /* Q == 2 : 12-18% */ - {{ 448,128}, {1353, 74}, {2238, 41}}, /* Q == 3 : 18-25% */ - {{ 556,128}, {1353, 74}, {2238, 47}}, /* Q == 4 : 25-32% */ - {{ 714,128}, {1418, 74}, {2436, 53}}, /* Q == 5 : 32-38% */ - {{ 883,128}, {1437, 74}, {2464, 61}}, /* Q == 6 : 38-44% */ - {{ 897,128}, {1515, 75}, {2622, 68}}, /* Q == 7 : 44-50% */ - {{ 926,128}, {1613, 75}, {2730, 75}}, /* Q == 8 : 50-56% */ - {{ 947,128}, {1729, 77}, {3359, 77}}, /* Q == 9 : 56-62% */ - {{1107,128}, {2083, 81}, {4006, 84}}, /* Q ==10 : 62-69% */ - {{1177,128}, {2379, 87}, {4785, 88}}, /* Q ==11 : 69-75% */ - {{1242,128}, {2415, 93}, {5155, 84}}, /* Q ==12 : 75-81% */ - {{1349,128}, {2644,106}, {5260,106}}, /* Q ==13 : 81-87% */ - {{1455,128}, {2422,124}, {4174,124}}, /* Q ==14 : 87-93% */ - {{ 722,128}, {1891,145}, {1936,146}}, /* Q ==15 : 93-99% */ -}; - -/** HUF_selectDecoder() : -* Tells which decoder is likely to decode faster, -* based on a set of pre-determined metrics. -* @return : 0==HUF_decompress4X2, 1==HUF_decompress4X4 . -* Assumption : 0 < cSrcSize < dstSize <= 128 KB */ -U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize) -{ - /* decoder timing evaluation */ - U32 const Q = (U32)(cSrcSize * 16 / dstSize); /* Q < 16 since dstSize > cSrcSize */ - U32 const D256 = (U32)(dstSize >> 8); - U32 const DTime0 = algoTime[Q][0].tableTime + (algoTime[Q][0].decode256Time * D256); - U32 DTime1 = algoTime[Q][1].tableTime + (algoTime[Q][1].decode256Time * D256); - DTime1 += DTime1 >> 3; /* advantage to algorithm using less memory, for cache eviction */ - - return DTime1 < DTime0; -} - - -typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); - -size_t HUF_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - static const decompressionAlgo decompress[2] = { HUF_decompress4X2, HUF_decompress4X4 }; - - /* validation checks */ - if (dstSize == 0) return ERROR(dstSize_tooSmall); - if (cSrcSize > dstSize) return ERROR(corruption_detected); /* invalid */ - if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; } /* not compressed */ - if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; } /* RLE */ - - { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); - return decompress[algoNb](dst, dstSize, cSrc, cSrcSize); - } -} - -size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - /* validation checks */ - if (dstSize == 0) return ERROR(dstSize_tooSmall); - if (cSrcSize > dstSize) return ERROR(corruption_detected); /* invalid */ - if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; } /* not compressed */ - if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; } /* RLE */ - - { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); - return algoNb ? HUF_decompress4X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) : - HUF_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ; - } -} - -size_t HUF_decompress4X_hufOnly (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - /* validation checks */ - if (dstSize == 0) return ERROR(dstSize_tooSmall); - if ((cSrcSize >= dstSize) || (cSrcSize <= 1)) return ERROR(corruption_detected); /* invalid */ - - { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); - return algoNb ? HUF_decompress4X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) : - HUF_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ; - } -} - -size_t HUF_decompress1X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize) -{ - /* validation checks */ - if (dstSize == 0) return ERROR(dstSize_tooSmall); - if (cSrcSize > dstSize) return ERROR(corruption_detected); /* invalid */ - if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; } /* not compressed */ - if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; } /* RLE */ - - { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); - return algoNb ? HUF_decompress1X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) : - HUF_decompress1X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ; - } -} diff --git a/contrib/lizard/lib/entropy/mem.h b/contrib/lizard/lib/entropy/mem.h deleted file mode 100644 index 708d897a1cc..00000000000 --- a/contrib/lizard/lib/entropy/mem.h +++ /dev/null @@ -1,372 +0,0 @@ -/** - * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. - * All rights reserved. - * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. - */ - -#ifndef MEM_H_MODULE -#define MEM_H_MODULE - -#if defined (__cplusplus) -extern "C" { -#endif - -/*-**************************************** -* Dependencies -******************************************/ -#include /* size_t, ptrdiff_t */ -#include /* memcpy */ - - -/*-**************************************** -* Compiler specifics -******************************************/ -#if defined(_MSC_VER) /* Visual Studio */ -# include /* _byteswap_ulong */ -# include /* _byteswap_* */ -#endif -#if defined(__GNUC__) -# define MEM_STATIC static __inline __attribute__((unused)) -#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) -# define MEM_STATIC static inline -#elif defined(_MSC_VER) -# define MEM_STATIC static __inline -#else -# define MEM_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */ -#endif - -/* code only tested on 32 and 64 bits systems */ -#define MEM_STATIC_ASSERT(c) { enum { XXH_static_assert = 1/(int)(!!(c)) }; } -MEM_STATIC void MEM_check(void) { MEM_STATIC_ASSERT((sizeof(size_t)==4) || (sizeof(size_t)==8)); } - - -/*-************************************************************** -* Basic Types -*****************************************************************/ -#if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) -# include - typedef uint8_t BYTE; - typedef uint16_t U16; - typedef int16_t S16; - typedef uint32_t U32; - typedef int32_t S32; - typedef uint64_t U64; - typedef int64_t S64; - typedef intptr_t iPtrDiff; -#else - typedef unsigned char BYTE; - typedef unsigned short U16; - typedef signed short S16; - typedef unsigned int U32; - typedef signed int S32; - typedef unsigned long long U64; - typedef signed long long S64; - typedef ptrdiff_t iPtrDiff; -#endif - - -/*-************************************************************** -* Memory I/O -*****************************************************************/ -/* MEM_FORCE_MEMORY_ACCESS : - * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable. - * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal. - * The below switch allow to select different access method for improved performance. - * Method 0 (default) : use `memcpy()`. Safe and portable. - * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable). - * This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`. - * Method 2 : direct access. This method is portable but violate C standard. - * It can generate buggy code on targets depending on alignment. - * In some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6) - * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details. - * Prefer these methods in priority order (0 > 1 > 2) - */ -#ifndef MEM_FORCE_MEMORY_ACCESS /* can be defined externally, on command line for example */ -# if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) ) -# define MEM_FORCE_MEMORY_ACCESS 2 -# elif defined(__INTEL_COMPILER) /*|| defined(_MSC_VER)*/ || \ - (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) )) -# define MEM_FORCE_MEMORY_ACCESS 1 -# endif -#endif - -MEM_STATIC unsigned MEM_32bits(void) { return sizeof(size_t)==4; } -MEM_STATIC unsigned MEM_64bits(void) { return sizeof(size_t)==8; } - -MEM_STATIC unsigned MEM_isLittleEndian(void) -{ - const union { U32 u; BYTE c[4]; } one = { 1 }; /* don't use static : performance detrimental */ - return one.c[0]; -} - -#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2) - -/* violates C standard, by lying on structure alignment. -Only use if no other choice to achieve best performance on target platform */ -MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; } -MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; } -MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; } -MEM_STATIC U64 MEM_readST(const void* memPtr) { return *(const size_t*) memPtr; } - -MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; } -MEM_STATIC void MEM_write32(void* memPtr, U32 value) { *(U32*)memPtr = value; } -MEM_STATIC void MEM_write64(void* memPtr, U64 value) { *(U64*)memPtr = value; } - -#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1) - -/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */ -/* currently only defined for gcc and icc */ -#if defined(_MSC_VER) || (defined(__INTEL_COMPILER) && defined(WIN32)) - __pragma( pack(push, 1) ) - typedef union { U16 u16; U32 u32; U64 u64; size_t st; } unalign; - __pragma( pack(pop) ) -#else - typedef union { U16 u16; U32 u32; U64 u64; size_t st; } __attribute__((packed)) unalign; -#endif - -MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; } -MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; } -MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; } -MEM_STATIC U64 MEM_readST(const void* ptr) { return ((const unalign*)ptr)->st; } - -MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; } -MEM_STATIC void MEM_write32(void* memPtr, U32 value) { ((unalign*)memPtr)->u32 = value; } -MEM_STATIC void MEM_write64(void* memPtr, U64 value) { ((unalign*)memPtr)->u64 = value; } - -#else - -/* default method, safe and standard. - can sometimes prove slower */ - -MEM_STATIC U16 MEM_read16(const void* memPtr) -{ - U16 val; memcpy(&val, memPtr, sizeof(val)); return val; -} - -MEM_STATIC U32 MEM_read32(const void* memPtr) -{ - U32 val; memcpy(&val, memPtr, sizeof(val)); return val; -} - -MEM_STATIC U64 MEM_read64(const void* memPtr) -{ - U64 val; memcpy(&val, memPtr, sizeof(val)); return val; -} - -MEM_STATIC size_t MEM_readST(const void* memPtr) -{ - size_t val; memcpy(&val, memPtr, sizeof(val)); return val; -} - -MEM_STATIC void MEM_write16(void* memPtr, U16 value) -{ - memcpy(memPtr, &value, sizeof(value)); -} - -MEM_STATIC void MEM_write32(void* memPtr, U32 value) -{ - memcpy(memPtr, &value, sizeof(value)); -} - -MEM_STATIC void MEM_write64(void* memPtr, U64 value) -{ - memcpy(memPtr, &value, sizeof(value)); -} - -#endif /* MEM_FORCE_MEMORY_ACCESS */ - -MEM_STATIC U32 MEM_swap32(U32 in) -{ -#if defined(_MSC_VER) /* Visual Studio */ - return _byteswap_ulong(in); -#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403) - return __builtin_bswap32(in); -#else - return ((in << 24) & 0xff000000 ) | - ((in << 8) & 0x00ff0000 ) | - ((in >> 8) & 0x0000ff00 ) | - ((in >> 24) & 0x000000ff ); -#endif -} - -MEM_STATIC U64 MEM_swap64(U64 in) -{ -#if defined(_MSC_VER) /* Visual Studio */ - return _byteswap_uint64(in); -#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403) - return __builtin_bswap64(in); -#else - return ((in << 56) & 0xff00000000000000ULL) | - ((in << 40) & 0x00ff000000000000ULL) | - ((in << 24) & 0x0000ff0000000000ULL) | - ((in << 8) & 0x000000ff00000000ULL) | - ((in >> 8) & 0x00000000ff000000ULL) | - ((in >> 24) & 0x0000000000ff0000ULL) | - ((in >> 40) & 0x000000000000ff00ULL) | - ((in >> 56) & 0x00000000000000ffULL); -#endif -} - -MEM_STATIC size_t MEM_swapST(size_t in) -{ - if (MEM_32bits()) - return (size_t)MEM_swap32((U32)in); - else - return (size_t)MEM_swap64((U64)in); -} - -/*=== Little endian r/w ===*/ - -MEM_STATIC U16 MEM_readLE16(const void* memPtr) -{ - if (MEM_isLittleEndian()) - return MEM_read16(memPtr); - else { - const BYTE* p = (const BYTE*)memPtr; - return (U16)(p[0] + (p[1]<<8)); - } -} - -MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val) -{ - if (MEM_isLittleEndian()) { - MEM_write16(memPtr, val); - } else { - BYTE* p = (BYTE*)memPtr; - p[0] = (BYTE)val; - p[1] = (BYTE)(val>>8); - } -} - -MEM_STATIC U32 MEM_readLE24(const void* memPtr) -{ - return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16); -} - -MEM_STATIC void MEM_writeLE24(void* memPtr, U32 val) -{ - MEM_writeLE16(memPtr, (U16)val); - ((BYTE*)memPtr)[2] = (BYTE)(val>>16); -} - -MEM_STATIC U32 MEM_readLE32(const void* memPtr) -{ - if (MEM_isLittleEndian()) - return MEM_read32(memPtr); - else - return MEM_swap32(MEM_read32(memPtr)); -} - -MEM_STATIC void MEM_writeLE32(void* memPtr, U32 val32) -{ - if (MEM_isLittleEndian()) - MEM_write32(memPtr, val32); - else - MEM_write32(memPtr, MEM_swap32(val32)); -} - -MEM_STATIC U64 MEM_readLE64(const void* memPtr) -{ - if (MEM_isLittleEndian()) - return MEM_read64(memPtr); - else - return MEM_swap64(MEM_read64(memPtr)); -} - -MEM_STATIC void MEM_writeLE64(void* memPtr, U64 val64) -{ - if (MEM_isLittleEndian()) - MEM_write64(memPtr, val64); - else - MEM_write64(memPtr, MEM_swap64(val64)); -} - -MEM_STATIC size_t MEM_readLEST(const void* memPtr) -{ - if (MEM_32bits()) - return (size_t)MEM_readLE32(memPtr); - else - return (size_t)MEM_readLE64(memPtr); -} - -MEM_STATIC void MEM_writeLEST(void* memPtr, size_t val) -{ - if (MEM_32bits()) - MEM_writeLE32(memPtr, (U32)val); - else - MEM_writeLE64(memPtr, (U64)val); -} - -/*=== Big endian r/w ===*/ - -MEM_STATIC U32 MEM_readBE32(const void* memPtr) -{ - if (MEM_isLittleEndian()) - return MEM_swap32(MEM_read32(memPtr)); - else - return MEM_read32(memPtr); -} - -MEM_STATIC void MEM_writeBE32(void* memPtr, U32 val32) -{ - if (MEM_isLittleEndian()) - MEM_write32(memPtr, MEM_swap32(val32)); - else - MEM_write32(memPtr, val32); -} - -MEM_STATIC U64 MEM_readBE64(const void* memPtr) -{ - if (MEM_isLittleEndian()) - return MEM_swap64(MEM_read64(memPtr)); - else - return MEM_read64(memPtr); -} - -MEM_STATIC void MEM_writeBE64(void* memPtr, U64 val64) -{ - if (MEM_isLittleEndian()) - MEM_write64(memPtr, MEM_swap64(val64)); - else - MEM_write64(memPtr, val64); -} - -MEM_STATIC size_t MEM_readBEST(const void* memPtr) -{ - if (MEM_32bits()) - return (size_t)MEM_readBE32(memPtr); - else - return (size_t)MEM_readBE64(memPtr); -} - -MEM_STATIC void MEM_writeBEST(void* memPtr, size_t val) -{ - if (MEM_32bits()) - MEM_writeBE32(memPtr, (U32)val); - else - MEM_writeBE64(memPtr, (U64)val); -} - - -/* function safe only for comparisons */ -MEM_STATIC U32 MEM_readMINMATCH(const void* memPtr, U32 length) -{ - switch (length) - { - default : - case 4 : return MEM_read32(memPtr); - case 3 : if (MEM_isLittleEndian()) - return MEM_read32(memPtr)<<8; - else - return MEM_read32(memPtr)>>8; - } -} - -#if defined (__cplusplus) -} -#endif - -#endif /* MEM_H_MODULE */ diff --git a/contrib/lizard/lib/liblizard.pc.in b/contrib/lizard/lib/liblizard.pc.in deleted file mode 100644 index 233c2e08faf..00000000000 --- a/contrib/lizard/lib/liblizard.pc.in +++ /dev/null @@ -1,15 +0,0 @@ -# Lizard - Fast LZ compression algorithm -# Copyright (C) 2011-2014, Yann Collet -# Copyright (C) 2016-2017, Przemyslaw Skibinski -# BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - -prefix=@PREFIX@ -libdir=@LIBDIR@ -includedir=@INCLUDEDIR@ - -Name: lizard -Description: extremely fast lossless compression algorithm library -URL: http://github.com/inikep/lizard -Version: @VERSION@ -Libs: -L@LIBDIR@ -llizard -Cflags: -I@INCLUDEDIR@ diff --git a/contrib/lizard/lib/lizard_common.h b/contrib/lizard/lib/lizard_common.h deleted file mode 100644 index 45730cf4e86..00000000000 --- a/contrib/lizard/lib/lizard_common.h +++ /dev/null @@ -1,504 +0,0 @@ -/* - Lizard - Fast LZ compression algorithm - Copyright (C) 2011-2015, Yann Collet - Copyright (C) 2016-2017, Przemyslaw Skibinski - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -#ifndef LIZARD_COMMON_H_2983 -#define LIZARD_COMMON_H_2983 - -#if defined (__cplusplus) -extern "C" { -#endif - - -/*-************************************ -* Memory routines -**************************************/ -#include /* malloc, calloc, free */ -#include /* memset, memcpy */ -#include /* intptr_t */ -#include "entropy/mem.h" -#include "lizard_compress.h" /* LIZARD_GCC_VERSION */ - -//#define LIZARD_USE_LOGS -#define LIZARD_LOG_COMPRESS(...) //printf(__VA_ARGS__) -#define LIZARD_LOG_DECOMPRESS(...) //printf(__VA_ARGS__) - -#define LIZARD_LOG_COMPRESS_LZ4(...) //printf(__VA_ARGS__) -#define COMPLOG_CODEWORDS_LZ4(...) //printf(__VA_ARGS__) -#define LIZARD_LOG_DECOMPRESS_LZ4(...) //printf(__VA_ARGS__) -#define DECOMPLOG_CODEWORDS_LZ4(...) //printf(__VA_ARGS__) - -#define LIZARD_LOG_COMPRESS_LIZv1(...) //printf(__VA_ARGS__) -#define COMPLOG_CODEWORDS_LIZv1(...) //printf(__VA_ARGS__) -#define LIZARD_LOG_DECOMPRESS_LIZv1(...) //printf(__VA_ARGS__) -#define DECOMPLOG_CODEWORDS_LIZv1(...) //printf(__VA_ARGS__) - - - - -/*-************************************ -* Common Constants -**************************************/ -#define MINMATCH 4 -//#define USE_LZ4_ONLY -//#define LIZARD_USE_TEST - -#define LIZARD_DICT_SIZE (1<<24) -#define WILDCOPYLENGTH 16 -#define LASTLITERALS WILDCOPYLENGTH -#define MFLIMIT (WILDCOPYLENGTH+MINMATCH) - -#define LIZARD_MAX_PRICE (1<<28) -#define LIZARD_INIT_LAST_OFFSET 0 -#define LIZARD_MAX_16BIT_OFFSET (1<<16) -#define MM_LONGOFF 16 -#define LIZARD_BLOCK_SIZE_PAD (LIZARD_BLOCK_SIZE+32) -#define LIZARD_COMPRESS_ADD_BUF (5*LIZARD_BLOCK_SIZE_PAD) -#ifndef LIZARD_NO_HUFFMAN - #define LIZARD_COMPRESS_ADD_HUF HUF_compressBound(LIZARD_BLOCK_SIZE_PAD) - #define LIZARD_HUF_BLOCK_SIZE LIZARD_BLOCK_SIZE -#else - #define LIZARD_COMPRESS_ADD_HUF 0 - #define LIZARD_HUF_BLOCK_SIZE 1 -#endif - -/* LZ4 codewords */ -#define ML_BITS_LZ4 4 -#define ML_MASK_LZ4 ((1U<= 1<<16 */ - U32 sufficientLength; /* used only by optimal parser: size of matches which is acceptable: larger == more compression, slower */ - U32 fullSearch; /* used only by optimal parser: perform full search of matches: 1 == more compression, slower */ - Lizard_parser_type parserType; - Lizard_decompress_type decompressType; -} Lizard_parameters; - - -struct Lizard_stream_s -{ - const BYTE* end; /* next block here to continue on current prefix */ - const BYTE* base; /* All index relative to this position */ - const BYTE* dictBase; /* alternate base for extDict */ - U32 dictLimit; /* below that point, need extDict */ - U32 lowLimit; /* below that point, no more dict */ - U32 nextToUpdate; /* index from which to continue dictionary update */ - U32 allocatedMemory; - int compressionLevel; - Lizard_parameters params; - U32 hashTableSize; - U32 chainTableSize; - U32* chainTable; - U32* hashTable; - int last_off; - const BYTE* off24pos; - U32 huffType; - U32 comprStreamLen; - - BYTE* huffBase; - BYTE* huffEnd; - BYTE* offset16Base; - BYTE* offset24Base; - BYTE* lenBase; - BYTE* literalsBase; - BYTE* flagsBase; - BYTE* offset16Ptr; - BYTE* offset24Ptr; - BYTE* lenPtr; - BYTE* literalsPtr; - BYTE* flagsPtr; - BYTE* offset16End; - BYTE* offset24End; - BYTE* lenEnd; - BYTE* literalsEnd; - BYTE* flagsEnd; - U32 flagFreq[256]; - U32 litFreq[256]; - U32 litSum, flagSum; - U32 litPriceSum, log2LitSum, log2FlagSum; - U32 cachedPrice; - U32 cachedLitLength; - const BYTE* cachedLiterals; - const BYTE* diffBase; - const BYTE* srcBase; - const BYTE* destBase; -}; - -struct Lizard_streamDecode_s { - const BYTE* externalDict; - size_t extDictSize; - const BYTE* prefixEnd; - size_t prefixSize; -}; - -struct Lizard_dstream_s -{ - const BYTE* offset16Ptr; - const BYTE* offset24Ptr; - const BYTE* lenPtr; - const BYTE* literalsPtr; - const BYTE* flagsPtr; - const BYTE* offset16End; - const BYTE* offset24End; - const BYTE* lenEnd; - const BYTE* literalsEnd; - const BYTE* flagsEnd; - const BYTE* diffBase; - intptr_t last_off; -}; - -typedef struct Lizard_dstream_s Lizard_dstream_t; - -/* ************************************* -* HC Pre-defined compression levels -***************************************/ -#define LIZARD_WINDOWLOG_LZ4 16 -#define LIZARD_CHAINLOG_LZ4 LIZARD_WINDOWLOG_LZ4 -#define LIZARD_HASHLOG_LZ4 18 -#define LIZARD_HASHLOG_LZ4SM 12 - -#define LIZARD_WINDOWLOG_LIZv1 22 -#define LIZARD_CHAINLOG_LIZv1 LIZARD_WINDOWLOG_LIZv1 -#define LIZARD_HASHLOG_LIZv1 18 - - - -static const Lizard_parameters Lizard_defaultParameters[LIZARD_MAX_CLEVEL+1-LIZARD_MIN_CLEVEL] = -{ - /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ - { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4SM, 0, 0, 0, 0, 0, 0, Lizard_parser_fastSmall, Lizard_coderwords_LZ4 }, // level 10 - { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_fast, Lizard_coderwords_LZ4 }, // level 11 - { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_noChain, Lizard_coderwords_LZ4 }, // level 12 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 2, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 13 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 4, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 14 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 8, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 15 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 16, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 16 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 256, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 17 - { LIZARD_WINDOWLOG_LZ4, LIZARD_WINDOWLOG_LZ4+1, LIZARD_HASHLOG_LZ4, 16, 16, 4, 0, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LZ4 }, // level 18 - { LIZARD_WINDOWLOG_LZ4, LIZARD_WINDOWLOG_LZ4+1, 23, 16, 256, 4, 0, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LZ4 }, // level 19 - /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ - { LIZARD_WINDOWLOG_LIZv1, 0, 14, 0, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_fastBig, Lizard_coderwords_LIZv1 }, // level 20 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 14, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 21 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 22 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 23 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 2, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 24 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 8, 4, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 25 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 8, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 26 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 128, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 27 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 24, 1<<10, 4, MM_LONGOFF, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 28 - { 24, 25, 23, 24, 1<<10, 4, MM_LONGOFF, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 29 -#ifndef LIZARD_NO_HUFFMAN - /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ - { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4SM, 0, 0, 0, 0, 0, 0, Lizard_parser_fastSmall, Lizard_coderwords_LZ4 }, // level 30 - { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_fast, Lizard_coderwords_LZ4 }, // level 31 - { LIZARD_WINDOWLOG_LZ4, 0, 14, 0, 0, 0, 0, 0, 0, Lizard_parser_noChain, Lizard_coderwords_LZ4 }, // level 32 - { LIZARD_WINDOWLOG_LZ4, 0, LIZARD_HASHLOG_LZ4, 0, 0, 0, 0, 0, 0, Lizard_parser_noChain, Lizard_coderwords_LZ4 }, // level 33 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 2, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 34 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 4, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 35 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 8, 5, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 36 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 16, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 37 - { LIZARD_WINDOWLOG_LZ4, LIZARD_CHAINLOG_LZ4, LIZARD_HASHLOG_LZ4, 0, 256, 4, 0, 0, 0, Lizard_parser_hashChain, Lizard_coderwords_LZ4 }, // level 38 - { LIZARD_WINDOWLOG_LZ4, LIZARD_WINDOWLOG_LZ4+1, 23, 16, 256, 4, 0, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LZ4 }, // level 39 - /* windLog, contentLog, HashLog, H3, Snum, SL, MMLongOff, SuffL, FS, Parser function, Decompressor type */ - { LIZARD_WINDOWLOG_LIZv1, 0, 14, 0, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_fastBig, Lizard_coderwords_LIZv1 }, // level 40 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 14, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 41 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 0, 0, Lizard_parser_priceFast, Lizard_coderwords_LIZv1 }, // level 42 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, LIZARD_HASHLOG_LIZv1, 13, 1, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 43 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 2, 5, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 44 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 8, 4, MM_LONGOFF, 64, 0, Lizard_parser_lowestPrice, Lizard_coderwords_LIZv1 }, // level 45 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1, 23, 16, 8, 4, MM_LONGOFF, 64, 0, Lizard_parser_optimalPrice, Lizard_coderwords_LIZv1 }, // level 46 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 8, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 47 - { LIZARD_WINDOWLOG_LIZv1, LIZARD_CHAINLOG_LIZv1+1, 23, 16, 128, 4, MM_LONGOFF, 64, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 48 - { 24, 25, 23, 24, 1<<10, 4, MM_LONGOFF, 1<<10, 1, Lizard_parser_optimalPriceBT, Lizard_coderwords_LIZv1 }, // level 49 -#endif -// { 10, 10, 10, 0, 0, 4, 0, 0, 0, Lizard_fast }, // min values -// { 24, 24, 28, 24, 1<<24, 7, 0, 1<<24, 2, Lizard_optimal_price }, // max values -}; - - - -/*-************************************ -* Compiler Options -**************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# include -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4293) /* disable: C4293: too large shift (32-bits) */ -#else -# if defined(__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */ -# if defined(__GNUC__) || defined(__clang__) -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif /* _MSC_VER */ - -#define LIZARD_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) -#if (LIZARD_GCC_VERSION >= 302) || (__INTEL_COMPILER >= 800) || defined(__clang__) -# define expect(expr,value) (__builtin_expect ((expr),(value)) ) -#else -# define expect(expr,value) (expr) -#endif - -#define likely(expr) expect((expr) != 0, 1) -#define unlikely(expr) expect((expr) != 0, 0) - -#define KB *(1 <<10) -#define MB *(1 <<20) -#define GB *(1U<<30) - -#define ALLOCATOR(n,s) calloc(n,s) -#define FREEMEM free -#define MEM_INIT memset -#ifndef MAX - #define MAX(a,b) ((a)>(b))?(a):(b) -#endif -#ifndef MIN - #define MIN(a,b) ((a)<(b)?(a):(b)) -#endif - -#if MINMATCH == 3 - #define MEM_readMINMATCH(ptr) (U32)(MEM_read32(ptr)<<8) -#else - #define MEM_readMINMATCH(ptr) (U32)(MEM_read32(ptr)) -#endif - - - - -/*-************************************ -* Reading and writing into memory -**************************************/ -#define STEPSIZE sizeof(size_t) - - -MEM_STATIC void Lizard_copy8(void* dst, const void* src) -{ - memcpy(dst,src,8); -} - -/* customized variant of memcpy, which can overwrite up to 7 bytes beyond dstEnd */ -MEM_STATIC void Lizard_wildCopy(void* dstPtr, const void* srcPtr, void* dstEnd) -{ - BYTE* d = (BYTE*)dstPtr; - const BYTE* s = (const BYTE*)srcPtr; - BYTE* const e = (BYTE*)dstEnd; - -#if 0 - const size_t l2 = 8 - (((size_t)d) & (sizeof(void*)-1)); - Lizard_copy8(d,s); if (d>e-9) return; - d+=l2; s+=l2; -#endif /* join to align */ - - do { Lizard_copy8(d,s); d+=8; s+=8; } while (d= 3) /* GCC Intrinsic */ - return 31 - __builtin_clz(val); -# else /* Software version */ - static const int DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 }; - U32 v = val; - int r; - v |= v >> 1; - v |= v >> 2; - v |= v >> 4; - v |= v >> 8; - v |= v >> 16; - r = DeBruijnClz[(U32)(v * 0x07C4ACDDU) >> 27]; - return r; -# endif -} - - -/*-************************************ -* Common functions -**************************************/ -MEM_STATIC unsigned Lizard_NbCommonBytes (register size_t val) -{ - if (MEM_isLittleEndian()) { - if (MEM_64bits()) { -# if defined(_MSC_VER) && defined(_WIN64) && !defined(LIZARD_FORCE_SW_BITCOUNT) - unsigned long r = 0; - _BitScanForward64( &r, (U64)val ); - return (int)(r>>3); -# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) - return (__builtin_ctzll((U64)val) >> 3); -# else - static const int DeBruijnBytePos[64] = { 0, 0, 0, 0, 0, 1, 1, 2, 0, 3, 1, 3, 1, 4, 2, 7, 0, 2, 3, 6, 1, 5, 3, 5, 1, 3, 4, 4, 2, 5, 6, 7, 7, 0, 1, 2, 3, 3, 4, 6, 2, 6, 5, 5, 3, 4, 5, 6, 7, 1, 2, 4, 6, 4, 4, 5, 7, 2, 6, 5, 7, 6, 7, 7 }; - return DeBruijnBytePos[((U64)((val & -(long long)val) * 0x0218A392CDABBD3FULL)) >> 58]; -# endif - } else /* 32 bits */ { -# if defined(_MSC_VER) && !defined(LIZARD_FORCE_SW_BITCOUNT) - unsigned long r; - _BitScanForward( &r, (U32)val ); - return (int)(r>>3); -# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) - return (__builtin_ctz((U32)val) >> 3); -# else - static const int DeBruijnBytePos[32] = { 0, 0, 3, 0, 3, 1, 3, 0, 3, 2, 2, 1, 3, 2, 0, 1, 3, 3, 1, 2, 2, 2, 2, 0, 3, 1, 2, 0, 1, 0, 1, 1 }; - return DeBruijnBytePos[((U32)((val & -(S32)val) * 0x077CB531U)) >> 27]; -# endif - } - } else /* Big Endian CPU */ { - if (MEM_64bits()) { -# if defined(_MSC_VER) && defined(_WIN64) && !defined(LIZARD_FORCE_SW_BITCOUNT) - unsigned long r = 0; - _BitScanReverse64( &r, val ); - return (unsigned)(r>>3); -# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) - return (__builtin_clzll((U64)val) >> 3); -# else - unsigned r; - if (!(val>>32)) { r=4; } else { r=0; val>>=32; } - if (!(val>>16)) { r+=2; val>>=8; } else { val>>=24; } - r += (!val); - return r; -# endif - } else /* 32 bits */ { -# if defined(_MSC_VER) && !defined(LIZARD_FORCE_SW_BITCOUNT) - unsigned long r = 0; - _BitScanReverse( &r, (unsigned long)val ); - return (unsigned)(r>>3); -# elif (defined(__clang__) || (LIZARD_GCC_VERSION >= 304)) && !defined(LIZARD_FORCE_SW_BITCOUNT) - return (__builtin_clz((U32)val) >> 3); -# else - unsigned r; - if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; } - r += (!val); - return r; -# endif - } - } -} - -MEM_STATIC unsigned Lizard_count(const BYTE* pIn, const BYTE* pMatch, const BYTE* pInLimit) -{ - const BYTE* const pStart = pIn; - - while (likely(pIn - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOTLizard_hash4Ptr - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - - -/* ************************************* -* Includes -***************************************/ -#include "lizard_compress.h" -#include "lizard_common.h" -#include -#include // intptr_t -#ifndef USE_LZ4_ONLY - #ifdef LIZARD_USE_TEST - #include "test/lizard_common_test.h" - #include "test/lizard_compress_test.h" - #else - #include "lizard_compress_liz.h" - #endif -#endif -#include "lizard_compress_lz4.h" -#include "entropy/huf.h" - - -/* ************************************* -* Local Macros -***************************************/ -#define DELTANEXT(p) chainTable[(p) & contentMask] -#define LIZARD_MINIMAL_HUFF_GAIN(comprSize) (comprSize + (comprSize/8) + 512) -#define LIZARD_MINIMAL_BLOCK_GAIN(comprSize) (comprSize + (comprSize/32) + 512) - - -/*-************************************ -* Local Utils -**************************************/ -int Lizard_versionNumber (void) { return LIZARD_VERSION_NUMBER; } -int Lizard_compressBound(int isize) { return LIZARD_COMPRESSBOUND(isize); } -int Lizard_sizeofState_MinLevel() { return Lizard_sizeofState(LIZARD_MIN_CLEVEL); } - - - -/* ************************************* -* Hash functions -***************************************/ -#define HASH_UPDATE_LIMIT 8 /* equal to MEM_read64 */ -static const U32 prime4bytes = 2654435761U; -static const U64 prime5bytes = 889523592379ULL; -static const U64 prime6bytes = 227718039650203ULL; -static const U64 prime7bytes = 58295818150454627ULL; - -#if MINMATCH == 3 -static const U32 prime3bytes = 506832829U; -static U32 Lizard_hash3(U32 u, U32 h) { return (u * prime3bytes) << (32-24) >> (32-h) ; } -static size_t Lizard_hash3Ptr(const void* ptr, U32 h) { return Lizard_hash3(MEM_read32(ptr), h); } -#endif - -static U32 Lizard_hash4(U32 u, U32 h) { return (u * prime4bytes) >> (32-h) ; } -static size_t Lizard_hash4Ptr(const void* ptr, U32 h) { return Lizard_hash4(MEM_read32(ptr), h); } - -static size_t Lizard_hash5(U64 u, U32 h) { return (size_t)((u * prime5bytes) << (64-40) >> (64-h)) ; } -static size_t Lizard_hash5Ptr(const void* p, U32 h) { return Lizard_hash5(MEM_read64(p), h); } - -static size_t Lizard_hash6(U64 u, U32 h) { return (size_t)((u * prime6bytes) << (64-48) >> (64-h)) ; } -static size_t Lizard_hash6Ptr(const void* p, U32 h) { return Lizard_hash6(MEM_read64(p), h); } - -static size_t Lizard_hash7(U64 u, U32 h) { return (size_t)((u * prime7bytes) << (64-56) >> (64-h)) ; } -static size_t Lizard_hash7Ptr(const void* p, U32 h) { return Lizard_hash7(MEM_read64(p), h); } - -static size_t Lizard_hashPtr(const void* p, U32 hBits, U32 mls) -{ - switch(mls) - { - default: - case 4: return Lizard_hash4Ptr(p, hBits); - case 5: return Lizard_hash5Ptr(p, hBits); - case 6: return Lizard_hash6Ptr(p, hBits); - case 7: return Lizard_hash7Ptr(p, hBits); - } -} - - - - -/************************************** -* Internal functions -**************************************/ -/** Lizard_count_2segments() : -* can count match length with `ip` & `match` in 2 different segments. -* convention : on reaching mEnd, match count continue starting from iStart -*/ -static size_t Lizard_count_2segments(const BYTE* ip, const BYTE* match, const BYTE* iEnd, const BYTE* mEnd, const BYTE* iStart) -{ - const BYTE* const vEnd = MIN( ip + (mEnd - match), iEnd); - size_t const matchLength = Lizard_count(ip, match, vEnd); - if (match + matchLength != mEnd) return matchLength; - return matchLength + Lizard_count(ip+matchLength, iStart, iEnd); -} - - -void Lizard_initBlock(Lizard_stream_t* ctx) -{ - ctx->offset16Ptr = ctx->offset16Base; - ctx->offset24Ptr = ctx->offset24Base; - ctx->lenPtr = ctx->lenBase; - ctx->literalsPtr = ctx->literalsBase; - ctx->flagsPtr = ctx->flagsBase; - ctx->last_off = LIZARD_INIT_LAST_OFFSET; /* reset last offset */ -} - - -FORCE_INLINE int Lizard_writeStream(int useHuff, Lizard_stream_t* ctx, BYTE* streamPtr, uint32_t streamLen, BYTE** op, BYTE* oend) -{ - if (useHuff && streamLen > 1024) { -#ifndef LIZARD_NO_HUFFMAN - int useHuffBuf; - if (*op + 6 > oend) { LIZARD_LOG_COMPRESS("*op[%p] + 6 > oend[%p]\n", *op, oend); return -1; } - - useHuffBuf = ((size_t)(oend - (*op + 6)) < HUF_compressBound(streamLen)); - if (useHuffBuf) { - if (streamLen > LIZARD_BLOCK_SIZE) { LIZARD_LOG_COMPRESS("streamLen[%d] > LIZARD_BLOCK_SIZE\n", streamLen); return -1; } - ctx->comprStreamLen = (U32)HUF_compress(ctx->huffBase, ctx->huffEnd - ctx->huffBase, streamPtr, streamLen); - } else { - ctx->comprStreamLen = (U32)HUF_compress(*op + 6, oend - (*op + 6), streamPtr, streamLen); - } - - if (!HUF_isError(ctx->comprStreamLen)) { - if (ctx->comprStreamLen > 0 && (LIZARD_MINIMAL_HUFF_GAIN(ctx->comprStreamLen) < streamLen)) { /* compressible */ - MEM_writeLE24(*op, streamLen); - MEM_writeLE24(*op+3, ctx->comprStreamLen); - if (useHuffBuf) { - if ((size_t)(oend - (*op + 6)) < ctx->comprStreamLen) { LIZARD_LOG_COMPRESS("*op[%p] oend[%p] comprStreamLen[%d]\n", *op, oend, (int)ctx->comprStreamLen); return -1; } - memcpy(*op + 6, ctx->huffBase, ctx->comprStreamLen); - } - *op += ctx->comprStreamLen + 6; - LIZARD_LOG_COMPRESS("HUF_compress streamLen=%d comprStreamLen=%d\n", (int)streamLen, (int)ctx->comprStreamLen); - return 1; - } else { LIZARD_LOG_COMPRESS("HUF_compress ERROR comprStreamLen=%d streamLen=%d\n", (int)ctx->comprStreamLen, (int)streamLen); } - } else { LIZARD_LOG_COMPRESS("HUF_compress ERROR %d: %s\n", (int)ctx->comprStreamLen, HUF_getErrorName(ctx->comprStreamLen)); } -#else - LIZARD_LOG_COMPRESS("compiled with LIZARD_NO_HUFFMAN\n"); - (void)ctx; - return -1; -#endif - } else ctx->comprStreamLen = 0; - - if (*op + 3 + streamLen > oend) { LIZARD_LOG_COMPRESS("*op[%p] + 3 + streamLen[%d] > oend[%p]\n", *op, streamLen, oend); return -1; } - MEM_writeLE24(*op, streamLen); - *op += 3; - memcpy(*op, streamPtr, streamLen); - *op += streamLen; - LIZARD_LOG_COMPRESS("Uncompressed streamLen=%d\n", (int)streamLen); - return 0; -} - - -int Lizard_writeBlock(Lizard_stream_t* ctx, const BYTE* ip, uint32_t inputSize, BYTE** op, BYTE* oend) -{ - int res; - uint32_t flagsLen = (uint32_t)(ctx->flagsPtr - ctx->flagsBase); - uint32_t literalsLen = (uint32_t)(ctx->literalsPtr - ctx->literalsBase); - uint32_t lenLen = (uint32_t)(ctx->lenPtr - ctx->lenBase); - uint32_t offset16Len = (uint32_t)(ctx->offset16Ptr - ctx->offset16Base); - uint32_t offset24Len = (uint32_t)(ctx->offset24Ptr - ctx->offset24Base); - uint32_t sum = flagsLen + literalsLen + lenLen + offset16Len + offset24Len; -#ifdef LIZARD_USE_LOGS - uint32_t comprFlagsLen, comprLiteralsLen; -#endif - - BYTE* start = *op; - - if ((literalsLen < WILDCOPYLENGTH) || (sum+5*3+1 > inputSize)) goto _write_uncompressed; - - *start = 0; - *op += 1; - - res = Lizard_writeStream(0, ctx, ctx->lenBase, lenLen, op, oend); - if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_LEN); - - res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_OFFSET16, ctx, ctx->offset16Base, offset16Len, op, oend); - if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_OFFSET16); - - res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_OFFSET24, ctx, ctx->offset24Base, offset24Len, op, oend); - if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_OFFSET24); - - res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_FLAGS, ctx, ctx->flagsBase, flagsLen, op, oend); - if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_FLAGS); -#ifdef LIZARD_USE_LOGS - comprFlagsLen = ctx->comprStreamLen; -#endif - - res = Lizard_writeStream(ctx->huffType&LIZARD_FLAG_LITERALS, ctx, ctx->literalsBase, literalsLen, op, oend); - if (res < 0) goto _output_error; else *start += (BYTE)(res*LIZARD_FLAG_LITERALS); -#ifdef LIZARD_USE_LOGS - comprLiteralsLen = ctx->comprStreamLen; - sum = (int)(*op-start); -#endif - - if (LIZARD_MINIMAL_BLOCK_GAIN((uint32_t)(*op-start)) > inputSize) goto _write_uncompressed; - - LIZARD_LOG_COMPRESS("%d: total=%d block=%d flagsLen[%.2f%%]=%d comprFlagsLen[%.2f%%]=%d literalsLen[%.2f%%]=%d comprLiteralsLen[%.2f%%]=%d lenLen=%d offset16Len[%.2f%%]=%d offset24Len[%.2f%%]=%d\n", (int)(ip - ctx->srcBase), - (int)(*op - ctx->destBase), sum, (flagsLen*100.0)/sum, flagsLen, (comprFlagsLen*100.0)/sum, comprFlagsLen, (literalsLen*100.0)/sum, literalsLen, (comprLiteralsLen*100.0)/sum, comprLiteralsLen, - lenLen, (offset16Len*100.0)/sum, offset16Len, (offset24Len*100.0)/sum, offset24Len); - return 0; - -_write_uncompressed: - LIZARD_LOG_COMPRESS("%d: total=%d block=%d UNCOMPRESSED inputSize=%u outSize=%d\n", (int)(ip - ctx->srcBase), - (int)(*op - ctx->destBase), (int)(*op-start), inputSize, (int)(oend-start)); - if ((uint32_t)(oend - start) < inputSize + 4) goto _output_error; - *start = LIZARD_FLAG_UNCOMPRESSED; - *op = start + 1; - MEM_writeLE24(*op, inputSize); - *op += 3; - memcpy(*op, ip, inputSize); - *op += inputSize; - return 0; - -_output_error: - LIZARD_LOG_COMPRESS("Lizard_writeBlock ERROR size=%d/%d flagsLen=%d literalsLen=%d lenLen=%d offset16Len=%d offset24Len=%d\n", (int)(*op-start), (int)(oend-start), flagsLen, literalsLen, lenLen, offset16Len, offset24Len); - return 1; -} - - -FORCE_INLINE int Lizard_encodeSequence ( - Lizard_stream_t* ctx, - const BYTE** ip, - const BYTE** anchor, - size_t matchLength, - const BYTE* const match) -{ -#ifdef USE_LZ4_ONLY - return Lizard_encodeSequence_LZ4(ctx, ip, anchor, matchLength, match); -#else - if (ctx->params.decompressType == Lizard_coderwords_LZ4) - return Lizard_encodeSequence_LZ4(ctx, ip, anchor, matchLength, match); - - return Lizard_encodeSequence_LIZv1(ctx, ip, anchor, matchLength, match); -#endif -} - - -FORCE_INLINE int Lizard_encodeLastLiterals ( - Lizard_stream_t* ctx, - const BYTE** ip, - const BYTE** anchor) -{ - LIZARD_LOG_COMPRESS("Lizard_encodeLastLiterals Lizard_coderwords_LZ4=%d\n", ctx->params.decompressType == Lizard_coderwords_LZ4); -#ifdef USE_LZ4_ONLY - return Lizard_encodeLastLiterals_LZ4(ctx, ip, anchor); -#else - if (ctx->params.decompressType == Lizard_coderwords_LZ4) - return Lizard_encodeLastLiterals_LZ4(ctx, ip, anchor); - - return Lizard_encodeLastLiterals_LIZv1(ctx, ip, anchor); -#endif -} - - -/************************************** -* Include parsers -**************************************/ -#include "lizard_parser_hashchain.h" -#include "lizard_parser_nochain.h" -#include "lizard_parser_fast.h" -#include "lizard_parser_fastsmall.h" -#include "lizard_parser_fastbig.h" -#ifndef USE_LZ4_ONLY - #include "lizard_parser_optimal.h" - #include "lizard_parser_lowestprice.h" - #include "lizard_parser_pricefast.h" -#endif - - -int Lizard_verifyCompressionLevel(int compressionLevel) -{ - if (compressionLevel > LIZARD_MAX_CLEVEL) compressionLevel = LIZARD_MAX_CLEVEL; - if (compressionLevel < LIZARD_MIN_CLEVEL) compressionLevel = LIZARD_DEFAULT_CLEVEL; - return compressionLevel; -} - - -int Lizard_sizeofState(int compressionLevel) -{ - Lizard_parameters params; - U32 hashTableSize, chainTableSize; - - compressionLevel = Lizard_verifyCompressionLevel(compressionLevel); - params = Lizard_defaultParameters[compressionLevel - LIZARD_MIN_CLEVEL]; -// hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog3)+((size_t)1 << params.hashLog))); - hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog))); - chainTableSize = (U32)(sizeof(U32)*((size_t)1 << params.contentLog)); - - return sizeof(Lizard_stream_t) + hashTableSize + chainTableSize + LIZARD_COMPRESS_ADD_BUF + (int)LIZARD_COMPRESS_ADD_HUF; -} - - -static void Lizard_init(Lizard_stream_t* ctx, const BYTE* start) -{ - // No need to use memset() on tables as values are always bound checked -#ifdef LIZARD_RESET_MEM - MEM_INIT((void*)ctx->hashTable, 0, ctx->hashTableSize); - MEM_INIT(ctx->chainTable, 0x01, ctx->chainTableSize); -#endif - // printf("memset hashTable=%p hashEnd=%p chainTable=%p chainEnd=%p\n", ctx->hashTable, ((BYTE*)ctx->hashTable) + ctx->hashTableSize, ctx->chainTable, ((BYTE*)ctx->chainTable)+ctx->chainTableSize); - ctx->nextToUpdate = LIZARD_DICT_SIZE; - ctx->base = start - LIZARD_DICT_SIZE; - ctx->end = start; - ctx->dictBase = start - LIZARD_DICT_SIZE; - ctx->dictLimit = LIZARD_DICT_SIZE; - ctx->lowLimit = LIZARD_DICT_SIZE; - ctx->last_off = LIZARD_INIT_LAST_OFFSET; - ctx->litSum = 0; -} - - -/* if ctx==NULL memory is allocated and returned as value */ -Lizard_stream_t* Lizard_initStream(Lizard_stream_t* ctx, int compressionLevel) -{ - Lizard_parameters params; - U32 hashTableSize, chainTableSize; - void *tempPtr; - - compressionLevel = Lizard_verifyCompressionLevel(compressionLevel); - params = Lizard_defaultParameters[compressionLevel - LIZARD_MIN_CLEVEL]; -// hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog3)+((size_t)1 << params.hashLog))); - hashTableSize = (U32)(sizeof(U32)*(((size_t)1 << params.hashLog))); - chainTableSize = (U32)(sizeof(U32)*((size_t)1 << params.contentLog)); - - if (!ctx) - { - ctx = (Lizard_stream_t*)malloc(sizeof(Lizard_stream_t) + hashTableSize + chainTableSize + LIZARD_COMPRESS_ADD_BUF + LIZARD_COMPRESS_ADD_HUF); - if (!ctx) { printf("ERROR: Cannot allocate %d MB (compressionLevel=%d)\n", (int)(sizeof(Lizard_stream_t) + hashTableSize + chainTableSize)>>20, compressionLevel); return 0; } - LIZARD_LOG_COMPRESS("Allocated %d MB (compressionLevel=%d)\n", (int)(sizeof(Lizard_stream_t) + hashTableSize + chainTableSize)>>20, compressionLevel); - ctx->allocatedMemory = sizeof(Lizard_stream_t) + hashTableSize + chainTableSize + LIZARD_COMPRESS_ADD_BUF + (U32)LIZARD_COMPRESS_ADD_HUF; - // printf("malloc from=%p to=%p hashTable=%p hashEnd=%p chainTable=%p chainEnd=%p\n", ctx, ((BYTE*)ctx)+sizeof(Lizard_stream_t) + hashTableSize + chainTableSize, ctx->hashTable, ((BYTE*)ctx->hashTable) + hashTableSize, ctx->chainTable, ((BYTE*)ctx->chainTable)+chainTableSize); - } - - tempPtr = ctx; - ctx->hashTable = (U32*)(tempPtr) + sizeof(Lizard_stream_t)/4; - ctx->hashTableSize = hashTableSize; - ctx->chainTable = ctx->hashTable + hashTableSize/4; - ctx->chainTableSize = chainTableSize; - ctx->params = params; - ctx->compressionLevel = (unsigned)compressionLevel; - if (compressionLevel < 30) - ctx->huffType = 0; - else - ctx->huffType = LIZARD_FLAG_LITERALS + LIZARD_FLAG_FLAGS; // + LIZARD_FLAG_OFFSET16 + LIZARD_FLAG_OFFSET24; - - ctx->literalsBase = (BYTE*)ctx->hashTable + ctx->hashTableSize + ctx->chainTableSize; - ctx->flagsBase = ctx->literalsEnd = ctx->literalsBase + LIZARD_BLOCK_SIZE_PAD; - ctx->lenBase = ctx->flagsEnd = ctx->flagsBase + LIZARD_BLOCK_SIZE_PAD; - ctx->offset16Base = ctx->lenEnd = ctx->lenBase + LIZARD_BLOCK_SIZE_PAD; - ctx->offset24Base = ctx->offset16End = ctx->offset16Base + LIZARD_BLOCK_SIZE_PAD; - ctx->huffBase = ctx->offset24End = ctx->offset24Base + LIZARD_BLOCK_SIZE_PAD; - ctx->huffEnd = ctx->huffBase + LIZARD_COMPRESS_ADD_HUF; - - return ctx; -} - - - -Lizard_stream_t* Lizard_createStream(int compressionLevel) -{ - Lizard_stream_t* ctx = Lizard_initStream(NULL, compressionLevel); - if (ctx) ctx->base = NULL; - return ctx; -} - - -/* initialization */ -Lizard_stream_t* Lizard_resetStream(Lizard_stream_t* ctx, int compressionLevel) -{ - size_t wanted = Lizard_sizeofState(compressionLevel); - - if (ctx->allocatedMemory < wanted) { - Lizard_freeStream(ctx); - ctx = Lizard_createStream(compressionLevel); - } else { - Lizard_initStream(ctx, compressionLevel); - } - - if (ctx) ctx->base = NULL; - return ctx; -} - - -int Lizard_freeStream(Lizard_stream_t* ctx) -{ - if (ctx) { - free(ctx); - } - return 0; -} - - -int Lizard_loadDict(Lizard_stream_t* Lizard_streamPtr, const char* dictionary, int dictSize) -{ - Lizard_stream_t* ctxPtr = (Lizard_stream_t*) Lizard_streamPtr; - if (dictSize > LIZARD_DICT_SIZE) { - dictionary += dictSize - LIZARD_DICT_SIZE; - dictSize = LIZARD_DICT_SIZE; - } - Lizard_init(ctxPtr, (const BYTE*)dictionary); - if (dictSize >= HASH_UPDATE_LIMIT) Lizard_Insert (ctxPtr, (const BYTE*)dictionary + (dictSize - (HASH_UPDATE_LIMIT-1))); - ctxPtr->end = (const BYTE*)dictionary + dictSize; - return dictSize; -} - - -static void Lizard_setExternalDict(Lizard_stream_t* ctxPtr, const BYTE* newBlock) -{ - if (ctxPtr->end >= ctxPtr->base + HASH_UPDATE_LIMIT) Lizard_Insert (ctxPtr, ctxPtr->end - (HASH_UPDATE_LIMIT-1)); /* Referencing remaining dictionary content */ - /* Only one memory segment for extDict, so any previous extDict is lost at this stage */ - ctxPtr->lowLimit = ctxPtr->dictLimit; - ctxPtr->dictLimit = (U32)(ctxPtr->end - ctxPtr->base); - ctxPtr->dictBase = ctxPtr->base; - ctxPtr->base = newBlock - ctxPtr->dictLimit; - ctxPtr->end = newBlock; - ctxPtr->nextToUpdate = ctxPtr->dictLimit; /* match referencing will resume from there */ -} - - -/* dictionary saving */ -int Lizard_saveDict (Lizard_stream_t* Lizard_streamPtr, char* safeBuffer, int dictSize) -{ - Lizard_stream_t* const ctx = (Lizard_stream_t*)Lizard_streamPtr; - int const prefixSize = (int)(ctx->end - (ctx->base + ctx->dictLimit)); - if (dictSize > LIZARD_DICT_SIZE) dictSize = LIZARD_DICT_SIZE; - if (dictSize < 4) dictSize = 0; - if (dictSize > prefixSize) dictSize = prefixSize; - memmove(safeBuffer, ctx->end - dictSize, dictSize); - { U32 const endIndex = (U32)(ctx->end - ctx->base); - ctx->end = (const BYTE*)safeBuffer + dictSize; - ctx->base = ctx->end - endIndex; - ctx->dictLimit = endIndex - dictSize; - ctx->lowLimit = endIndex - dictSize; - if (ctx->nextToUpdate < ctx->dictLimit) ctx->nextToUpdate = ctx->dictLimit; - } - return dictSize; -} - -FORCE_INLINE int Lizard_compress_generic ( - void* ctxvoid, - const char* source, - char* dest, - int inputSize, - int maxOutputSize) -{ - Lizard_stream_t* ctx = (Lizard_stream_t*) ctxvoid; - size_t dictSize = (size_t)(ctx->end - ctx->base) - ctx->dictLimit; - const BYTE* ip = (const BYTE*) source; - BYTE* op = (BYTE*) dest; - BYTE* const oend = op + maxOutputSize; - int res; - - (void)dictSize; - LIZARD_LOG_COMPRESS("Lizard_compress_generic source=%p inputSize=%d dest=%p maxOutputSize=%d cLevel=%d dictBase=%p dictSize=%d\n", source, inputSize, dest, maxOutputSize, ctx->compressionLevel, ctx->dictBase, (int)dictSize); - *op++ = (BYTE)ctx->compressionLevel; - maxOutputSize--; // can be lower than 0 - ctx->end += inputSize; - ctx->srcBase = ctx->off24pos = ip; - ctx->destBase = (BYTE*)dest; - - while (inputSize > 0) - { - int inputPart = MIN(LIZARD_BLOCK_SIZE, inputSize); - - if (ctx->huffType) Lizard_rescaleFreqs(ctx); - Lizard_initBlock(ctx); - ctx->diffBase = ip; - - switch(ctx->params.parserType) - { - default: - case Lizard_parser_fastSmall: - res = Lizard_compress_fastSmall(ctx, ip, ip+inputPart); break; - case Lizard_parser_fast: - res = Lizard_compress_fast(ctx, ip, ip+inputPart); break; - case Lizard_parser_noChain: - res = Lizard_compress_noChain(ctx, ip, ip+inputPart); break; - case Lizard_parser_hashChain: - res = Lizard_compress_hashChain(ctx, ip, ip+inputPart); break; -#ifndef USE_LZ4_ONLY - case Lizard_parser_fastBig: - res = Lizard_compress_fastBig(ctx, ip, ip+inputPart); break; - case Lizard_parser_priceFast: - res = Lizard_compress_priceFast(ctx, ip, ip+inputPart); break; - case Lizard_parser_lowestPrice: - res = Lizard_compress_lowestPrice(ctx, ip, ip+inputPart); break; - case Lizard_parser_optimalPrice: - case Lizard_parser_optimalPriceBT: - res = Lizard_compress_optimalPrice(ctx, ip, ip+inputPart); break; -#else - case Lizard_parser_priceFast: - case Lizard_parser_lowestPrice: - case Lizard_parser_optimalPrice: - case Lizard_parser_optimalPriceBT: - res = 0; -#endif - } - - LIZARD_LOG_COMPRESS("Lizard_compress_generic res=%d inputPart=%d \n", res, inputPart); - if (res <= 0) return res; - - if (Lizard_writeBlock(ctx, ip, inputPart, &op, oend)) goto _output_error; - - ip += inputPart; - inputSize -= inputPart; - LIZARD_LOG_COMPRESS("Lizard_compress_generic in=%d out=%d\n", (int)(ip-(const BYTE*)source), (int)(op-(BYTE*)dest)); - } - - LIZARD_LOG_COMPRESS("Lizard_compress_generic total=%d\n", (int)(op-(BYTE*)dest)); - return (int)(op-(BYTE*)dest); -_output_error: - LIZARD_LOG_COMPRESS("Lizard_compress_generic ERROR\n"); - return 0; -} - - -int Lizard_compress_continue (Lizard_stream_t* ctxPtr, - const char* source, char* dest, - int inputSize, int maxOutputSize) -{ - /* auto-init if required */ - if (ctxPtr->base == NULL) Lizard_init(ctxPtr, (const BYTE*) source); - - /* Check overflow */ - if ((size_t)(ctxPtr->end - ctxPtr->base) > 2 GB) { - size_t dictSize = (size_t)(ctxPtr->end - ctxPtr->base) - ctxPtr->dictLimit; - if (dictSize > LIZARD_DICT_SIZE) dictSize = LIZARD_DICT_SIZE; - Lizard_loadDict((Lizard_stream_t*)ctxPtr, (const char*)(ctxPtr->end) - dictSize, (int)dictSize); - } - - /* Check if blocks follow each other */ - if ((const BYTE*)source != ctxPtr->end) - Lizard_setExternalDict(ctxPtr, (const BYTE*)source); - - /* Check overlapping input/dictionary space */ - { const BYTE* sourceEnd = (const BYTE*) source + inputSize; - const BYTE* const dictBegin = ctxPtr->dictBase + ctxPtr->lowLimit; - const BYTE* const dictEnd = ctxPtr->dictBase + ctxPtr->dictLimit; - if ((sourceEnd > dictBegin) && ((const BYTE*)source < dictEnd)) { - if (sourceEnd > dictEnd) sourceEnd = dictEnd; - ctxPtr->lowLimit = (U32)(sourceEnd - ctxPtr->dictBase); - if (ctxPtr->dictLimit - ctxPtr->lowLimit < 4) ctxPtr->lowLimit = ctxPtr->dictLimit; - } - } - - return Lizard_compress_generic (ctxPtr, source, dest, inputSize, maxOutputSize); -} - - -int Lizard_compress_extState (void* state, const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel) -{ - Lizard_stream_t* ctx = (Lizard_stream_t*) state; - if (((size_t)(state)&(sizeof(void*)-1)) != 0) return 0; /* Error : state is not aligned for pointers (32 or 64 bits) */ - - /* initialize stream */ - Lizard_initStream(ctx, compressionLevel); - Lizard_init((Lizard_stream_t*)state, (const BYTE*)src); - - return Lizard_compress_generic (state, src, dst, srcSize, maxDstSize); -} - - -int Lizard_compress(const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel) -{ - int cSize; - Lizard_stream_t* statePtr = Lizard_createStream(compressionLevel); - - if (!statePtr) return 0; - cSize = Lizard_compress_extState(statePtr, src, dst, srcSize, maxDstSize, compressionLevel); - - Lizard_freeStream(statePtr); - return cSize; -} - - -/************************************** -* Level1 functions -**************************************/ -int Lizard_compress_extState_MinLevel(void* state, const char* source, char* dest, int inputSize, int maxOutputSize) -{ - return Lizard_compress_extState(state, source, dest, inputSize, maxOutputSize, LIZARD_MIN_CLEVEL); -} - -int Lizard_compress_MinLevel(const char* source, char* dest, int inputSize, int maxOutputSize) -{ - return Lizard_compress(source, dest, inputSize, maxOutputSize, LIZARD_MIN_CLEVEL); -} - -Lizard_stream_t* Lizard_createStream_MinLevel(void) -{ - return Lizard_createStream(LIZARD_MIN_CLEVEL); -} - -Lizard_stream_t* Lizard_resetStream_MinLevel(Lizard_stream_t* Lizard_stream) -{ - return Lizard_resetStream (Lizard_stream, LIZARD_MIN_CLEVEL); -} diff --git a/contrib/lizard/lib/lizard_compress.h b/contrib/lizard/lib/lizard_compress.h deleted file mode 100644 index d84246e2099..00000000000 --- a/contrib/lizard/lib/lizard_compress.h +++ /dev/null @@ -1,208 +0,0 @@ -/* - Lizard - Fast LZ compression algorithm - Header File - Copyright (C) 2011-2016, Yann Collet - Copyright (C) 2016-2017, Przemyslaw Skibinski - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ -#ifndef LIZARD_H_2983 -#define LIZARD_H_2983 - -#if defined (__cplusplus) -extern "C" { -#endif - -/* - * lizard_compress.h provides block compression functions. It gives full buffer control to user. - * Block compression functions are not-enough to send information, - * since it's still necessary to provide metadata (such as compressed size), - * and each application can do it in whichever way it wants. - * For interoperability, there is Lizard frame specification (lizard_Frame_format.md). - * A library is provided to take care of it, see lizard_frame.h. -*/ - - -/*^*************************************************************** -* Export parameters -*****************************************************************/ -/* -* LIZARD_DLL_EXPORT : -* Enable exporting of functions when building a Windows DLL -*/ -#if defined(LIZARD_DLL_EXPORT) && (LIZARD_DLL_EXPORT==1) -# define LIZARDLIB_API __declspec(dllexport) -#elif defined(LIZARD_DLL_IMPORT) && (LIZARD_DLL_IMPORT==1) -# define LIZARDLIB_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ -#else -# define LIZARDLIB_API -#endif - - -/*-************************************ -* Version -**************************************/ -#define LIZARD_VERSION_MAJOR 1 /* for breaking interface changes */ -#define LIZARD_VERSION_MINOR 0 /* for new (non-breaking) interface capabilities */ -#define LIZARD_VERSION_RELEASE 0 /* for tweaks, bug-fixes, or development */ - -#define LIZARD_VERSION_NUMBER (LIZARD_VERSION_MAJOR *100*100 + LIZARD_VERSION_MINOR *100 + LIZARD_VERSION_RELEASE) -int Lizard_versionNumber (void); - -#define LIZARD_LIB_VERSION LIZARD_VERSION_MAJOR.LIZARD_VERSION_MINOR.LIZARD_VERSION_RELEASE -#define LIZARD_QUOTE(str) #str -#define LIZARD_EXPAND_AND_QUOTE(str) LIZARD_QUOTE(str) -#define LIZARD_VERSION_STRING LIZARD_EXPAND_AND_QUOTE(LIZARD_LIB_VERSION) -const char* Lizard_versionString (void); - -typedef struct Lizard_stream_s Lizard_stream_t; - -#define LIZARD_MIN_CLEVEL 10 /* minimum compression level */ -#ifndef LIZARD_NO_HUFFMAN - #define LIZARD_MAX_CLEVEL 49 /* maximum compression level */ -#else - #define LIZARD_MAX_CLEVEL 29 /* maximum compression level */ -#endif -#define LIZARD_DEFAULT_CLEVEL 17 - - -/*-************************************ -* Simple Functions -**************************************/ - -LIZARDLIB_API int Lizard_compress (const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel); - -/* -Lizard_compress() : - Compresses 'sourceSize' bytes from buffer 'source' - into already allocated 'dest' buffer of size 'maxDestSize'. - Compression is guaranteed to succeed if 'maxDestSize' >= Lizard_compressBound(sourceSize). - It also runs faster, so it's a recommended setting. - If the function cannot compress 'source' into a more limited 'dest' budget, - compression stops *immediately*, and the function result is zero. - As a consequence, 'dest' content is not valid. - This function never writes outside 'dest' buffer, nor read outside 'source' buffer. - sourceSize : Max supported value is LIZARD_MAX_INPUT_VALUE - maxDestSize : full or partial size of buffer 'dest' (which must be already allocated) - return : the number of bytes written into buffer 'dest' (necessarily <= maxOutputSize) - or 0 if compression fails -*/ - - -/*-************************************ -* Advanced Functions -**************************************/ -#define LIZARD_MAX_INPUT_SIZE 0x7E000000 /* 2 113 929 216 bytes */ -#define LIZARD_BLOCK_SIZE (1<<17) -#define LIZARD_BLOCK64K_SIZE (1<<16) -#define LIZARD_COMPRESSBOUND(isize) ((unsigned)(isize) > (unsigned)LIZARD_MAX_INPUT_SIZE ? 0 : (isize) + 1 + 1 + ((isize/LIZARD_BLOCK_SIZE)+1)*4) - - -/*! -Lizard_compressBound() : - Provides the maximum size that Lizard compression may output in a "worst case" scenario (input data not compressible) - This function is primarily useful for memory allocation purposes (destination buffer size). - Macro LIZARD_COMPRESSBOUND() is also provided for compilation-time evaluation (stack memory allocation for example). - Note that Lizard_compress() compress faster when dest buffer size is >= Lizard_compressBound(srcSize) - inputSize : max supported value is LIZARD_MAX_INPUT_SIZE - return : maximum output size in a "worst case" scenario - or 0, if input size is too large ( > LIZARD_MAX_INPUT_SIZE) -*/ -LIZARDLIB_API int Lizard_compressBound(int inputSize); - - -/*! -Lizard_compress_extState() : - Same compression function, just using an externally allocated memory space to store compression state. - Use Lizard_sizeofState() to know how much memory must be allocated, - and allocate it on 8-bytes boundaries (using malloc() typically). - Then, provide it as 'void* state' to compression function. -*/ -LIZARDLIB_API int Lizard_sizeofState(int compressionLevel); - -LIZARDLIB_API int Lizard_compress_extState(void* state, const char* src, char* dst, int srcSize, int maxDstSize, int compressionLevel); - - - -/*-********************************************* -* Streaming Compression Functions -***********************************************/ - -/*! Lizard_createStream() will allocate and initialize an `Lizard_stream_t` structure. - * Lizard_freeStream() releases its memory. - * In the context of a DLL (liblizard), please use these methods rather than the static struct. - * They are more future proof, in case of a change of `Lizard_stream_t` size. - */ -LIZARDLIB_API Lizard_stream_t* Lizard_createStream(int compressionLevel); -LIZARDLIB_API int Lizard_freeStream (Lizard_stream_t* streamPtr); - - -/*! Lizard_resetStream() : - * Use this function to reset/reuse an allocated `Lizard_stream_t` structure - */ -LIZARDLIB_API Lizard_stream_t* Lizard_resetStream (Lizard_stream_t* streamPtr, int compressionLevel); - - -/*! Lizard_loadDict() : - * Use this function to load a static dictionary into Lizard_stream. - * Any previous data will be forgotten, only 'dictionary' will remain in memory. - * Loading a size of 0 is allowed. - * Return : dictionary size, in bytes (necessarily <= LIZARD_DICT_SIZE) - */ -LIZARDLIB_API int Lizard_loadDict (Lizard_stream_t* streamPtr, const char* dictionary, int dictSize); - - -/*! Lizard_compress_continue() : - * Compress buffer content 'src', using data from previously compressed blocks as dictionary to improve compression ratio. - * Important : Previous data blocks are assumed to still be present and unmodified ! - * 'dst' buffer must be already allocated. - * If maxDstSize >= Lizard_compressBound(srcSize), compression is guaranteed to succeed, and runs faster. - * If not, and if compressed data cannot fit into 'dst' buffer size, compression stops, and function returns a zero. - */ -LIZARDLIB_API int Lizard_compress_continue (Lizard_stream_t* streamPtr, const char* src, char* dst, int srcSize, int maxDstSize); - - -/*! Lizard_saveDict() : - * If previously compressed data block is not guaranteed to remain available at its memory location, - * save it into a safer place (char* safeBuffer). - * Note : you don't need to call Lizard_loadDict() afterwards, - * dictionary is immediately usable, you can therefore call Lizard_compress_continue(). - * Return : saved dictionary size in bytes (necessarily <= dictSize), or 0 if error. - */ -LIZARDLIB_API int Lizard_saveDict (Lizard_stream_t* streamPtr, char* safeBuffer, int dictSize); - - - - - -#if defined (__cplusplus) -} -#endif - -#endif /* LIZARD_H_2983827168210 */ diff --git a/contrib/lizard/lib/lizard_compress_liz.h b/contrib/lizard/lib/lizard_compress_liz.h deleted file mode 100644 index f531f36703a..00000000000 --- a/contrib/lizard/lib/lizard_compress_liz.h +++ /dev/null @@ -1,301 +0,0 @@ -#define LIZARD_FREQ_DIV 5 - -FORCE_INLINE void Lizard_setLog2Prices(Lizard_stream_t* ctx) -{ - ctx->log2LitSum = Lizard_highbit32(ctx->litSum+1); - ctx->log2FlagSum = Lizard_highbit32(ctx->flagSum+1); -} - - -MEM_STATIC void Lizard_rescaleFreqs(Lizard_stream_t* ctx) -{ - unsigned u; - - ctx->cachedLiterals = NULL; - ctx->cachedPrice = ctx->cachedLitLength = 0; - - ctx->litPriceSum = 0; - - if (ctx->litSum == 0) { - ctx->litSum = 2 * 256; - ctx->flagSum = 2 * 256; - - for (u=0; u < 256; u++) { - ctx->litFreq[u] = 2; - ctx->flagFreq[u] = 2; - } - } else { - ctx->litSum = 0; - ctx->flagSum = 0; - - for (u=0; u < 256; u++) { - ctx->litFreq[u] = 1 + (ctx->litFreq[u]>>LIZARD_FREQ_DIV); - ctx->litSum += ctx->litFreq[u]; - ctx->flagFreq[u] = 1 + (ctx->flagFreq[u]>>LIZARD_FREQ_DIV); - ctx->flagSum += ctx->flagFreq[u]; - } - } - - Lizard_setLog2Prices(ctx); -} - - -FORCE_INLINE int Lizard_encodeSequence_LIZv1 ( - Lizard_stream_t* ctx, - const BYTE** ip, - const BYTE** anchor, - size_t matchLength, - const BYTE* const match) -{ - U32 offset = (U32)(*ip - match); - size_t length = (size_t)(*ip - *anchor); - BYTE* token = (ctx->flagsPtr)++; - - if (length > 0 || offset < LIZARD_MAX_16BIT_OFFSET) { - /* Encode Literal length */ - // if ((limitedOutputBuffer) && (ctx->literalsPtr > oend - length - LIZARD_LENGTH_SIZE_LIZv1(length) - WILDCOPYLENGTH)) { LIZARD_LOG_COMPRESS_LIZv1("encodeSequence overflow1\n"); return 1; } /* Check output limit */ - if (length >= MAX_SHORT_LITLEN) - { size_t len; - *token = MAX_SHORT_LITLEN; - len = length - MAX_SHORT_LITLEN; - if (len >= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(len)); ctx->literalsPtr += 4; } - else if (len >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(len)); ctx->literalsPtr += 3; } - else *(ctx->literalsPtr)++ = (BYTE)len; - } - else *token = (BYTE)length; - - /* Copy Literals */ - Lizard_wildCopy(ctx->literalsPtr, *anchor, (ctx->literalsPtr) + length); -#ifndef LIZARD_NO_HUFFMAN - if (ctx->huffType) { - ctx->litSum += (U32)length; - ctx->litPriceSum += (U32)(length * ctx->log2LitSum); - { U32 u; - for (u=0; u < length; u++) { - ctx->litPriceSum -= Lizard_highbit32(ctx->litFreq[ctx->literalsPtr[u]]+1); - ctx->litFreq[ctx->literalsPtr[u]]++; - } } - } -#endif - ctx->literalsPtr += length; - - - if (offset >= LIZARD_MAX_16BIT_OFFSET) { - COMPLOG_CODEWORDS_LIZv1("T32+ literal=%u match=%u offset=%d\n", (U32)length, 0, 0); - *token+=(1<huffType) { - ctx->flagFreq[*token]++; - ctx->flagSum++; - } -#endif - token = (ctx->flagsPtr)++; - } - } - - /* Encode Offset */ - if (offset >= LIZARD_MAX_16BIT_OFFSET) // 24-bit offset - { - if (matchLength < MM_LONGOFF) printf("ERROR matchLength=%d/%d\n", (int)matchLength, MM_LONGOFF), exit(1); - - // if ((limitedOutputBuffer) && (ctx->literalsPtr > oend - 8 /*LIZARD_LENGTH_SIZE_LIZv1(length)*/)) { LIZARD_LOG_COMPRESS_LIZv1("encodeSequence overflow2\n"); return 1; } /* Check output limit */ - if (matchLength - MM_LONGOFF >= LIZARD_LAST_LONG_OFF) - { - size_t len = matchLength - MM_LONGOFF - LIZARD_LAST_LONG_OFF; - *token = LIZARD_LAST_LONG_OFF; - if (len >= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(len)); ctx->literalsPtr += 4; } - else if (len >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(len)); ctx->literalsPtr += 3; } - else *(ctx->literalsPtr)++ = (BYTE)len; - COMPLOG_CODEWORDS_LIZv1("T31 literal=%u match=%u offset=%d\n", 0, (U32)matchLength, offset); - } - else - { - COMPLOG_CODEWORDS_LIZv1("T0-30 literal=%u match=%u offset=%d\n", 0, (U32)matchLength, offset); - *token = (BYTE)(matchLength - MM_LONGOFF); - } - - MEM_writeLE24(ctx->offset24Ptr, offset); - ctx->offset24Ptr += 3; - ctx->last_off = offset; - ctx->off24pos = *ip; - } - else - { - COMPLOG_CODEWORDS_LIZv1("T32+ literal=%u match=%u offset=%d\n", (U32)length, (U32)matchLength, offset); - if (offset == 0) - { - *token+=(1<last_off); exit(1); } - - ctx->last_off = offset; - MEM_writeLE16(ctx->offset16Ptr, (U16)ctx->last_off); ctx->offset16Ptr += 2; - } - - /* Encode MatchLength */ - length = matchLength; - // if ((limitedOutputBuffer) && (ctx->literalsPtr > oend - 5 /*LIZARD_LENGTH_SIZE_LIZv1(length)*/)) { LIZARD_LOG_COMPRESS_LIZv1("encodeSequence overflow2\n"); return 1; } /* Check output limit */ - if (length >= MAX_SHORT_MATCHLEN) { - *token += (BYTE)(MAX_SHORT_MATCHLEN<= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(length)); ctx->literalsPtr += 4; } - else if (length >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(length)); ctx->literalsPtr += 3; } - else *(ctx->literalsPtr)++ = (BYTE)length; - } - else *token += (BYTE)(length<huffType) { - ctx->flagFreq[*token]++; - ctx->flagSum++; - Lizard_setLog2Prices(ctx); - } -#endif - - /* Prepare next loop */ - *ip += matchLength; - *anchor = *ip; - - return 0; -} - - -FORCE_INLINE int Lizard_encodeLastLiterals_LIZv1 ( - Lizard_stream_t* ctx, - const BYTE** ip, - const BYTE** anchor) -{ - size_t length = (int)(*ip - *anchor); - (void)ctx; - - memcpy(ctx->literalsPtr, *anchor, length); - ctx->literalsPtr += length; - return 0; -} - - -#define LIZARD_PRICE_MULT 1 -#define LIZARD_GET_TOKEN_PRICE_LIZv1(token) (LIZARD_PRICE_MULT * (ctx->log2FlagSum - Lizard_highbit32(ctx->flagFreq[token]+1))) - - -FORCE_INLINE size_t Lizard_get_price_LIZv1(Lizard_stream_t* const ctx, int rep, const BYTE *ip, const BYTE *off24pos, size_t litLength, U32 offset, size_t matchLength) -{ - size_t price = 0; - BYTE token = 0; -#ifndef LIZARD_NO_HUFFMAN - const BYTE* literals = ip - litLength; - U32 u; - if ((ctx->huffType) && (ctx->params.parserType != Lizard_parser_lowestPrice)) { - if (ctx->cachedLiterals == literals && litLength >= ctx->cachedLitLength) { - size_t const additional = litLength - ctx->cachedLitLength; - const BYTE* literals2 = ctx->cachedLiterals + ctx->cachedLitLength; - price = ctx->cachedPrice + LIZARD_PRICE_MULT * additional * ctx->log2LitSum; - for (u=0; u < additional; u++) - price -= LIZARD_PRICE_MULT * Lizard_highbit32(ctx->litFreq[literals2[u]]+1); - ctx->cachedPrice = (U32)price; - ctx->cachedLitLength = (U32)litLength; - } else { - price = LIZARD_PRICE_MULT * litLength * ctx->log2LitSum; - for (u=0; u < litLength; u++) - price -= LIZARD_PRICE_MULT * Lizard_highbit32(ctx->litFreq[literals[u]]+1); - - if (litLength >= 12) { - ctx->cachedLiterals = literals; - ctx->cachedPrice = (U32)price; - ctx->cachedLitLength = (U32)litLength; - } - } - } - else - price += 8*litLength; /* Copy Literals */ -#else - price += 8*litLength; /* Copy Literals */ - (void)ip; - (void)ctx; -#endif - - (void)off24pos; - (void)rep; - - if (litLength > 0 || offset < LIZARD_MAX_16BIT_OFFSET) { - /* Encode Literal length */ - if (litLength >= MAX_SHORT_LITLEN) - { size_t len = litLength - MAX_SHORT_LITLEN; - token = MAX_SHORT_LITLEN; - if (len >= (1<<16)) price += 32; - else if (len >= 254) price += 24; - else price += 8; - } - else token = (BYTE)litLength; - - if (offset >= LIZARD_MAX_16BIT_OFFSET) { - token+=(1<huffType && ctx->params.parserType != Lizard_parser_lowestPrice) - price += LIZARD_GET_TOKEN_PRICE_LIZv1(token); - else - price += 8; - } - } - - /* Encode Offset */ - if (offset >= LIZARD_MAX_16BIT_OFFSET) { // 24-bit offset - if (matchLength < MM_LONGOFF) return LIZARD_MAX_PRICE; // error - - if (matchLength - MM_LONGOFF >= LIZARD_LAST_LONG_OFF) { - size_t len = matchLength - MM_LONGOFF - LIZARD_LAST_LONG_OFF; - token = LIZARD_LAST_LONG_OFF; - if (len >= (1<<16)) price += 32; - else if (len >= 254) price += 24; - else price += 8; - } else { - token = (BYTE)(matchLength - MM_LONGOFF); - } - - price += 24; - } else { - size_t length; - if (offset == 0) { - token+=(1<= MAX_SHORT_MATCHLEN) { - token += (BYTE)(MAX_SHORT_MATCHLEN<= (1<<16)) price += 32; - else if (length >= 254) price += 24; - else price += 8; - } - else token += (BYTE)(length< 0 || matchLength > 0) { - int offset_load = Lizard_highbit32(offset); - if (ctx->huffType) { - price += ((offset_load>=20) ? ((offset_load-19)*4) : 0); - price += 4 + (matchLength==1); - } else { - price += ((offset_load>=16) ? ((offset_load-15)*4) : 0); - price += 6 + (matchLength==1); - } - if (ctx->huffType && ctx->params.parserType != Lizard_parser_lowestPrice) - price += LIZARD_GET_TOKEN_PRICE_LIZv1(token); - else - price += 8; - } else { - if (ctx->huffType && ctx->params.parserType != Lizard_parser_lowestPrice) - price += LIZARD_GET_TOKEN_PRICE_LIZv1(token); // 1=better ratio - } - - return price; -} diff --git a/contrib/lizard/lib/lizard_compress_lz4.h b/contrib/lizard/lib/lizard_compress_lz4.h deleted file mode 100644 index 00686b6837e..00000000000 --- a/contrib/lizard/lib/lizard_compress_lz4.h +++ /dev/null @@ -1,162 +0,0 @@ -#define LIZARD_LENGTH_SIZE_LZ4(len) ((len >= (1<<16)+RUN_MASK_LZ4) ? 5 : ((len >= 254+RUN_MASK_LZ4) ? 3 : ((len >= RUN_MASK_LZ4) ? 1 : 0))) - -FORCE_INLINE int Lizard_encodeSequence_LZ4 ( - Lizard_stream_t* ctx, - const BYTE** ip, - const BYTE** anchor, - size_t matchLength, - const BYTE* const match) -{ - size_t length = (size_t)(*ip - *anchor); - BYTE* token = (ctx->flagsPtr)++; - (void) ctx; - - COMPLOG_CODEWORDS_LZ4("literal : %u -- match : %u -- offset : %u\n", (U32)(*ip - *anchor), (U32)matchLength, (U32)(*ip-match)); - - /* Encode Literal length */ - // if (ctx->literalsPtr > ctx->literalsEnd - length - LIZARD_LENGTH_SIZE_LZ4(length) - 2 - WILDCOPYLENGTH) { LIZARD_LOG_COMPRESS_LZ4("encodeSequence overflow1\n"); return 1; } /* Check output limit */ - if (length >= RUN_MASK_LZ4) - { size_t len = length - RUN_MASK_LZ4; - *token = RUN_MASK_LZ4; - if (len >= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(len)); ctx->literalsPtr += 4; } - else if (len >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(len)); ctx->literalsPtr += 3; } - else *(ctx->literalsPtr)++ = (BYTE)len; - } - else *token = (BYTE)length; - - /* Copy Literals */ - if (length > 0) { - Lizard_wildCopy(ctx->literalsPtr, *anchor, (ctx->literalsPtr) + length); -#if 0 //def LIZARD_USE_HUFFMAN - ctx->litSum += (U32)length; - ctx->litPriceSum += (U32)(length * ctx->log2LitSum); - { U32 u; - for (u=0; u < length; u++) { - ctx->litPriceSum -= Lizard_highbit32(ctx->litFreq[ctx->literalsPtr[u]]+1); - ctx->litFreq[ctx->literalsPtr[u]]++; - } } -#endif - ctx->literalsPtr += length; - } - - /* Encode Offset */ - MEM_writeLE16(ctx->literalsPtr, (U16)(*ip-match)); - ctx->literalsPtr+=2; - - /* Encode MatchLength */ - length = matchLength - MINMATCH; - // if (ctx->literalsPtr > ctx->literalsEnd - 5 /*LIZARD_LENGTH_SIZE_LZ4(length)*/) { LIZARD_LOG_COMPRESS_LZ4("encodeSequence overflow2\n"); return 1; } /* Check output limit */ - if (length >= ML_MASK_LZ4) { - *token += (BYTE)(ML_MASK_LZ4<= (1<<16)) { *(ctx->literalsPtr) = 255; MEM_writeLE24(ctx->literalsPtr+1, (U32)(length)); ctx->literalsPtr += 4; } - else if (length >= 254) { *(ctx->literalsPtr) = 254; MEM_writeLE16(ctx->literalsPtr+1, (U16)(length)); ctx->literalsPtr += 3; } - else *(ctx->literalsPtr)++ = (BYTE)length; - } - else *token += (BYTE)(length<huffType) { - ctx->flagFreq[*token]++; - ctx->flagSum++; - Lizard_setLog2Prices(ctx); - } -#endif - - /* Prepare next loop */ - *ip += matchLength; - *anchor = *ip; - - return 0; -} - - -FORCE_INLINE int Lizard_encodeLastLiterals_LZ4 ( - Lizard_stream_t* ctx, - const BYTE** ip, - const BYTE** anchor) -{ - size_t length = (int)(*ip - *anchor); - - (void)ctx; - - memcpy(ctx->literalsPtr, *anchor, length); - ctx->literalsPtr += length; - return 0; -} - - -#define LIZARD_GET_TOKEN_PRICE_LZ4(token) (ctx->log2FlagSum - Lizard_highbit32(ctx->flagFreq[token]+1)) - -FORCE_INLINE size_t Lizard_get_price_LZ4(Lizard_stream_t* const ctx, const BYTE *ip, const size_t litLength, U32 offset, size_t matchLength) -{ - size_t price = 0; - BYTE token = 0; -#if 0 //def LIZARD_USE_HUFFMAN - const BYTE* literals = ip - litLength; - U32 u; - - if (ctx->cachedLiterals == literals && litLength >= ctx->cachedLitLength) { - size_t const additional = litLength - ctx->cachedLitLength; - const BYTE* literals2 = ctx->cachedLiterals + ctx->cachedLitLength; - price = ctx->cachedPrice + additional * ctx->log2LitSum; - for (u=0; u < additional; u++) - price -= Lizard_highbit32(ctx->litFreq[literals2[u]]+1); - ctx->cachedPrice = (U32)price; - ctx->cachedLitLength = (U32)litLength; - } else { - price = litLength * ctx->log2LitSum; - for (u=0; u < litLength; u++) - price -= Lizard_highbit32(ctx->litFreq[literals[u]]+1); - - if (litLength >= 12) { - ctx->cachedLiterals = literals; - ctx->cachedPrice = (U32)price; - ctx->cachedLitLength = (U32)litLength; - } - } -#else - price += 8*litLength; /* Copy Literals */ - (void)ip; - (void)ctx; -#endif - - /* Encode Literal length */ - if (litLength >= RUN_MASK_LZ4) { - size_t len = litLength - RUN_MASK_LZ4; - token = RUN_MASK_LZ4; - if (len >= (1<<16)) price += 32; - else if (len >= 254) price += 24; - else price += 8; - } - else token = (BYTE)litLength; - - - /* Encode MatchLength */ - if (offset) { - size_t length; - price += 16; /* Encode Offset */ - - if (offset < 8) return LIZARD_MAX_PRICE; // error - if (matchLength < MINMATCH) return LIZARD_MAX_PRICE; // error - - length = matchLength - MINMATCH; - if (length >= ML_MASK_LZ4) { - token += (BYTE)(ML_MASK_LZ4<= (1<<16)) price += 32; - else if (length >= 254) price += 24; - else price += 8; - } - else token += (BYTE)(length<huffType) { - if (offset > 0 || matchLength > 0) price += 2; - price += LIZARD_GET_TOKEN_PRICE_LZ4(token); - } else { - price += 8; // token - } - - return price; -} diff --git a/contrib/lizard/lib/lizard_decompress.c b/contrib/lizard/lib/lizard_decompress.c deleted file mode 100644 index df2eb699deb..00000000000 --- a/contrib/lizard/lib/lizard_decompress.c +++ /dev/null @@ -1,372 +0,0 @@ -/* - Lizard - Fast LZ compression algorithm - Copyright (C) 2011-2016, Yann Collet - Copyright (C) 2016-2017, Przemyslaw Skibinski - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - - -/************************************** -* Includes -**************************************/ -//#define LIZARD_STATS 1 // 0=simple stats, 1=more, 2=full -#ifdef LIZARD_STATS - #include "test/lizard_stats.h" -#endif -#include "lizard_compress.h" -#include "lizard_decompress.h" -#include "lizard_common.h" -#include // printf -#include // intptr_t - - -/*-************************************ -* Local Structures and types -**************************************/ -typedef enum { noDict = 0, withPrefix64k, usingExtDict } dict_directive; -typedef enum { full = 0, partial = 1 } earlyEnd_directive; - -#include "lizard_decompress_lz4.h" -#ifndef USE_LZ4_ONLY - #ifdef LIZARD_USE_TEST - #include "test/lizard_common_test.h" - #include "test/lizard_decompress_test.h" - #else - #include "lizard_decompress_liz.h" - #endif -#endif -#include "entropy/huf.h" - - -/*-***************************** -* Decompression functions -*******************************/ - -FORCE_INLINE size_t Lizard_readStream(int flag, const BYTE** ip, const BYTE* const iend, BYTE* op, BYTE* const oend, const BYTE** streamPtr, const BYTE** streamEnd, int streamFlag) -{ - if (!flag) { - if (*ip > iend - 3) return 0; - *streamPtr = *ip + 3; - *streamEnd = *streamPtr + MEM_readLE24(*ip); - if (*streamEnd < *streamPtr) return 0; - *ip = *streamEnd; -#ifdef LIZARD_STATS - uncompr_stream[streamFlag] += *streamEnd-*streamPtr; -#else - (void)streamFlag; -#endif - return 1; - } else { -#ifndef LIZARD_NO_HUFFMAN - size_t res, streamLen, comprStreamLen; - - if (*ip > iend - 6) return 0; - streamLen = MEM_readLE24(*ip); - comprStreamLen = MEM_readLE24(*ip + 3); - - if ((op > oend - streamLen) || (*ip + comprStreamLen > iend - 6)) return 0; - res = HUF_decompress(op, streamLen, *ip + 6, comprStreamLen); - if (HUF_isError(res) || (res != streamLen)) return 0; - - *ip += comprStreamLen + 6; - *streamPtr = op; - *streamEnd = *streamPtr + streamLen; -#ifdef LIZARD_STATS - compr_stream[streamFlag] += comprStreamLen + 6; - decompr_stream[streamFlag] += *streamEnd-*streamPtr; -#endif - return 1; -#else - fprintf(stderr, "compiled with LIZARD_NO_HUFFMAN\n"); - (void)op; (void)oend; - return 0; -#endif - } -} - - -FORCE_INLINE int Lizard_decompress_generic( - const char* source, - char* const dest, - int inputSize, - int outputSize, /* this value is the max size of Output Buffer. */ - int partialDecoding, /* full, partial */ - int targetOutputSize, /* only used if partialDecoding==partial */ - int dict, /* noDict, withPrefix64k, usingExtDict */ - const BYTE* const lowPrefix, /* == dest if dict == noDict */ - const BYTE* const dictStart, /* only if dict==usingExtDict */ - const size_t dictSize /* note : = 0 if noDict */ - ) -{ - /* Local Variables */ - const BYTE* ip = (const BYTE*) source, *istart = (const BYTE*) source; - const BYTE* const iend = ip + inputSize; - BYTE* op = (BYTE*) dest; - BYTE* const oend = op + outputSize; - BYTE* oexit = op + targetOutputSize; - Lizard_parameters params; - Lizard_dstream_t ctx; - BYTE* decompFlagsBase, *decompOff24Base, *decompOff16Base, *decompLiteralsBase = NULL; - int res, compressionLevel; - - if (inputSize < 1) { LIZARD_LOG_DECOMPRESS("inputSize=%d outputSize=%d targetOutputSize=%d partialDecoding=%d\n", inputSize, outputSize, targetOutputSize, partialDecoding); return 0; } - - compressionLevel = *ip++; - - if (compressionLevel < LIZARD_MIN_CLEVEL || compressionLevel > LIZARD_MAX_CLEVEL) { - LIZARD_LOG_DECOMPRESS("ERROR Lizard_decompress_generic inputSize=%d compressionLevel=%d\n", inputSize, compressionLevel); - return -1; - } - - LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic ip=%p inputSize=%d targetOutputSize=%d dest=%p outputSize=%d cLevel=%d dict=%d dictSize=%d dictStart=%p partialDecoding=%d\n", ip, inputSize, targetOutputSize, dest, outputSize, compressionLevel, dict, (int)dictSize, dictStart, partialDecoding); - - decompLiteralsBase = (BYTE*)malloc(4*LIZARD_HUF_BLOCK_SIZE); - if (!decompLiteralsBase) return -1; - decompFlagsBase = decompLiteralsBase + LIZARD_HUF_BLOCK_SIZE; - decompOff24Base = decompFlagsBase + LIZARD_HUF_BLOCK_SIZE; - decompOff16Base = decompOff24Base + LIZARD_HUF_BLOCK_SIZE; - -#ifdef LIZARD_STATS - init_stats(); -#endif - (void)istart; - - while (ip < iend) - { - res = *ip++; - if (res == LIZARD_FLAG_UNCOMPRESSED) /* uncompressed */ - { - uint32_t length; - if (ip > iend - 3) { LIZARD_LOG_DECOMPRESS("UNCOMPRESSED ip[%p] > iend[%p] - 3\n", ip, iend); goto _output_error; } - length = MEM_readLE24(ip); - ip += 3; - // printf("%d: total=%d block=%d UNCOMPRESSED op=%p oexit=%p oend=%p\n", (int)(op-(BYTE*)dest) ,(int)(ip-istart), length, op, oexit, oend); - if (ip + length > iend || op + length > oend) { LIZARD_LOG_DECOMPRESS("UNCOMPRESSED ip[%p]+length[%d] > iend[%p]\n", ip, length, iend); goto _output_error; } - memcpy(op, ip, length); - op += length; - ip += length; - if ((partialDecoding) && (op >= oexit)) break; -#ifdef LIZARD_STATS - uncompr_stream[LIZARD_STREAM_UNCOMPRESSED] += length; -#endif - continue; - } - - if (res&LIZARD_FLAG_LEN) { - LIZARD_LOG_DECOMPRESS("res=%d\n", res); goto _output_error; - } - - if (ip > iend - 5*3) goto _output_error; - ctx.lenPtr = (const BYTE*)ip + 3; - ctx.lenEnd = ctx.lenPtr + MEM_readLE24(ip); - if (ctx.lenEnd < ctx.lenPtr || (ctx.lenEnd > iend - 3)) goto _output_error; -#ifdef LIZARD_STATS - uncompr_stream[LIZARD_STREAM_LEN] += ctx.lenEnd-ctx.lenPtr + 3; -#endif - ip = ctx.lenEnd; - - { size_t streamLen; -#ifdef LIZARD_USE_LOGS - const BYTE* ipos; - size_t comprFlagsLen, comprLiteralsLen, total; -#endif - streamLen = Lizard_readStream(res&LIZARD_FLAG_OFFSET16, &ip, iend, decompOff16Base, decompOff16Base + LIZARD_HUF_BLOCK_SIZE, &ctx.offset16Ptr, &ctx.offset16End, LIZARD_STREAM_OFFSET16); - if (streamLen == 0) goto _output_error; - - streamLen = Lizard_readStream(res&LIZARD_FLAG_OFFSET24, &ip, iend, decompOff24Base, decompOff24Base + LIZARD_HUF_BLOCK_SIZE, &ctx.offset24Ptr, &ctx.offset24End, LIZARD_STREAM_OFFSET24); - if (streamLen == 0) goto _output_error; - -#ifdef LIZARD_USE_LOGS - ipos = ip; - streamLen = Lizard_readStream(res&LIZARD_FLAG_FLAGS, &ip, iend, decompFlagsBase, decompFlagsBase + LIZARD_HUF_BLOCK_SIZE, &ctx.flagsPtr, &ctx.flagsEnd, LIZARD_STREAM_FLAGS); - if (streamLen == 0) goto _output_error; - streamLen = (size_t)(ctx.flagsEnd-ctx.flagsPtr); - comprFlagsLen = ((size_t)(ip - ipos) + 3 >= streamLen) ? 0 : (size_t)(ip - ipos); - ipos = ip; -#else - streamLen = Lizard_readStream(res&LIZARD_FLAG_FLAGS, &ip, iend, decompFlagsBase, decompFlagsBase + LIZARD_HUF_BLOCK_SIZE, &ctx.flagsPtr, &ctx.flagsEnd, LIZARD_STREAM_FLAGS); - if (streamLen == 0) goto _output_error; -#endif - - streamLen = Lizard_readStream(res&LIZARD_FLAG_LITERALS, &ip, iend, decompLiteralsBase, decompLiteralsBase + LIZARD_HUF_BLOCK_SIZE, &ctx.literalsPtr, &ctx.literalsEnd, LIZARD_STREAM_LITERALS); - if (streamLen == 0) goto _output_error; -#ifdef LIZARD_USE_LOGS - streamLen = (size_t)(ctx.literalsEnd-ctx.literalsPtr); - comprLiteralsLen = ((size_t)(ip - ipos) + 3 >= streamLen) ? 0 : (size_t)(ip - ipos); - total = (size_t)(ip-(ctx.lenEnd-1)); -#endif - - if (ip > iend) goto _output_error; - - LIZARD_LOG_DECOMPRESS("%d: total=%d block=%d flagsLen=%d(HUF=%d) literalsLen=%d(HUF=%d) offset16Len=%d offset24Len=%d lengthsLen=%d \n", (int)(op-(BYTE*)dest) ,(int)(ip-istart), (int)total, - (int)(ctx.flagsEnd-ctx.flagsPtr), (int)comprFlagsLen, (int)(ctx.literalsEnd-ctx.literalsPtr), (int)comprLiteralsLen, - (int)(ctx.offset16End-ctx.offset16Ptr), (int)(ctx.offset24End-ctx.offset24Ptr), (int)(ctx.lenEnd-ctx.lenPtr)); - } - - ctx.last_off = -LIZARD_INIT_LAST_OFFSET; - params = Lizard_defaultParameters[compressionLevel - LIZARD_MIN_CLEVEL]; - if (params.decompressType == Lizard_coderwords_LZ4) - res = Lizard_decompress_LZ4(&ctx, op, outputSize, partialDecoding, targetOutputSize, dict, lowPrefix, dictStart, dictSize, compressionLevel); - else -#ifdef USE_LZ4_ONLY - res = Lizard_decompress_LZ4(&ctx, op, outputSize, partialDecoding, targetOutputSize, dict, lowPrefix, dictStart, dictSize, compressionLevel); -#else - res = Lizard_decompress_LIZv1(&ctx, op, outputSize, partialDecoding, targetOutputSize, dict, lowPrefix, dictStart, dictSize, compressionLevel); -#endif - LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic res=%d inputSize=%d\n", res, (int)(ctx.literalsEnd-ctx.lenEnd)); - - if (res <= 0) { free(decompLiteralsBase); return res; } - - op += res; - outputSize -= res; - if ((partialDecoding) && (op >= oexit)) break; - } - -#ifdef LIZARD_STATS - print_stats(); -#endif - - LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic total=%d\n", (int)(op-(BYTE*)dest)); - free(decompLiteralsBase); - return (int)(op-(BYTE*)dest); - -_output_error: - LIZARD_LOG_DECOMPRESS("Lizard_decompress_generic ERROR ip=%p iend=%p\n", ip, iend); - free(decompLiteralsBase); - return -1; -} - - -int Lizard_decompress_safe(const char* source, char* dest, int compressedSize, int maxDecompressedSize) -{ - return Lizard_decompress_generic(source, dest, compressedSize, maxDecompressedSize, full, 0, noDict, (BYTE*)dest, NULL, 0); -} - -int Lizard_decompress_safe_partial(const char* source, char* dest, int compressedSize, int targetOutputSize, int maxDecompressedSize) -{ - return Lizard_decompress_generic(source, dest, compressedSize, maxDecompressedSize, partial, targetOutputSize, noDict, (BYTE*)dest, NULL, 0); -} - - -/*===== streaming decompression functions =====*/ - - -/* - * If you prefer dynamic allocation methods, - * Lizard_createStreamDecode() - * provides a pointer (void*) towards an initialized Lizard_streamDecode_t structure. - */ -Lizard_streamDecode_t* Lizard_createStreamDecode(void) -{ - Lizard_streamDecode_t* lizards = (Lizard_streamDecode_t*) ALLOCATOR(1, sizeof(Lizard_streamDecode_t)); - return lizards; -} - -int Lizard_freeStreamDecode (Lizard_streamDecode_t* Lizard_stream) -{ - FREEMEM(Lizard_stream); - return 0; -} - -/*! - * Lizard_setStreamDecode() : - * Use this function to instruct where to find the dictionary. - * This function is not necessary if previous data is still available where it was decoded. - * Loading a size of 0 is allowed (same effect as no dictionary). - * Return : 1 if OK, 0 if error - */ -int Lizard_setStreamDecode (Lizard_streamDecode_t* Lizard_streamDecode, const char* dictionary, int dictSize) -{ - Lizard_streamDecode_t* lizardsd = (Lizard_streamDecode_t*) Lizard_streamDecode; - lizardsd->prefixSize = (size_t) dictSize; - lizardsd->prefixEnd = (const BYTE*) dictionary + dictSize; - lizardsd->externalDict = NULL; - lizardsd->extDictSize = 0; - return 1; -} - -/* -*_continue() : - These decoding functions allow decompression of multiple blocks in "streaming" mode. - Previously decoded blocks must still be available at the memory position where they were decoded. - If it's not possible, save the relevant part of decoded data into a safe buffer, - and indicate where it stands using Lizard_setStreamDecode() -*/ -int Lizard_decompress_safe_continue (Lizard_streamDecode_t* Lizard_streamDecode, const char* source, char* dest, int compressedSize, int maxOutputSize) -{ - Lizard_streamDecode_t* lizardsd = (Lizard_streamDecode_t*) Lizard_streamDecode; - int result; - - if (lizardsd->prefixEnd == (BYTE*)dest) { - result = Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, - full, 0, usingExtDict, lizardsd->prefixEnd - lizardsd->prefixSize, lizardsd->externalDict, lizardsd->extDictSize); - if (result <= 0) return result; - lizardsd->prefixSize += result; - lizardsd->prefixEnd += result; - } else { - lizardsd->extDictSize = lizardsd->prefixSize; - lizardsd->externalDict = lizardsd->prefixEnd - lizardsd->extDictSize; - result = Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, - full, 0, usingExtDict, (BYTE*)dest, lizardsd->externalDict, lizardsd->extDictSize); - if (result <= 0) return result; - lizardsd->prefixSize = result; - lizardsd->prefixEnd = (BYTE*)dest + result; - } - - return result; -} - - -/* -Advanced decoding functions : -*_usingDict() : - These decoding functions work the same as "_continue" ones, - the dictionary must be explicitly provided within parameters -*/ - -int Lizard_decompress_safe_usingDict(const char* source, char* dest, int compressedSize, int maxOutputSize, const char* dictStart, int dictSize) -{ - if (dictSize==0) - return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, noDict, (BYTE*)dest, NULL, 0); - if (dictStart+dictSize == dest) - { - if (dictSize >= (int)(LIZARD_DICT_SIZE - 1)) - return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, withPrefix64k, (BYTE*)dest-LIZARD_DICT_SIZE, NULL, 0); - return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, noDict, (BYTE*)dest-dictSize, NULL, 0); - } - return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, usingExtDict, (BYTE*)dest, (const BYTE*)dictStart, dictSize); -} - -/* debug function */ -int Lizard_decompress_safe_forceExtDict(const char* source, char* dest, int compressedSize, int maxOutputSize, const char* dictStart, int dictSize) -{ - return Lizard_decompress_generic(source, dest, compressedSize, maxOutputSize, full, 0, usingExtDict, (BYTE*)dest, (const BYTE*)dictStart, dictSize); -} - diff --git a/contrib/lizard/lib/lizard_decompress.h b/contrib/lizard/lib/lizard_decompress.h deleted file mode 100644 index ad9fc8ee941..00000000000 --- a/contrib/lizard/lib/lizard_decompress.h +++ /dev/null @@ -1,152 +0,0 @@ -/* - Lizard - Fast LZ compression algorithm - Header File - Copyright (C) 2011-2016, Yann Collet - Copyright (C) 2016-2017, Przemyslaw Skibinski - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ -#ifndef LIZARD_DECOMPRESS_H_2983 -#define LIZARD_DECOMPRESS_H_2983 - -#if defined (__cplusplus) -extern "C" { -#endif - - -/*^*************************************************************** -* Export parameters -*****************************************************************/ -/* -* LIZARD_DLL_EXPORT : -* Enable exporting of functions when building a Windows DLL -*/ -#if defined(LIZARD_DLL_EXPORT) && (LIZARD_DLL_EXPORT==1) -# define LIZARDDLIB_API __declspec(dllexport) -#elif defined(LIZARD_DLL_IMPORT) && (LIZARD_DLL_IMPORT==1) -# define LIZARDDLIB_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/ -#else -# define LIZARDDLIB_API -#endif - - -/*-************************************ -* Simple Functions -**************************************/ - -/* -Lizard_decompress_safe() : - compressedSize : is the precise full size of the compressed block. - maxDecompressedSize : is the size of destination buffer, which must be already allocated. - return : the number of bytes decompressed into destination buffer (necessarily <= maxDecompressedSize) - If destination buffer is not large enough, decoding will stop and output an error code (<0). - If the source stream is detected malformed, the function will stop decoding and return a negative result. - This function is protected against buffer overflow exploits, including malicious data packets. - It never writes outside output buffer, nor reads outside input buffer. -*/ -LIZARDDLIB_API int Lizard_decompress_safe (const char* source, char* dest, int compressedSize, int maxDecompressedSize); - - - -/*! -Lizard_decompress_safe_partial() : - This function decompress a compressed block of size 'compressedSize' at position 'source' - into destination buffer 'dest' of size 'maxDecompressedSize'. - The function tries to stop decompressing operation as soon as 'targetOutputSize' has been reached, - reducing decompression time. - return : the number of bytes decoded in the destination buffer (necessarily <= maxDecompressedSize) - Note : this number can be < 'targetOutputSize' should the compressed block to decode be smaller. - Always control how many bytes were decoded. - If the source stream is detected malformed, the function will stop decoding and return a negative result. - This function never writes outside of output buffer, and never reads outside of input buffer. It is therefore protected against malicious data packets -*/ -LIZARDDLIB_API int Lizard_decompress_safe_partial (const char* source, char* dest, int compressedSize, int targetOutputSize, int maxDecompressedSize); - - - -/*-********************************************** -* Streaming Decompression Functions -************************************************/ -typedef struct Lizard_streamDecode_s Lizard_streamDecode_t; - -/* - * Lizard_streamDecode_t - * information structure to track an Lizard stream. - * init this structure content using Lizard_setStreamDecode or memset() before first use ! - * - * In the context of a DLL (liblizard) please prefer usage of construction methods below. - * They are more future proof, in case of a change of Lizard_streamDecode_t size in the future. - * Lizard_createStreamDecode will allocate and initialize an Lizard_streamDecode_t structure - * Lizard_freeStreamDecode releases its memory. - */ -LIZARDDLIB_API Lizard_streamDecode_t* Lizard_createStreamDecode(void); -LIZARDDLIB_API int Lizard_freeStreamDecode (Lizard_streamDecode_t* Lizard_stream); - -/*! Lizard_setStreamDecode() : - * Use this function to instruct where to find the dictionary. - * Setting a size of 0 is allowed (same effect as reset). - * @return : 1 if OK, 0 if error - */ -LIZARDDLIB_API int Lizard_setStreamDecode (Lizard_streamDecode_t* Lizard_streamDecode, const char* dictionary, int dictSize); - -/* -*_continue() : - These decoding functions allow decompression of multiple blocks in "streaming" mode. - Previously decoded blocks *must* remain available at the memory position where they were decoded (up to LIZARD_DICT_SIZE) - In the case of a ring buffers, decoding buffer must be either : - - Exactly same size as encoding buffer, with same update rule (block boundaries at same positions) - In which case, the decoding & encoding ring buffer can have any size, including small ones ( < LIZARD_DICT_SIZE). - - Larger than encoding buffer, by a minimum of maxBlockSize more bytes. - maxBlockSize is implementation dependent. It's the maximum size you intend to compress into a single block. - In which case, encoding and decoding buffers do not need to be synchronized, - and encoding ring buffer can have any size, including small ones ( < LIZARD_DICT_SIZE). - - _At least_ LIZARD_DICT_SIZE + 8 bytes + maxBlockSize. - In which case, encoding and decoding buffers do not need to be synchronized, - and encoding ring buffer can have any size, including larger than decoding buffer. - Whenever these conditions are not possible, save the last LIZARD_DICT_SIZE of decoded data into a safe buffer, - and indicate where it is saved using Lizard_setStreamDecode() -*/ -LIZARDDLIB_API int Lizard_decompress_safe_continue (Lizard_streamDecode_t* Lizard_streamDecode, const char* source, char* dest, int compressedSize, int maxDecompressedSize); - - -/* -Advanced decoding functions : -*_usingDict() : - These decoding functions work the same as - a combination of Lizard_setStreamDecode() followed by Lizard_decompress_x_continue() - They are stand-alone. They don't need nor update an Lizard_streamDecode_t structure. -*/ -LIZARDDLIB_API int Lizard_decompress_safe_usingDict (const char* source, char* dest, int compressedSize, int maxDecompressedSize, const char* dictStart, int dictSize); - - -#if defined (__cplusplus) -} -#endif - -#endif /* LIZARD_DECOMPRESS_H_2983827168210 */ diff --git a/contrib/lizard/lib/lizard_decompress_liz.h b/contrib/lizard/lib/lizard_decompress_liz.h deleted file mode 100644 index 7e80eb5e01b..00000000000 --- a/contrib/lizard/lib/lizard_decompress_liz.h +++ /dev/null @@ -1,220 +0,0 @@ -/* - [0_MMMM_LLL] - 16-bit offset, 4-bit match length (4-15+), 3-bit literal length (0-7+) - [1_MMMM_LLL] - last offset, 4-bit match length (0-15+), 3-bit literal length (0-7+) - flag 31 - 24-bit offset, match length (47+), no literal length - flag 0-30 - 24-bit offset, 31 match lengths (16-46), no literal length -*/ - -/*! Lizard_decompress_LIZv1() : - * This generic decompression function cover all use cases. - * It shall be instantiated several times, using different sets of directives - * Note that it is important this generic function is really inlined, - * in order to remove useless branches during compilation optimization. - */ -FORCE_INLINE int Lizard_decompress_LIZv1( - Lizard_dstream_t* ctx, - BYTE* const dest, - int outputSize, /* this value is the max size of Output Buffer. */ - - int partialDecoding, /* full, partial */ - int targetOutputSize, /* only used if partialDecoding==partial */ - int dict, /* noDict, withPrefix64k, usingExtDict */ - const BYTE* const lowPrefix, /* == dest if dict == noDict */ - const BYTE* const dictStart, /* only if dict==usingExtDict */ - const size_t dictSize, /* note : = 0 if noDict */ - int compressionLevel - ) -{ - /* Local Variables */ - int inputSize = (int)(ctx->flagsEnd - ctx->flagsPtr); - const BYTE* const blockBase = ctx->flagsPtr; - const BYTE* const iend = ctx->literalsEnd; - - BYTE* op = dest; - BYTE* const oend = op + outputSize; - BYTE* cpy = NULL; - BYTE* oexit = op + targetOutputSize; - const BYTE* const lowLimit = lowPrefix - dictSize; - const BYTE* const dictEnd = (const BYTE*)dictStart + dictSize; - - const int checkOffset = (dictSize < (int)(LIZARD_DICT_SIZE)); - - intptr_t last_off = ctx->last_off; - intptr_t length = 0; - (void)compressionLevel; - - /* Special cases */ - if (unlikely(outputSize==0)) return ((inputSize==1) && (*ctx->flagsPtr==0)) ? 0 : -1; /* Empty output buffer */ - - /* Main Loop : decode sequences */ - while (ctx->flagsPtr < ctx->flagsEnd) { - unsigned token; - const BYTE* match; - // intptr_t litLength; - - if ((partialDecoding) && (op >= oexit)) return (int) (op-dest); - - /* get literal length */ - token = *ctx->flagsPtr++; - - if (token >= 32) - { - if ((length=(token & MAX_SHORT_LITLEN)) == MAX_SHORT_LITLEN) { - if (unlikely(ctx->literalsPtr > iend - 1)) { LIZARD_LOG_DECOMPRESS_LIZv1("1"); goto _output_error; } - length = *ctx->literalsPtr; - if unlikely(length >= 254) { - if (length == 254) { - length = MEM_readLE16(ctx->literalsPtr+1); - ctx->literalsPtr += 2; - } else { - length = MEM_readLE24(ctx->literalsPtr+1); - ctx->literalsPtr += 3; - } - } - length += MAX_SHORT_LITLEN; - ctx->literalsPtr++; - if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LIZv1("2"); goto _output_error; } /* overflow detection */ - if (unlikely((size_t)(ctx->literalsPtr+length)<(size_t)(ctx->literalsPtr))) { LIZARD_LOG_DECOMPRESS_LIZv1("3"); goto _output_error; } /* overflow detection */ - } - - /* copy literals */ - cpy = op + length; - if (unlikely(cpy > oend - WILDCOPYLENGTH || ctx->literalsPtr > iend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LIZv1("offset outside buffers\n"); goto _output_error; } /* Error : offset outside buffers */ - #if 1 - Lizard_wildCopy16(op, ctx->literalsPtr, cpy); - op = cpy; - ctx->literalsPtr += length; - #else - Lizard_copy8(op, ctx->literalsPtr); - Lizard_copy8(op+8, ctx->literalsPtr+8); - if (length > 16) - Lizard_wildCopy16(op + 16, ctx->literalsPtr + 16, cpy); - op = cpy; - ctx->literalsPtr += length; - #endif - - /* get offset */ - if (unlikely(ctx->offset16Ptr > ctx->offset16End)) { LIZARD_LOG_DECOMPRESS_LIZv1("(ctx->offset16Ptr > ctx->offset16End\n"); goto _output_error; } -#if 1 - { /* branchless */ - intptr_t new_off = MEM_readLE16(ctx->offset16Ptr); - uintptr_t not_repCode = (uintptr_t)(token >> ML_RUN_BITS) - 1; - last_off ^= not_repCode & (last_off ^ -new_off); - ctx->offset16Ptr = (BYTE*)((uintptr_t)ctx->offset16Ptr + (not_repCode & 2)); - } -#else - if ((token >> ML_RUN_BITS) == 0) - { - last_off = -(intptr_t)MEM_readLE16(ctx->offset16Ptr); - ctx->offset16Ptr += 2; - } -#endif - - /* get matchlength */ - length = (token >> RUN_BITS_LIZv1) & MAX_SHORT_MATCHLEN; - if (length == MAX_SHORT_MATCHLEN) { - if (unlikely(ctx->literalsPtr > iend - 1)) { LIZARD_LOG_DECOMPRESS_LIZv1("6"); goto _output_error; } - length = *ctx->literalsPtr; - if unlikely(length >= 254) { - if (length == 254) { - length = MEM_readLE16(ctx->literalsPtr+1); - ctx->literalsPtr += 2; - } else { - length = MEM_readLE24(ctx->literalsPtr+1); - ctx->literalsPtr += 3; - } - } - length += MAX_SHORT_MATCHLEN; - ctx->literalsPtr++; - if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LIZv1("7"); goto _output_error; } /* overflow detection */ - } - - DECOMPLOG_CODEWORDS_LIZv1("T32+ literal=%u match=%u offset=%d ipos=%d opos=%d\n", (U32)litLength, (U32)length, (int)-last_off, (U32)(ctx->flagsPtr-blockBase), (U32)(op-dest)); - } - else - if (token < LIZARD_LAST_LONG_OFF) - { - if (unlikely(ctx->offset24Ptr > ctx->offset24End - 3)) { LIZARD_LOG_DECOMPRESS_LIZv1("8"); goto _output_error; } - length = token + MM_LONGOFF; - last_off = -(intptr_t)MEM_readLE24(ctx->offset24Ptr); - ctx->offset24Ptr += 3; - DECOMPLOG_CODEWORDS_LIZv1("T0-30 literal=%u match=%u offset=%d\n", 0, (U32)length, (int)-last_off); - } - else - { - if (unlikely(ctx->literalsPtr > iend - 1)) { LIZARD_LOG_DECOMPRESS_LIZv1("9"); goto _output_error; } - length = *ctx->literalsPtr; - if unlikely(length >= 254) { - if (length == 254) { - length = MEM_readLE16(ctx->literalsPtr+1); - ctx->literalsPtr += 2; - } else { - length = MEM_readLE24(ctx->literalsPtr+1); - ctx->literalsPtr += 3; - } - } - ctx->literalsPtr++; - length += LIZARD_LAST_LONG_OFF + MM_LONGOFF; - - if (unlikely(ctx->offset24Ptr > ctx->offset24End - 3)) { LIZARD_LOG_DECOMPRESS_LIZv1("10"); goto _output_error; } - last_off = -(intptr_t)MEM_readLE24(ctx->offset24Ptr); - ctx->offset24Ptr += 3; - } - - - match = op + last_off; - if ((checkOffset) && ((unlikely((uintptr_t)(-last_off) > (uintptr_t)op) || (match < lowLimit)))) { LIZARD_LOG_DECOMPRESS_LIZv1("lowPrefix[%p]-dictSize[%d]=lowLimit[%p] match[%p]=op[%p]-last_off[%d]\n", lowPrefix, (int)dictSize, lowLimit, match, op, (int)last_off); goto _output_error; } /* Error : offset outside buffers */ - - /* check external dictionary */ - if ((dict==usingExtDict) && (match < lowPrefix)) { - if (unlikely(op + length > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LIZv1("12"); goto _output_error; } /* doesn't respect parsing restriction */ - - if (length <= (intptr_t)(lowPrefix - match)) { - /* match can be copied as a single segment from external dictionary */ - memmove(op, dictEnd - (lowPrefix-match), length); - op += length; - } else { - /* match encompass external dictionary and current block */ - size_t const copySize = (size_t)(lowPrefix-match); - size_t const restSize = length - copySize; - memcpy(op, dictEnd - copySize, copySize); - op += copySize; - if (restSize > (size_t)(op-lowPrefix)) { /* overlap copy */ - BYTE* const endOfMatch = op + restSize; - const BYTE* copyFrom = lowPrefix; - while (op < endOfMatch) *op++ = *copyFrom++; - } else { - memcpy(op, lowPrefix, restSize); - op += restSize; - } } - continue; - } - - /* copy match within block */ - cpy = op + length; - if (unlikely(cpy > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LIZv1("13match=%p lowLimit=%p\n", match, lowLimit); goto _output_error; } /* Error : offset outside buffers */ - Lizard_copy8(op, match); - Lizard_copy8(op+8, match+8); - if (length > 16) - Lizard_wildCopy16(op + 16, match + 16, cpy); - op = cpy; - } - - /* last literals */ - length = ctx->literalsEnd - ctx->literalsPtr; - cpy = op + length; - if ((length < 0) || (ctx->literalsPtr+length != iend) || (cpy > oend)) { LIZARD_LOG_DECOMPRESS_LIZv1("14"); goto _output_error; } /* Error : input must be consumed */ - memcpy(op, ctx->literalsPtr, length); - ctx->literalsPtr += length; - op += length; - - /* end of decoding */ - ctx->last_off = last_off; - return (int) (op-dest); /* Nb of output bytes decoded */ - - /* Overflow error detected */ -_output_error: - LIZARD_LOG_DECOMPRESS_LIZv1("_output_error=%d ctx->flagsPtr=%p blockBase=%p\n", (int) (-(ctx->flagsPtr-blockBase))-1, ctx->flagsPtr, blockBase); - LIZARD_LOG_DECOMPRESS_LIZv1("cpy=%p oend=%p ctx->literalsPtr+length[%d]=%p iend=%p\n", cpy, oend, (int)length, ctx->literalsPtr+length, iend); - return (int) (-(ctx->flagsPtr-blockBase))-1; -} diff --git a/contrib/lizard/lib/lizard_decompress_lz4.h b/contrib/lizard/lib/lizard_decompress_lz4.h deleted file mode 100644 index ffbef1cd976..00000000000 --- a/contrib/lizard/lib/lizard_decompress_lz4.h +++ /dev/null @@ -1,163 +0,0 @@ -/*! Lizard_decompress_LZ4() : - * This generic decompression function cover all use cases. - * It shall be instantiated several times, using different sets of directives - * Note that it is important this generic function is really inlined, - * in order to remove useless branches during compilation optimization. - */ -FORCE_INLINE int Lizard_decompress_LZ4( - Lizard_dstream_t* ctx, - BYTE* const dest, - int outputSize, /* this value is the max size of Output Buffer. */ - - int partialDecoding, /* full, partial */ - int targetOutputSize, /* only used if partialDecoding==partial */ - int dict, /* noDict, withPrefix64k, usingExtDict */ - const BYTE* const lowPrefix, /* == dest if dict == noDict */ - const BYTE* const dictStart, /* only if dict==usingExtDict */ - const size_t dictSize, /* note : = 0 if noDict */ - int compressionLevel - ) -{ - /* Local Variables */ - int inputSize = (int)(ctx->flagsEnd - ctx->flagsPtr); - const BYTE* const blockBase = ctx->flagsPtr; - const BYTE* const iend = ctx->literalsEnd; - BYTE* op = dest; - BYTE* const oend = op + outputSize; - BYTE* cpy = NULL; - BYTE* oexit = op + targetOutputSize; - const BYTE* const lowLimit = lowPrefix - dictSize; - const BYTE* const dictEnd = (const BYTE*)dictStart + dictSize; - - const int checkOffset = (dictSize < (int)(LIZARD_DICT_SIZE)); - - intptr_t length = 0; - (void)compressionLevel; - - /* Special cases */ - if (unlikely(outputSize==0)) return ((inputSize==1) && (*ctx->flagsPtr==0)) ? 0 : -1; /* Empty output buffer */ - - /* Main Loop : decode sequences */ - while (ctx->flagsPtr < ctx->flagsEnd) { - unsigned token; - const BYTE* match; - size_t offset; - - /* get literal length */ - token = *ctx->flagsPtr++; - if ((length=(token & RUN_MASK_LZ4)) == RUN_MASK_LZ4) { - if (unlikely(ctx->literalsPtr > iend - 5)) { LIZARD_LOG_DECOMPRESS_LZ4("0"); goto _output_error; } - length = *ctx->literalsPtr; - if unlikely(length >= 254) { - if (length == 254) { - length = MEM_readLE16(ctx->literalsPtr+1); - ctx->literalsPtr += 2; - } else { - length = MEM_readLE24(ctx->literalsPtr+1); - ctx->literalsPtr += 3; - } - } - length += RUN_MASK_LZ4; - ctx->literalsPtr++; - if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LZ4("1"); goto _output_error; } /* overflow detection */ - if (unlikely((size_t)(ctx->literalsPtr+length)<(size_t)(ctx->literalsPtr))) { LIZARD_LOG_DECOMPRESS_LZ4("2"); goto _output_error; } /* overflow detection */ - } - - /* copy literals */ - cpy = op + length; - if (unlikely(cpy > oend - WILDCOPYLENGTH || ctx->literalsPtr + length > iend - (2 + WILDCOPYLENGTH))) { LIZARD_LOG_DECOMPRESS_LZ4("offset outside buffers\n"); goto _output_error; } /* Error : offset outside buffers */ - -#if 1 - Lizard_wildCopy16(op, ctx->literalsPtr, cpy); - op = cpy; - ctx->literalsPtr += length; -#else - Lizard_copy8(op, ctx->literalsPtr); - Lizard_copy8(op+8, ctx->literalsPtr+8); - if (length > 16) - Lizard_wildCopy16(op + 16, ctx->literalsPtr + 16, cpy); - op = cpy; - ctx->literalsPtr += length; -#endif - if ((partialDecoding) && (op >= oexit)) return (int) (op-dest); - - /* get offset */ - offset = MEM_readLE16(ctx->literalsPtr); - ctx->literalsPtr += 2; - - match = op - offset; - if ((checkOffset) && (unlikely(match < lowLimit))) { LIZARD_LOG_DECOMPRESS_LZ4("lowPrefix[%p]-dictSize[%d]=lowLimit[%p] match[%p]=op[%p]-offset[%d]\n", lowPrefix, (int)dictSize, lowLimit, match, op, (int)offset); goto _output_error; } /* Error : offset outside buffers */ - - /* get matchlength */ - length = token >> RUN_BITS_LZ4; - if (length == ML_MASK_LZ4) { - if (unlikely(ctx->literalsPtr > iend - 5)) { LIZARD_LOG_DECOMPRESS_LZ4("4"); goto _output_error; } - length = *ctx->literalsPtr; - if unlikely(length >= 254) { - if (length == 254) { - length = MEM_readLE16(ctx->literalsPtr+1); - ctx->literalsPtr += 2; - } else { - length = MEM_readLE24(ctx->literalsPtr+1); - ctx->literalsPtr += 3; - } - } - length += ML_MASK_LZ4; - ctx->literalsPtr++; - if (unlikely((size_t)(op+length)<(size_t)(op))) { LIZARD_LOG_DECOMPRESS_LZ4("5"); goto _output_error; } /* overflow detection */ - } - length += MINMATCH; - - /* check external dictionary */ - if ((dict==usingExtDict) && (match < lowPrefix)) { - if (unlikely(op + length > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LZ4("6"); goto _output_error; } /* doesn't respect parsing restriction */ - - if (length <= (intptr_t)(lowPrefix - match)) { - /* match can be copied as a single segment from external dictionary */ - memmove(op, dictEnd - (lowPrefix-match), length); - op += length; - } else { - /* match encompass external dictionary and current block */ - size_t const copySize = (size_t)(lowPrefix-match); - size_t const restSize = length - copySize; - memcpy(op, dictEnd - copySize, copySize); - op += copySize; - if (restSize > (size_t)(op-lowPrefix)) { /* overlap copy */ - BYTE* const endOfMatch = op + restSize; - const BYTE* copyFrom = lowPrefix; - while (op < endOfMatch) *op++ = *copyFrom++; - } else { - memcpy(op, lowPrefix, restSize); - op += restSize; - } } - continue; - } - - /* copy match within block */ - cpy = op + length; - if (unlikely(cpy > oend - WILDCOPYLENGTH)) { LIZARD_LOG_DECOMPRESS_LZ4("1match=%p lowLimit=%p\n", match, lowLimit); goto _output_error; } /* Error : offset outside buffers */ - Lizard_copy8(op, match); - Lizard_copy8(op+8, match+8); - if (length > 16) - Lizard_wildCopy16(op + 16, match + 16, cpy); - op = cpy; - if ((partialDecoding) && (op >= oexit)) return (int) (op-dest); - } - - /* last literals */ - length = ctx->literalsEnd - ctx->literalsPtr; - cpy = op + length; - if ((length < 0) || (ctx->literalsPtr+length != iend) || (cpy > oend)) { LIZARD_LOG_DECOMPRESS_LZ4("9"); goto _output_error; } /* Error : input must be consumed */ - memcpy(op, ctx->literalsPtr, length); - ctx->literalsPtr += length; - op += length; - - /* end of decoding */ - return (int) (op-dest); /* Nb of output bytes decoded */ - - /* Overflow error detected */ -_output_error: - LIZARD_LOG_DECOMPRESS_LZ4("_output_error=%d ctx->flagsPtr=%p blockBase=%p\n", (int) (-(ctx->flagsPtr-blockBase))-1, ctx->flagsPtr, blockBase); - LIZARD_LOG_DECOMPRESS_LZ4("cpy=%p oend=%p ctx->literalsPtr+length[%d]=%p iend=%p\n", cpy, oend, (int)length, ctx->literalsPtr+length, iend); - return (int) (-(ctx->flagsPtr-blockBase))-1; -} diff --git a/contrib/lizard/lib/lizard_frame.c b/contrib/lizard/lib/lizard_frame.c deleted file mode 100644 index f4afbb94e0a..00000000000 --- a/contrib/lizard/lib/lizard_frame.c +++ /dev/null @@ -1,1362 +0,0 @@ -/* -Lizard auto-framing library -Copyright (C) 2011-2016, Yann Collet -Copyright (C) 2016-2017, Przemyslaw Skibinski - -BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - -* Redistributions of source code must retain the above copyright -notice, this list of conditions and the following disclaimer. -* Redistributions in binary form must reproduce the above -copyright notice, this list of conditions and the following disclaimer -in the documentation and/or other materials provided with the -distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -You can contact the author at : -- Lizard source repository : https://github.com/inikep/lizard -*/ - -/* LizardF is a stand-alone API to create Lizard-compressed Frames -* in full conformance with specification v1.5.0 -* All related operations, including memory management, are handled by the library. -* */ - - -/*-************************************ -* Compiler Options -**************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -#endif - - - -/*-************************************ -* Includes -**************************************/ -#include "lizard_frame_static.h" -#include "lizard_compress.h" -#include "lizard_decompress.h" -#include "lizard_common.h" /* LIZARD_DICT_SIZE */ -#define XXH_STATIC_LINKING_ONLY -#include "xxhash/xxhash.h" -#include - - - -/* unoptimized version; solves endianess & alignment issues */ -static U32 LizardF_readLE32 (const void* src) -{ - const BYTE* const srcPtr = (const BYTE*)src; - U32 value32 = srcPtr[0]; - value32 += (srcPtr[1]<<8); - value32 += (srcPtr[2]<<16); - value32 += ((U32)srcPtr[3])<<24; - return value32; -} - -static void LizardF_writeLE32 (BYTE* dstPtr, U32 value32) -{ - dstPtr[0] = (BYTE)value32; - dstPtr[1] = (BYTE)(value32 >> 8); - dstPtr[2] = (BYTE)(value32 >> 16); - dstPtr[3] = (BYTE)(value32 >> 24); -} - -static U64 LizardF_readLE64 (const BYTE* srcPtr) -{ - U64 value64 = srcPtr[0]; - value64 += ((U64)srcPtr[1]<<8); - value64 += ((U64)srcPtr[2]<<16); - value64 += ((U64)srcPtr[3]<<24); - value64 += ((U64)srcPtr[4]<<32); - value64 += ((U64)srcPtr[5]<<40); - value64 += ((U64)srcPtr[6]<<48); - value64 += ((U64)srcPtr[7]<<56); - return value64; -} - -static void LizardF_writeLE64 (BYTE* dstPtr, U64 value64) -{ - dstPtr[0] = (BYTE)value64; - dstPtr[1] = (BYTE)(value64 >> 8); - dstPtr[2] = (BYTE)(value64 >> 16); - dstPtr[3] = (BYTE)(value64 >> 24); - dstPtr[4] = (BYTE)(value64 >> 32); - dstPtr[5] = (BYTE)(value64 >> 40); - dstPtr[6] = (BYTE)(value64 >> 48); - dstPtr[7] = (BYTE)(value64 >> 56); -} - - -/*-************************************ -* Constants -**************************************/ -#define _1BIT 0x01 -#define _2BITS 0x03 -#define _3BITS 0x07 -#define _4BITS 0x0F -#define _8BITS 0xFF - -#define LIZARDF_MAGIC_SKIPPABLE_START 0x184D2A50U -#define LIZARDF_MAGICNUMBER 0x184D2206U -#define LIZARDF_BLOCKUNCOMPRESSED_FLAG 0x80000000U -#define LIZARDF_BLOCKSIZEID_DEFAULT LizardF_max128KB - -static const size_t minFHSize = 7; -static const size_t maxFHSize = 15; -static const size_t BHSize = 4; - - -/*-************************************ -* Structures and local types -**************************************/ -typedef struct LizardF_cctx_s -{ - LizardF_preferences_t prefs; - U32 version; - U32 cStage; - size_t maxBlockSize; - size_t maxBufferSize; - BYTE* tmpBuff; - BYTE* tmpIn; - size_t tmpInSize; - U64 totalInSize; - XXH32_state_t xxh; - Lizard_stream_t* lizardCtxPtr; - U32 lizardCtxLevel; /* 0: unallocated; 1: Lizard_stream_t; */ -} LizardF_cctx_t; - -typedef struct LizardF_dctx_s -{ - LizardF_frameInfo_t frameInfo; - U32 version; - U32 dStage; - U64 frameRemainingSize; - size_t maxBlockSize; - size_t maxBufferSize; - const BYTE* srcExpect; - BYTE* tmpIn; - size_t tmpInSize; - size_t tmpInTarget; - BYTE* tmpOutBuffer; - const BYTE* dict; - size_t dictSize; - BYTE* tmpOut; - size_t tmpOutSize; - size_t tmpOutStart; - XXH32_state_t xxh; - BYTE header[16]; -} LizardF_dctx_t; - - -/*-************************************ -* Error management -**************************************/ -#define LIZARDF_GENERATE_STRING(STRING) #STRING, -static const char* LizardF_errorStrings[] = { LIZARDF_LIST_ERRORS(LIZARDF_GENERATE_STRING) }; - - -unsigned LizardF_isError(LizardF_errorCode_t code) -{ - return (code > (LizardF_errorCode_t)(-LizardF_ERROR_maxCode)); -} - -const char* LizardF_getErrorName(LizardF_errorCode_t code) -{ - static const char* codeError = "Unspecified error code"; - if (LizardF_isError(code)) return LizardF_errorStrings[-(int)(code)]; - return codeError; -} - - -/*-************************************ -* Private functions -**************************************/ -static size_t LizardF_getBlockSize(unsigned blockSizeID) -{ - static const size_t blockSizes[7] = { 128 KB, 256 KB, 1 MB, 4 MB, 16 MB, 64 MB, 256 MB }; - - if (blockSizeID == 0) blockSizeID = LIZARDF_BLOCKSIZEID_DEFAULT; - blockSizeID -= 1; - if (blockSizeID >= 7) return (size_t)-LizardF_ERROR_maxBlockSize_invalid; - - return blockSizes[blockSizeID]; -} - -static LizardF_blockSizeID_t LizardF_optimalBSID(const LizardF_blockSizeID_t requestedBSID, const size_t srcSize) -{ - LizardF_blockSizeID_t proposedBSID = LizardF_max128KB; - size_t maxBlockSize; - - while (requestedBSID > proposedBSID) - { - maxBlockSize = LizardF_getBlockSize(proposedBSID); - if (srcSize <= maxBlockSize) { - return proposedBSID; - } - proposedBSID = (LizardF_blockSizeID_t)((int)proposedBSID + 1); - } - return requestedBSID; -} - - -static BYTE LizardF_headerChecksum (const void* header, size_t length) -{ - U32 xxh = XXH32(header, length, 0); - return (BYTE)(xxh >> 8); -} - - -/*-************************************ -* Simple compression functions -**************************************/ - -size_t LizardF_compressFrameBound(size_t srcSize, const LizardF_preferences_t* preferencesPtr) -{ - LizardF_preferences_t prefs; - size_t headerSize; - size_t streamSize; - - if (preferencesPtr!=NULL) prefs = *preferencesPtr; - else memset(&prefs, 0, sizeof(prefs)); - - prefs.frameInfo.blockSizeID = LizardF_optimalBSID(prefs.frameInfo.blockSizeID, srcSize); - prefs.autoFlush = 1; - - headerSize = maxFHSize; /* header size, including magic number and frame content size*/ - streamSize = LizardF_compressBound(srcSize, &prefs); - - return headerSize + streamSize; -} - - - -/*! LizardF_compressFrame() : -* Compress an entire srcBuffer into a valid Lizard frame, as defined by specification v1.5.0, in a single step. -* The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. -* You can get the minimum value of dstMaxSize by using LizardF_compressFrameBound() -* If this condition is not respected, LizardF_compressFrame() will fail (result is an errorCode) -* The LizardF_preferences_t structure is optional : you can provide NULL as argument. All preferences will then be set to default. -* The result of the function is the number of bytes written into dstBuffer. -* The function outputs an error code if it fails (can be tested using LizardF_isError()) -*/ -size_t LizardF_compressFrame(void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_preferences_t* preferencesPtr) -{ - LizardF_cctx_t cctxI; - LizardF_preferences_t prefs; - LizardF_compressOptions_t options; - LizardF_errorCode_t errorCode; - BYTE* const dstStart = (BYTE*) dstBuffer; - BYTE* dstPtr = dstStart; - BYTE* const dstEnd = dstStart + dstMaxSize; - - memset(&cctxI, 0, sizeof(cctxI)); /* works because no allocation */ - memset(&options, 0, sizeof(options)); - - cctxI.version = LIZARDF_VERSION; - cctxI.maxBufferSize = 5 MB; /* mess with real buffer size to prevent allocation; works because autoflush==1 & stableSrc==1 */ - - if (preferencesPtr!=NULL) - prefs = *preferencesPtr; - else - memset(&prefs, 0, sizeof(prefs)); - if (prefs.frameInfo.contentSize != 0) - prefs.frameInfo.contentSize = (U64)srcSize; /* auto-correct content size if selected (!=0) */ - - prefs.frameInfo.blockSizeID = LizardF_optimalBSID(prefs.frameInfo.blockSizeID, srcSize); - prefs.autoFlush = 1; - if (srcSize <= LizardF_getBlockSize(prefs.frameInfo.blockSizeID)) - prefs.frameInfo.blockMode = LizardF_blockIndependent; /* no need for linked blocks */ - - options.stableSrc = 1; - - if (dstMaxSize < LizardF_compressFrameBound(srcSize, &prefs)) - return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; - - errorCode = LizardF_compressBegin(&cctxI, dstBuffer, dstMaxSize, &prefs); /* write header */ - if (LizardF_isError(errorCode)) goto error; - dstPtr += errorCode; /* header size */ - - errorCode = LizardF_compressUpdate(&cctxI, dstPtr, dstEnd-dstPtr, srcBuffer, srcSize, &options); - if (LizardF_isError(errorCode)) goto error; - dstPtr += errorCode; - - errorCode = LizardF_compressEnd(&cctxI, dstPtr, dstEnd-dstPtr, &options); /* flush last block, and generate suffix */ - if (LizardF_isError(errorCode)) goto error; - dstPtr += errorCode; - - Lizard_freeStream(cctxI.lizardCtxPtr); - FREEMEM(cctxI.tmpBuff); - return (dstPtr - dstStart); -error: - Lizard_freeStream(cctxI.lizardCtxPtr); - FREEMEM(cctxI.tmpBuff); - return errorCode; -} - - -/*-********************************* -* Advanced compression functions -***********************************/ - -/* LizardF_createCompressionContext() : -* The first thing to do is to create a compressionContext object, which will be used in all compression operations. -* This is achieved using LizardF_createCompressionContext(), which takes as argument a version and an LizardF_preferences_t structure. -* The version provided MUST be LIZARDF_VERSION. It is intended to track potential version differences between different binaries. -* The function will provide a pointer to an allocated LizardF_compressionContext_t object. -* If the result LizardF_errorCode_t is not OK_NoError, there was an error during context creation. -* Object can release its memory using LizardF_freeCompressionContext(); -*/ -LizardF_errorCode_t LizardF_createCompressionContext(LizardF_compressionContext_t* LizardF_compressionContextPtr, unsigned version) -{ - LizardF_cctx_t* cctxPtr; - - cctxPtr = (LizardF_cctx_t*)ALLOCATOR(1, sizeof(LizardF_cctx_t)); - if (cctxPtr==NULL) return (LizardF_errorCode_t)(-LizardF_ERROR_allocation_failed); - - cctxPtr->version = version; - cctxPtr->cStage = 0; /* Next stage : write header */ - - *LizardF_compressionContextPtr = (LizardF_compressionContext_t)cctxPtr; - - return LizardF_OK_NoError; -} - - -LizardF_errorCode_t LizardF_freeCompressionContext(LizardF_compressionContext_t LizardF_compressionContext) -{ - LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)LizardF_compressionContext; - - if (cctxPtr != NULL) { /* null pointers can be safely provided to this function, like free() */ - Lizard_freeStream(cctxPtr->lizardCtxPtr); - FREEMEM(cctxPtr->tmpBuff); - FREEMEM(LizardF_compressionContext); - } - - return LizardF_OK_NoError; -} - - -/*! LizardF_compressBegin() : -* will write the frame header into dstBuffer. -* dstBuffer must be large enough to accommodate a header (dstMaxSize). Maximum header size is LizardF_MAXHEADERFRAME_SIZE bytes. -* The result of the function is the number of bytes written into dstBuffer for the header -* or an error code (can be tested using LizardF_isError()) -*/ -size_t LizardF_compressBegin(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const LizardF_preferences_t* preferencesPtr) -{ - LizardF_preferences_t prefNull; - LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; - BYTE* const dstStart = (BYTE*)dstBuffer; - BYTE* dstPtr = dstStart; - BYTE* headerStart; - size_t requiredBuffSize; - - if (dstMaxSize < maxFHSize) return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; - if (cctxPtr->cStage != 0) return (size_t)-LizardF_ERROR_GENERIC; - memset(&prefNull, 0, sizeof(prefNull)); - if (preferencesPtr == NULL) preferencesPtr = &prefNull; - cctxPtr->prefs = *preferencesPtr; - - /* ctx Management */ - if (cctxPtr->lizardCtxLevel == 0) { - cctxPtr->lizardCtxPtr = Lizard_createStream(cctxPtr->prefs.compressionLevel); - cctxPtr->lizardCtxLevel = 1; - } - - /* Buffer Management */ - if (cctxPtr->prefs.frameInfo.blockSizeID == 0) cctxPtr->prefs.frameInfo.blockSizeID = LIZARDF_BLOCKSIZEID_DEFAULT; - cctxPtr->maxBlockSize = LizardF_getBlockSize(cctxPtr->prefs.frameInfo.blockSizeID); - requiredBuffSize = cctxPtr->maxBlockSize + ((cctxPtr->prefs.frameInfo.blockMode == LizardF_blockLinked) * 2 * LIZARD_DICT_SIZE); - - if (preferencesPtr->autoFlush) - requiredBuffSize = (cctxPtr->prefs.frameInfo.blockMode == LizardF_blockLinked) * LIZARD_DICT_SIZE; /* just needs dict */ - - if (cctxPtr->maxBufferSize < requiredBuffSize) { - cctxPtr->maxBufferSize = requiredBuffSize; - FREEMEM(cctxPtr->tmpBuff); - cctxPtr->tmpBuff = (BYTE*)ALLOCATOR(1, requiredBuffSize); - if (cctxPtr->tmpBuff == NULL) { printf("ERROR in LizardF_compressBegin: Cannot allocate %d MB\n", (int)(requiredBuffSize>>20)); return (size_t)-LizardF_ERROR_allocation_failed; } - } - cctxPtr->tmpIn = cctxPtr->tmpBuff; - cctxPtr->tmpInSize = 0; - XXH32_reset(&(cctxPtr->xxh), 0); - cctxPtr->lizardCtxPtr = Lizard_resetStream((Lizard_stream_t*)(cctxPtr->lizardCtxPtr), cctxPtr->prefs.compressionLevel); - if (!cctxPtr->lizardCtxPtr) return (size_t)-LizardF_ERROR_allocation_failed; - - /* Magic Number */ - LizardF_writeLE32(dstPtr, LIZARDF_MAGICNUMBER); - dstPtr += 4; - headerStart = dstPtr; - - /* FLG Byte */ - *dstPtr++ = (BYTE)(((1 & _2BITS) << 6) /* Version('01') */ - + ((cctxPtr->prefs.frameInfo.blockMode & _1BIT ) << 5) /* Block mode */ - + ((cctxPtr->prefs.frameInfo.contentChecksumFlag & _1BIT ) << 2) /* Frame checksum */ - + ((cctxPtr->prefs.frameInfo.contentSize > 0) << 3)); /* Frame content size */ - /* BD Byte */ - *dstPtr++ = (BYTE)((cctxPtr->prefs.frameInfo.blockSizeID & _3BITS) << 4); - /* Optional Frame content size field */ - if (cctxPtr->prefs.frameInfo.contentSize) { - LizardF_writeLE64(dstPtr, cctxPtr->prefs.frameInfo.contentSize); - dstPtr += 8; - cctxPtr->totalInSize = 0; - } - /* CRC Byte */ - *dstPtr = LizardF_headerChecksum(headerStart, dstPtr - headerStart); - dstPtr++; - - cctxPtr->cStage = 1; /* header written, now request input data block */ - - return (dstPtr - dstStart); -} - - -/* LizardF_compressBound() : gives the size of Dst buffer given a srcSize to handle worst case situations. -* The LizardF_frameInfo_t structure is optional : -* you can provide NULL as argument, preferences will then be set to cover worst case situations. -* */ -size_t LizardF_compressBound(size_t srcSize, const LizardF_preferences_t* preferencesPtr) -{ - LizardF_preferences_t prefsNull; - memset(&prefsNull, 0, sizeof(prefsNull)); - prefsNull.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; /* worst case */ - { const LizardF_preferences_t* prefsPtr = (preferencesPtr==NULL) ? &prefsNull : preferencesPtr; - LizardF_blockSizeID_t bid = prefsPtr->frameInfo.blockSizeID; - size_t blockSize = LizardF_getBlockSize(bid); - unsigned nbBlocks = (unsigned)(srcSize / blockSize) + 1; - size_t lastBlockSize = prefsPtr->autoFlush ? srcSize % blockSize : blockSize; - size_t blockInfo = 4; /* default, without block CRC option */ - size_t frameEnd = 4 + (prefsPtr->frameInfo.contentChecksumFlag*4); - - return (blockInfo * nbBlocks) + (blockSize * (nbBlocks-1)) + lastBlockSize + frameEnd;; - } -} - - -typedef int (*compressFunc_t)(void* ctx, const char* src, char* dst, int srcSize, int dstSize, int level); - -static size_t LizardF_compressBlock(void* dst, const void* src, size_t srcSize, compressFunc_t compress, void* lizardctx, int level) -{ - /* compress one block */ - BYTE* cSizePtr = (BYTE*)dst; - U32 cSize; - cSize = (U32)compress(lizardctx, (const char*)src, (char*)(cSizePtr+4), (int)(srcSize), (int)(srcSize-1), level); - LizardF_writeLE32(cSizePtr, cSize); - if (cSize == 0) { /* compression failed */ - cSize = (U32)srcSize; - LizardF_writeLE32(cSizePtr, cSize + LIZARDF_BLOCKUNCOMPRESSED_FLAG); - memcpy(cSizePtr+4, src, srcSize); - } - return cSize + 4; -} - - - -static int LizardF_localLizard_compress_continue(void* ctx, const char* src, char* dst, int srcSize, int dstSize, int level) -{ - (void)level; - return Lizard_compress_continue((Lizard_stream_t*)ctx, src, dst, srcSize, dstSize); -} - -static compressFunc_t LizardF_selectCompression(LizardF_blockMode_t blockMode) -{ - if (blockMode == LizardF_blockIndependent) return Lizard_compress_extState; - return LizardF_localLizard_compress_continue; -} - -static int LizardF_localSaveDict(LizardF_cctx_t* cctxPtr) -{ - return Lizard_saveDict ((Lizard_stream_t*)(cctxPtr->lizardCtxPtr), (char*)(cctxPtr->tmpBuff), LIZARD_DICT_SIZE); -} - -typedef enum { notDone, fromTmpBuffer, fromSrcBuffer } LizardF_lastBlockStatus; - -/*! LizardF_compressUpdate() : -* LizardF_compressUpdate() can be called repetitively to compress as much data as necessary. -* The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. -* If this condition is not respected, LizardF_compress() will fail (result is an errorCode) -* You can get the minimum value of dstMaxSize by using LizardF_compressBound() -* The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. -* The result of the function is the number of bytes written into dstBuffer : it can be zero, meaning input data was just buffered. -* The function outputs an error code if it fails (can be tested using LizardF_isError()) -*/ -size_t LizardF_compressUpdate(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_compressOptions_t* compressOptionsPtr) -{ - LizardF_compressOptions_t cOptionsNull; - LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; - size_t blockSize = cctxPtr->maxBlockSize; - const BYTE* srcPtr = (const BYTE*)srcBuffer; - const BYTE* const srcEnd = srcPtr + srcSize; - BYTE* const dstStart = (BYTE*)dstBuffer; - BYTE* dstPtr = dstStart; - LizardF_lastBlockStatus lastBlockCompressed = notDone; - compressFunc_t compress; - - - if (cctxPtr->cStage != 1) return (size_t)-LizardF_ERROR_GENERIC; - if (dstMaxSize < LizardF_compressBound(srcSize, &(cctxPtr->prefs))) return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; - memset(&cOptionsNull, 0, sizeof(cOptionsNull)); - if (compressOptionsPtr == NULL) compressOptionsPtr = &cOptionsNull; - - /* select compression function */ - compress = LizardF_selectCompression(cctxPtr->prefs.frameInfo.blockMode); - - /* complete tmp buffer */ - if (cctxPtr->tmpInSize > 0) { /* some data already within tmp buffer */ - size_t sizeToCopy = blockSize - cctxPtr->tmpInSize; - if (sizeToCopy > srcSize) { - /* add src to tmpIn buffer */ - memcpy(cctxPtr->tmpIn + cctxPtr->tmpInSize, srcBuffer, srcSize); - srcPtr = srcEnd; - cctxPtr->tmpInSize += srcSize; - /* still needs some CRC */ - } else { - /* complete tmpIn block and then compress it */ - lastBlockCompressed = fromTmpBuffer; - memcpy(cctxPtr->tmpIn + cctxPtr->tmpInSize, srcBuffer, sizeToCopy); - srcPtr += sizeToCopy; - - dstPtr += LizardF_compressBlock(dstPtr, cctxPtr->tmpIn, blockSize, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); - - if (cctxPtr->prefs.frameInfo.blockMode==LizardF_blockLinked) cctxPtr->tmpIn += blockSize; - cctxPtr->tmpInSize = 0; - } - } - - while ((size_t)(srcEnd - srcPtr) >= blockSize) { - /* compress full block */ - lastBlockCompressed = fromSrcBuffer; - dstPtr += LizardF_compressBlock(dstPtr, srcPtr, blockSize, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); - srcPtr += blockSize; - } - - if ((cctxPtr->prefs.autoFlush) && (srcPtr < srcEnd)) { - /* compress remaining input < blockSize */ - lastBlockCompressed = fromSrcBuffer; - dstPtr += LizardF_compressBlock(dstPtr, srcPtr, srcEnd - srcPtr, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); - srcPtr = srcEnd; - } - - /* preserve dictionary if necessary */ - if ((cctxPtr->prefs.frameInfo.blockMode==LizardF_blockLinked) && (lastBlockCompressed==fromSrcBuffer)) { - if (compressOptionsPtr->stableSrc) { - cctxPtr->tmpIn = cctxPtr->tmpBuff; - } else { - int realDictSize = LizardF_localSaveDict(cctxPtr); - if (realDictSize==0) return (size_t)-LizardF_ERROR_GENERIC; - cctxPtr->tmpIn = cctxPtr->tmpBuff + realDictSize; - } - } - - /* keep tmpIn within limits */ - if ((cctxPtr->tmpIn + blockSize) > (cctxPtr->tmpBuff + cctxPtr->maxBufferSize) /* necessarily LizardF_blockLinked && lastBlockCompressed==fromTmpBuffer */ - && !(cctxPtr->prefs.autoFlush)) - { - int realDictSize = LizardF_localSaveDict(cctxPtr); - cctxPtr->tmpIn = cctxPtr->tmpBuff + realDictSize; - } - - /* some input data left, necessarily < blockSize */ - if (srcPtr < srcEnd) { - /* fill tmp buffer */ - size_t sizeToCopy = srcEnd - srcPtr; - memcpy(cctxPtr->tmpIn, srcPtr, sizeToCopy); - cctxPtr->tmpInSize = sizeToCopy; - } - - if (cctxPtr->prefs.frameInfo.contentChecksumFlag == LizardF_contentChecksumEnabled) - XXH32_update(&(cctxPtr->xxh), srcBuffer, srcSize); - - cctxPtr->totalInSize += srcSize; - return dstPtr - dstStart; -} - - -/*! LizardF_flush() : -* Should you need to create compressed data immediately, without waiting for a block to be filled, -* you can call Lizard_flush(), which will immediately compress any remaining data stored within compressionContext. -* The result of the function is the number of bytes written into dstBuffer -* (it can be zero, this means there was no data left within compressionContext) -* The function outputs an error code if it fails (can be tested using LizardF_isError()) -* The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. -*/ -size_t LizardF_flush(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* compressOptionsPtr) -{ - LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; - BYTE* const dstStart = (BYTE*)dstBuffer; - BYTE* dstPtr = dstStart; - compressFunc_t compress; - - - if (cctxPtr->tmpInSize == 0) return 0; /* nothing to flush */ - if (cctxPtr->cStage != 1) return (size_t)-LizardF_ERROR_GENERIC; - if (dstMaxSize < (cctxPtr->tmpInSize + 8)) return (size_t)-LizardF_ERROR_dstMaxSize_tooSmall; /* +8 : block header(4) + block checksum(4) */ - (void)compressOptionsPtr; /* not yet useful */ - - /* select compression function */ - compress = LizardF_selectCompression(cctxPtr->prefs.frameInfo.blockMode); - - /* compress tmp buffer */ - dstPtr += LizardF_compressBlock(dstPtr, cctxPtr->tmpIn, cctxPtr->tmpInSize, compress, cctxPtr->lizardCtxPtr, cctxPtr->prefs.compressionLevel); - if (cctxPtr->prefs.frameInfo.blockMode==LizardF_blockLinked) cctxPtr->tmpIn += cctxPtr->tmpInSize; - cctxPtr->tmpInSize = 0; - - /* keep tmpIn within limits */ - if ((cctxPtr->tmpIn + cctxPtr->maxBlockSize) > (cctxPtr->tmpBuff + cctxPtr->maxBufferSize)) { /* necessarily LizardF_blockLinked */ - int realDictSize = LizardF_localSaveDict(cctxPtr); - cctxPtr->tmpIn = cctxPtr->tmpBuff + realDictSize; - } - - return dstPtr - dstStart; -} - - -/*! LizardF_compressEnd() : -* When you want to properly finish the compressed frame, just call LizardF_compressEnd(). -* It will flush whatever data remained within compressionContext (like Lizard_flush()) -* but also properly finalize the frame, with an endMark and a checksum. -* The result of the function is the number of bytes written into dstBuffer (necessarily >= 4 (endMark size)) -* The function outputs an error code if it fails (can be tested using LizardF_isError()) -* The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. -* compressionContext can then be used again, starting with LizardF_compressBegin(). The preferences will remain the same. -*/ -size_t LizardF_compressEnd(LizardF_compressionContext_t compressionContext, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* compressOptionsPtr) -{ - LizardF_cctx_t* cctxPtr = (LizardF_cctx_t*)compressionContext; - BYTE* const dstStart = (BYTE*)dstBuffer; - BYTE* dstPtr = dstStart; - size_t errorCode; - - errorCode = LizardF_flush(compressionContext, dstBuffer, dstMaxSize, compressOptionsPtr); - if (LizardF_isError(errorCode)) return errorCode; - dstPtr += errorCode; - - LizardF_writeLE32(dstPtr, 0); - dstPtr+=4; /* endMark */ - - if (cctxPtr->prefs.frameInfo.contentChecksumFlag == LizardF_contentChecksumEnabled) { - U32 xxh = XXH32_digest(&(cctxPtr->xxh)); - LizardF_writeLE32(dstPtr, xxh); - dstPtr+=4; /* content Checksum */ - } - - cctxPtr->cStage = 0; /* state is now re-usable (with identical preferences) */ - cctxPtr->maxBufferSize = 0; /* reuse HC context */ - - if (cctxPtr->prefs.frameInfo.contentSize) { - if (cctxPtr->prefs.frameInfo.contentSize != cctxPtr->totalInSize) - return (size_t)-LizardF_ERROR_frameSize_wrong; - } - - return dstPtr - dstStart; -} - - -/*-*************************************************** -* Frame Decompression -*****************************************************/ - -/* Resource management */ - -/*! LizardF_createDecompressionContext() : -* Create a decompressionContext object, which will track all decompression operations. -* Provides a pointer to a fully allocated and initialized LizardF_decompressionContext object. -* Object can later be released using LizardF_freeDecompressionContext(). -* @return : if != 0, there was an error during context creation. -*/ -LizardF_errorCode_t LizardF_createDecompressionContext(LizardF_decompressionContext_t* LizardF_decompressionContextPtr, unsigned versionNumber) -{ - LizardF_dctx_t* const dctxPtr = (LizardF_dctx_t*)ALLOCATOR(1, sizeof(LizardF_dctx_t)); - if (dctxPtr==NULL) return (LizardF_errorCode_t)-LizardF_ERROR_GENERIC; - - dctxPtr->version = versionNumber; - *LizardF_decompressionContextPtr = (LizardF_decompressionContext_t)dctxPtr; - return LizardF_OK_NoError; -} - -LizardF_errorCode_t LizardF_freeDecompressionContext(LizardF_decompressionContext_t LizardF_decompressionContext) -{ - LizardF_errorCode_t result = LizardF_OK_NoError; - LizardF_dctx_t* const dctxPtr = (LizardF_dctx_t*)LizardF_decompressionContext; - if (dctxPtr != NULL) { /* can accept NULL input, like free() */ - result = (LizardF_errorCode_t)dctxPtr->dStage; - FREEMEM(dctxPtr->tmpIn); - FREEMEM(dctxPtr->tmpOutBuffer); - FREEMEM(dctxPtr); - } - return result; -} - - -/* ******************************************************************** */ -/* ********************* Decompression ******************************** */ -/* ******************************************************************** */ - -typedef enum { dstage_getHeader=0, dstage_storeHeader, - dstage_getCBlockSize, dstage_storeCBlockSize, - dstage_copyDirect, - dstage_getCBlock, dstage_storeCBlock, - dstage_decodeCBlock, dstage_decodeCBlock_intoDst, - dstage_decodeCBlock_intoTmp, dstage_flushOut, - dstage_getSuffix, dstage_storeSuffix, - dstage_getSFrameSize, dstage_storeSFrameSize, - dstage_skipSkippable -} dStage_t; - - -/*! LizardF_headerSize() : -* @return : size of frame header -* or an error code, which can be tested using LizardF_isError() -*/ -static size_t LizardF_headerSize(const void* src, size_t srcSize) -{ - /* minimal srcSize to determine header size */ - if (srcSize < 5) return (size_t)-LizardF_ERROR_frameHeader_incomplete; - - /* special case : skippable frames */ - if ((LizardF_readLE32(src) & 0xFFFFFFF0U) == LIZARDF_MAGIC_SKIPPABLE_START) return 8; - - /* control magic number */ - if (LizardF_readLE32(src) != LIZARDF_MAGICNUMBER) return (size_t)-LizardF_ERROR_frameType_unknown; - - /* Frame Header Size */ - { BYTE const FLG = ((const BYTE*)src)[4]; - U32 const contentSizeFlag = (FLG>>3) & _1BIT; - return contentSizeFlag ? maxFHSize : minFHSize; - } -} - - -/*! LizardF_decodeHeader() : - input : `srcVoidPtr` points at the **beginning of the frame** - output : set internal values of dctx, such as - dctxPtr->frameInfo and dctxPtr->dStage. - Also allocates internal buffers. - @return : nb Bytes read from srcVoidPtr (necessarily <= srcSize) - or an error code (testable with LizardF_isError()) -*/ -static size_t LizardF_decodeHeader(LizardF_dctx_t* dctxPtr, const void* srcVoidPtr, size_t srcSize) -{ - BYTE FLG, BD, HC; - unsigned version, blockMode, blockChecksumFlag, contentSizeFlag, contentChecksumFlag, blockSizeID; - size_t bufferNeeded, currentBlockSize; - size_t frameHeaderSize; - const BYTE* srcPtr = (const BYTE*)srcVoidPtr; - - /* need to decode header to get frameInfo */ - if (srcSize < minFHSize) return (size_t)-LizardF_ERROR_frameHeader_incomplete; /* minimal frame header size */ - memset(&(dctxPtr->frameInfo), 0, sizeof(dctxPtr->frameInfo)); - - /* special case : skippable frames */ - if ((LizardF_readLE32(srcPtr) & 0xFFFFFFF0U) == LIZARDF_MAGIC_SKIPPABLE_START) { - dctxPtr->frameInfo.frameType = LizardF_skippableFrame; - if (srcVoidPtr == (void*)(dctxPtr->header)) { - dctxPtr->tmpInSize = srcSize; - dctxPtr->tmpInTarget = 8; - dctxPtr->dStage = dstage_storeSFrameSize; - return srcSize; - } else { - dctxPtr->dStage = dstage_getSFrameSize; - return 4; - } - } - - /* control magic number */ - if (LizardF_readLE32(srcPtr) != LIZARDF_MAGICNUMBER) return (size_t)-LizardF_ERROR_frameType_unknown; - dctxPtr->frameInfo.frameType = LizardF_frame; - - /* Flags */ - FLG = srcPtr[4]; - version = (FLG>>6) & _2BITS; - blockMode = (FLG>>5) & _1BIT; - blockChecksumFlag = (FLG>>4) & _1BIT; - contentSizeFlag = (FLG>>3) & _1BIT; - contentChecksumFlag = (FLG>>2) & _1BIT; - - /* Frame Header Size */ - frameHeaderSize = contentSizeFlag ? maxFHSize : minFHSize; - - if (srcSize < frameHeaderSize) { - /* not enough input to fully decode frame header */ - if (srcPtr != dctxPtr->header) - memcpy(dctxPtr->header, srcPtr, srcSize); - dctxPtr->tmpInSize = srcSize; - dctxPtr->tmpInTarget = frameHeaderSize; - dctxPtr->dStage = dstage_storeHeader; - return srcSize; - } - - BD = srcPtr[5]; - blockSizeID = (BD>>4) & _3BITS; - - /* validate */ - if (version != 1) return (size_t)-LizardF_ERROR_headerVersion_wrong; /* Version Number, only supported value */ - if (blockChecksumFlag != 0) return (size_t)-LizardF_ERROR_blockChecksum_unsupported; /* Not supported for the time being */ - if (((FLG>>0)&_2BITS) != 0) return (size_t)-LizardF_ERROR_reservedFlag_set; /* Reserved bits */ - if (((BD>>7)&_1BIT) != 0) return (size_t)-LizardF_ERROR_reservedFlag_set; /* Reserved bit */ - if (blockSizeID < 1) return (size_t)-LizardF_ERROR_maxBlockSize_invalid; /* 1-7 only supported values for the time being */ - if (((BD>>0)&_4BITS) != 0) return (size_t)-LizardF_ERROR_reservedFlag_set; /* Reserved bits */ - - /* check */ - HC = LizardF_headerChecksum(srcPtr+4, frameHeaderSize-5); - if (HC != srcPtr[frameHeaderSize-1]) return (size_t)-LizardF_ERROR_headerChecksum_invalid; /* Bad header checksum error */ - - /* save */ - dctxPtr->frameInfo.blockMode = (LizardF_blockMode_t)blockMode; - dctxPtr->frameInfo.contentChecksumFlag = (LizardF_contentChecksum_t)contentChecksumFlag; - dctxPtr->frameInfo.blockSizeID = (LizardF_blockSizeID_t)blockSizeID; - currentBlockSize = dctxPtr->maxBlockSize; - dctxPtr->maxBlockSize = LizardF_getBlockSize(blockSizeID); - if (contentSizeFlag) - dctxPtr->frameRemainingSize = dctxPtr->frameInfo.contentSize = LizardF_readLE64(srcPtr+6); - - /* init */ - if (contentChecksumFlag) XXH32_reset(&(dctxPtr->xxh), 0); - - /* alloc */ - bufferNeeded = dctxPtr->maxBlockSize + ((dctxPtr->frameInfo.blockMode==LizardF_blockLinked) * 2 * LIZARD_DICT_SIZE); - if (bufferNeeded > dctxPtr->maxBufferSize || dctxPtr->maxBlockSize > currentBlockSize) { /* tmp buffers too small */ - FREEMEM(dctxPtr->tmpIn); - FREEMEM(dctxPtr->tmpOutBuffer); - dctxPtr->maxBufferSize = 0; - dctxPtr->tmpIn = (BYTE*)ALLOCATOR(1, dctxPtr->maxBlockSize); - if (dctxPtr->tmpIn == NULL) return (size_t)-LizardF_ERROR_GENERIC; - dctxPtr->tmpOutBuffer= (BYTE*)ALLOCATOR(1, bufferNeeded); - if (dctxPtr->tmpOutBuffer== NULL) return (size_t)-LizardF_ERROR_GENERIC; - dctxPtr->maxBufferSize = bufferNeeded; - } - dctxPtr->tmpInSize = 0; - dctxPtr->tmpInTarget = 0; - dctxPtr->dict = dctxPtr->tmpOutBuffer; - dctxPtr->dictSize = 0; - dctxPtr->tmpOut = dctxPtr->tmpOutBuffer; - dctxPtr->tmpOutStart = 0; - dctxPtr->tmpOutSize = 0; - - dctxPtr->dStage = dstage_getCBlockSize; - - return frameHeaderSize; -} - - -/*! LizardF_getFrameInfo() : -* Decodes frame header information, such as blockSize. -* It is optional : you could start by calling directly LizardF_decompress() instead. -* The objective is to extract header information without starting decompression, typically for allocation purposes. -* LizardF_getFrameInfo() can also be used *after* starting decompression, on a valid LizardF_decompressionContext_t. -* The number of bytes read from srcBuffer will be provided within *srcSizePtr (necessarily <= original value). -* You are expected to resume decompression from where it stopped (srcBuffer + *srcSizePtr) -* @return : hint of the better `srcSize` to use for next call to LizardF_decompress, -* or an error code which can be tested using LizardF_isError(). -*/ -LizardF_errorCode_t LizardF_getFrameInfo(LizardF_decompressionContext_t dCtx, LizardF_frameInfo_t* frameInfoPtr, - const void* srcBuffer, size_t* srcSizePtr) -{ - LizardF_dctx_t* dctxPtr = (LizardF_dctx_t*)dCtx; - - if (dctxPtr->dStage > dstage_storeHeader) { /* note : requires dstage_* header related to be at beginning of enum */ - /* frameInfo already decoded */ - size_t o=0, i=0; - *srcSizePtr = 0; - *frameInfoPtr = dctxPtr->frameInfo; - return LizardF_decompress(dCtx, NULL, &o, NULL, &i, NULL); /* returns : recommended nb of bytes for LizardF_decompress() */ - } else { - size_t nextSrcSize, o=0; - size_t const hSize = LizardF_headerSize(srcBuffer, *srcSizePtr); - if (LizardF_isError(hSize)) { *srcSizePtr=0; return hSize; } - if (*srcSizePtr < hSize) { *srcSizePtr=0; return (size_t)-LizardF_ERROR_frameHeader_incomplete; } - - *srcSizePtr = hSize; - nextSrcSize = LizardF_decompress(dCtx, NULL, &o, srcBuffer, srcSizePtr, NULL); - if (dctxPtr->dStage <= dstage_storeHeader) return (size_t)-LizardF_ERROR_frameHeader_incomplete; /* should not happen, already checked */ - *frameInfoPtr = dctxPtr->frameInfo; - return nextSrcSize; - } -} - - -/* trivial redirector, for common prototype */ -static int LizardF_decompress_safe (const char* source, char* dest, int compressedSize, int maxDecompressedSize, const char* dictStart, int dictSize) -{ - (void)dictStart; (void)dictSize; - return Lizard_decompress_safe (source, dest, compressedSize, maxDecompressedSize); -} - - -static void LizardF_updateDict(LizardF_dctx_t* dctxPtr, const BYTE* dstPtr, size_t dstSize, const BYTE* dstPtr0, unsigned withinTmp) -{ - if (dctxPtr->dictSize==0) - dctxPtr->dict = (const BYTE*)dstPtr; /* priority to dictionary continuity */ - - if (dctxPtr->dict + dctxPtr->dictSize == dstPtr) { /* dictionary continuity */ - dctxPtr->dictSize += dstSize; - return; - } - - if (dstPtr - dstPtr0 + dstSize >= LIZARD_DICT_SIZE) { /* dstBuffer large enough to become dictionary */ - dctxPtr->dict = (const BYTE*)dstPtr0; - dctxPtr->dictSize = dstPtr - dstPtr0 + dstSize; - return; - } - - if ((withinTmp) && (dctxPtr->dict == dctxPtr->tmpOutBuffer)) { - /* assumption : dctxPtr->dict + dctxPtr->dictSize == dctxPtr->tmpOut + dctxPtr->tmpOutStart */ - dctxPtr->dictSize += dstSize; - return; - } - - if (withinTmp) { /* copy relevant dict portion in front of tmpOut within tmpOutBuffer */ - size_t preserveSize = dctxPtr->tmpOut - dctxPtr->tmpOutBuffer; - size_t copySize = LIZARD_DICT_SIZE - dctxPtr->tmpOutSize; - const BYTE* oldDictEnd = dctxPtr->dict + dctxPtr->dictSize - dctxPtr->tmpOutStart; - if (dctxPtr->tmpOutSize > LIZARD_DICT_SIZE) copySize = 0; - if (copySize > preserveSize) copySize = preserveSize; - - memcpy(dctxPtr->tmpOutBuffer + preserveSize - copySize, oldDictEnd - copySize, copySize); - - dctxPtr->dict = dctxPtr->tmpOutBuffer; - dctxPtr->dictSize = preserveSize + dctxPtr->tmpOutStart + dstSize; - return; - } - - if (dctxPtr->dict == dctxPtr->tmpOutBuffer) { /* copy dst into tmp to complete dict */ - if (dctxPtr->dictSize + dstSize > dctxPtr->maxBufferSize) { /* tmp buffer not large enough */ - size_t preserveSize = LIZARD_DICT_SIZE - dstSize; /* note : dstSize < LIZARD_DICT_SIZE */ - memcpy(dctxPtr->tmpOutBuffer, dctxPtr->dict + dctxPtr->dictSize - preserveSize, preserveSize); - dctxPtr->dictSize = preserveSize; - } - memcpy(dctxPtr->tmpOutBuffer + dctxPtr->dictSize, dstPtr, dstSize); - dctxPtr->dictSize += dstSize; - return; - } - - /* join dict & dest into tmp */ - { size_t preserveSize = LIZARD_DICT_SIZE - dstSize; /* note : dstSize < LIZARD_DICT_SIZE */ - if (preserveSize > dctxPtr->dictSize) preserveSize = dctxPtr->dictSize; - memcpy(dctxPtr->tmpOutBuffer, dctxPtr->dict + dctxPtr->dictSize - preserveSize, preserveSize); - memcpy(dctxPtr->tmpOutBuffer + preserveSize, dstPtr, dstSize); - dctxPtr->dict = dctxPtr->tmpOutBuffer; - dctxPtr->dictSize = preserveSize + dstSize; - } -} - - - -/*! LizardF_decompress() : -* Call this function repetitively to regenerate data compressed within srcBuffer. -* The function will attempt to decode *srcSizePtr from srcBuffer, into dstBuffer of maximum size *dstSizePtr. -* -* The number of bytes regenerated into dstBuffer will be provided within *dstSizePtr (necessarily <= original value). -* -* The number of bytes effectively read from srcBuffer will be provided within *srcSizePtr (necessarily <= original value). -* If the number of bytes read is < number of bytes provided, then the decompression operation is not complete. -* You will have to call it again, continuing from where it stopped. -* -* The function result is an hint of the better srcSize to use for next call to LizardF_decompress. -* Basically, it's the size of the current (or remaining) compressed block + header of next block. -* Respecting the hint provides some boost to performance, since it allows less buffer shuffling. -* Note that this is just a hint, you can always provide any srcSize you want. -* When a frame is fully decoded, the function result will be 0. -* If decompression failed, function result is an error code which can be tested using LizardF_isError(). -*/ -size_t LizardF_decompress(LizardF_decompressionContext_t decompressionContext, - void* dstBuffer, size_t* dstSizePtr, - const void* srcBuffer, size_t* srcSizePtr, - const LizardF_decompressOptions_t* decompressOptionsPtr) -{ - LizardF_dctx_t* dctxPtr = (LizardF_dctx_t*)decompressionContext; - LizardF_decompressOptions_t optionsNull; - const BYTE* const srcStart = (const BYTE*)srcBuffer; - const BYTE* const srcEnd = srcStart + *srcSizePtr; - const BYTE* srcPtr = srcStart; - BYTE* const dstStart = (BYTE*)dstBuffer; - BYTE* const dstEnd = dstStart + *dstSizePtr; - BYTE* dstPtr = dstStart; - const BYTE* selectedIn = NULL; - unsigned doAnotherStage = 1; - size_t nextSrcSizeHint = 1; - - - memset(&optionsNull, 0, sizeof(optionsNull)); - if (decompressOptionsPtr==NULL) decompressOptionsPtr = &optionsNull; - *srcSizePtr = 0; - *dstSizePtr = 0; - - /* expect to continue decoding src buffer where it left previously */ - if (dctxPtr->srcExpect != NULL) { - if (srcStart != dctxPtr->srcExpect) return (size_t)-LizardF_ERROR_srcPtr_wrong; - } - - /* programmed as a state machine */ - - while (doAnotherStage) { - - switch(dctxPtr->dStage) - { - - case dstage_getHeader: - if ((size_t)(srcEnd-srcPtr) >= maxFHSize) { /* enough to decode - shortcut */ - LizardF_errorCode_t const hSize = LizardF_decodeHeader(dctxPtr, srcPtr, srcEnd-srcPtr); - if (LizardF_isError(hSize)) return hSize; - srcPtr += hSize; - break; - } - dctxPtr->tmpInSize = 0; - dctxPtr->tmpInTarget = minFHSize; /* minimum to attempt decode */ - dctxPtr->dStage = dstage_storeHeader; - /* fallthrough */ - - case dstage_storeHeader: - { size_t sizeToCopy = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; - if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; - memcpy(dctxPtr->header + dctxPtr->tmpInSize, srcPtr, sizeToCopy); - dctxPtr->tmpInSize += sizeToCopy; - srcPtr += sizeToCopy; - if (dctxPtr->tmpInSize < dctxPtr->tmpInTarget) { - nextSrcSizeHint = (dctxPtr->tmpInTarget - dctxPtr->tmpInSize) + BHSize; /* rest of header + nextBlockHeader */ - doAnotherStage = 0; /* not enough src data, ask for some more */ - break; - } - { LizardF_errorCode_t const hSize = LizardF_decodeHeader(dctxPtr, dctxPtr->header, dctxPtr->tmpInTarget); - if (LizardF_isError(hSize)) return hSize; - } - break; - } - - case dstage_getCBlockSize: - if ((size_t)(srcEnd - srcPtr) >= BHSize) { - selectedIn = srcPtr; - srcPtr += BHSize; - } else { - /* not enough input to read cBlockSize field */ - dctxPtr->tmpInSize = 0; - dctxPtr->dStage = dstage_storeCBlockSize; - } - - if (dctxPtr->dStage == dstage_storeCBlockSize) /* can be skipped */ - case dstage_storeCBlockSize: - { - size_t sizeToCopy = BHSize - dctxPtr->tmpInSize; - if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; - memcpy(dctxPtr->tmpIn + dctxPtr->tmpInSize, srcPtr, sizeToCopy); - srcPtr += sizeToCopy; - dctxPtr->tmpInSize += sizeToCopy; - if (dctxPtr->tmpInSize < BHSize) { /* not enough input to get full cBlockSize; wait for more */ - nextSrcSizeHint = BHSize - dctxPtr->tmpInSize; - doAnotherStage = 0; - break; - } - selectedIn = dctxPtr->tmpIn; - } - - /* case dstage_decodeCBlockSize: */ /* no more direct access, to prevent scan-build warning */ - { size_t const nextCBlockSize = LizardF_readLE32(selectedIn) & 0x7FFFFFFFU; - if (nextCBlockSize==0) { /* frameEnd signal, no more CBlock */ - dctxPtr->dStage = dstage_getSuffix; - break; - } - if (nextCBlockSize > dctxPtr->maxBlockSize) return (size_t)-LizardF_ERROR_GENERIC; /* invalid cBlockSize */ - dctxPtr->tmpInTarget = nextCBlockSize; - if (LizardF_readLE32(selectedIn) & LIZARDF_BLOCKUNCOMPRESSED_FLAG) { - dctxPtr->dStage = dstage_copyDirect; - break; - } - dctxPtr->dStage = dstage_getCBlock; - if (dstPtr==dstEnd) { - nextSrcSizeHint = nextCBlockSize + BHSize; - doAnotherStage = 0; - } - break; - } - - case dstage_copyDirect: /* uncompressed block */ - { size_t sizeToCopy = dctxPtr->tmpInTarget; - if ((size_t)(srcEnd-srcPtr) < sizeToCopy) sizeToCopy = srcEnd - srcPtr; /* not enough input to read full block */ - if ((size_t)(dstEnd-dstPtr) < sizeToCopy) sizeToCopy = dstEnd - dstPtr; - memcpy(dstPtr, srcPtr, sizeToCopy); - if (dctxPtr->frameInfo.contentChecksumFlag) XXH32_update(&(dctxPtr->xxh), srcPtr, sizeToCopy); - if (dctxPtr->frameInfo.contentSize) dctxPtr->frameRemainingSize -= sizeToCopy; - - /* dictionary management */ - if (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) - LizardF_updateDict(dctxPtr, dstPtr, sizeToCopy, dstStart, 0); - - srcPtr += sizeToCopy; - dstPtr += sizeToCopy; - if (sizeToCopy == dctxPtr->tmpInTarget) { /* all copied */ - dctxPtr->dStage = dstage_getCBlockSize; - break; - } - dctxPtr->tmpInTarget -= sizeToCopy; /* still need to copy more */ - nextSrcSizeHint = dctxPtr->tmpInTarget + BHSize; - doAnotherStage = 0; - break; - } - - case dstage_getCBlock: /* entry from dstage_decodeCBlockSize */ - if ((size_t)(srcEnd-srcPtr) < dctxPtr->tmpInTarget) { - dctxPtr->tmpInSize = 0; - dctxPtr->dStage = dstage_storeCBlock; - break; - } - selectedIn = srcPtr; - srcPtr += dctxPtr->tmpInTarget; - dctxPtr->dStage = dstage_decodeCBlock; - break; - - case dstage_storeCBlock: - { size_t sizeToCopy = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; - if (sizeToCopy > (size_t)(srcEnd-srcPtr)) sizeToCopy = srcEnd-srcPtr; - memcpy(dctxPtr->tmpIn + dctxPtr->tmpInSize, srcPtr, sizeToCopy); - dctxPtr->tmpInSize += sizeToCopy; - srcPtr += sizeToCopy; - if (dctxPtr->tmpInSize < dctxPtr->tmpInTarget) { /* need more input */ - nextSrcSizeHint = (dctxPtr->tmpInTarget - dctxPtr->tmpInSize) + BHSize; - doAnotherStage=0; - break; - } - selectedIn = dctxPtr->tmpIn; - dctxPtr->dStage = dstage_decodeCBlock; - } - /* fallthrough */ - - case dstage_decodeCBlock: - if ((size_t)(dstEnd-dstPtr) < dctxPtr->maxBlockSize) /* not enough place into dst : decode into tmpOut */ - dctxPtr->dStage = dstage_decodeCBlock_intoTmp; - else - dctxPtr->dStage = dstage_decodeCBlock_intoDst; - break; - - case dstage_decodeCBlock_intoDst: - { int (*decoder)(const char*, char*, int, int, const char*, int); - int decodedSize; - - if (dctxPtr->frameInfo.blockMode == LizardF_blockLinked) - decoder = Lizard_decompress_safe_usingDict; - else - decoder = LizardF_decompress_safe; - - decodedSize = decoder((const char*)selectedIn, (char*)dstPtr, (int)dctxPtr->tmpInTarget, (int)dctxPtr->maxBlockSize, (const char*)dctxPtr->dict, (int)dctxPtr->dictSize); - if (decodedSize < 0) return (size_t)-LizardF_ERROR_GENERIC; /* decompression failed */ - if (dctxPtr->frameInfo.contentChecksumFlag) XXH32_update(&(dctxPtr->xxh), dstPtr, decodedSize); - if (dctxPtr->frameInfo.contentSize) dctxPtr->frameRemainingSize -= decodedSize; - - /* dictionary management */ - if (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) - LizardF_updateDict(dctxPtr, dstPtr, decodedSize, dstStart, 0); - - dstPtr += decodedSize; - dctxPtr->dStage = dstage_getCBlockSize; - break; - } - - case dstage_decodeCBlock_intoTmp: - /* not enough place into dst : decode into tmpOut */ - { int (*decoder)(const char*, char*, int, int, const char*, int); - int decodedSize; - - if (dctxPtr->frameInfo.blockMode == LizardF_blockLinked) - decoder = Lizard_decompress_safe_usingDict; - else - decoder = LizardF_decompress_safe; - - /* ensure enough place for tmpOut */ - if (dctxPtr->frameInfo.blockMode == LizardF_blockLinked) { - if (dctxPtr->dict == dctxPtr->tmpOutBuffer) { - if (dctxPtr->dictSize > 2 * LIZARD_DICT_SIZE) { - memcpy(dctxPtr->tmpOutBuffer, dctxPtr->dict + dctxPtr->dictSize - LIZARD_DICT_SIZE, LIZARD_DICT_SIZE); - dctxPtr->dictSize = LIZARD_DICT_SIZE; - } - dctxPtr->tmpOut = dctxPtr->tmpOutBuffer + dctxPtr->dictSize; - } else { /* dict not within tmp */ - size_t reservedDictSpace = dctxPtr->dictSize; - if (reservedDictSpace > LIZARD_DICT_SIZE) reservedDictSpace = LIZARD_DICT_SIZE; - dctxPtr->tmpOut = dctxPtr->tmpOutBuffer + reservedDictSpace; - } - } - - /* Decode */ - decodedSize = decoder((const char*)selectedIn, (char*)dctxPtr->tmpOut, (int)dctxPtr->tmpInTarget, (int)dctxPtr->maxBlockSize, (const char*)dctxPtr->dict, (int)dctxPtr->dictSize); - if (decodedSize < 0) return (size_t)-LizardF_ERROR_decompressionFailed; /* decompression failed */ - if (dctxPtr->frameInfo.contentChecksumFlag) XXH32_update(&(dctxPtr->xxh), dctxPtr->tmpOut, decodedSize); - if (dctxPtr->frameInfo.contentSize) dctxPtr->frameRemainingSize -= decodedSize; - dctxPtr->tmpOutSize = decodedSize; - dctxPtr->tmpOutStart = 0; - dctxPtr->dStage = dstage_flushOut; - break; - } - - case dstage_flushOut: /* flush decoded data from tmpOut to dstBuffer */ - { size_t sizeToCopy = dctxPtr->tmpOutSize - dctxPtr->tmpOutStart; - if (sizeToCopy > (size_t)(dstEnd-dstPtr)) sizeToCopy = dstEnd-dstPtr; - memcpy(dstPtr, dctxPtr->tmpOut + dctxPtr->tmpOutStart, sizeToCopy); - - /* dictionary management */ - if (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) - LizardF_updateDict(dctxPtr, dstPtr, sizeToCopy, dstStart, 1); - - dctxPtr->tmpOutStart += sizeToCopy; - dstPtr += sizeToCopy; - - /* end of flush ? */ - if (dctxPtr->tmpOutStart == dctxPtr->tmpOutSize) { - dctxPtr->dStage = dstage_getCBlockSize; - break; - } - nextSrcSizeHint = BHSize; - doAnotherStage = 0; /* still some data to flush */ - break; - } - - case dstage_getSuffix: - { size_t const suffixSize = dctxPtr->frameInfo.contentChecksumFlag * 4; - if (dctxPtr->frameRemainingSize) return (size_t)-LizardF_ERROR_frameSize_wrong; /* incorrect frame size decoded */ - if (suffixSize == 0) { /* frame completed */ - nextSrcSizeHint = 0; - dctxPtr->dStage = dstage_getHeader; - doAnotherStage = 0; - break; - } - if ((srcEnd - srcPtr) < 4) { /* not enough size for entire CRC */ - dctxPtr->tmpInSize = 0; - dctxPtr->dStage = dstage_storeSuffix; - } else { - selectedIn = srcPtr; - srcPtr += 4; - } - } - - if (dctxPtr->dStage == dstage_storeSuffix) /* can be skipped */ - case dstage_storeSuffix: - { - size_t sizeToCopy = 4 - dctxPtr->tmpInSize; - if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; - memcpy(dctxPtr->tmpIn + dctxPtr->tmpInSize, srcPtr, sizeToCopy); - srcPtr += sizeToCopy; - dctxPtr->tmpInSize += sizeToCopy; - if (dctxPtr->tmpInSize < 4) { /* not enough input to read complete suffix */ - nextSrcSizeHint = 4 - dctxPtr->tmpInSize; - doAnotherStage=0; - break; - } - selectedIn = dctxPtr->tmpIn; - } - - /* case dstage_checkSuffix: */ /* no direct call, to avoid scan-build warning */ - { U32 const readCRC = LizardF_readLE32(selectedIn); - U32 const resultCRC = XXH32_digest(&(dctxPtr->xxh)); - if (readCRC != resultCRC) return (size_t)-LizardF_ERROR_contentChecksum_invalid; - nextSrcSizeHint = 0; - dctxPtr->dStage = dstage_getHeader; - doAnotherStage = 0; - break; - } - - case dstage_getSFrameSize: - if ((srcEnd - srcPtr) >= 4) { - selectedIn = srcPtr; - srcPtr += 4; - } else { - /* not enough input to read cBlockSize field */ - dctxPtr->tmpInSize = 4; - dctxPtr->tmpInTarget = 8; - dctxPtr->dStage = dstage_storeSFrameSize; - } - - if (dctxPtr->dStage == dstage_storeSFrameSize) - case dstage_storeSFrameSize: - { - size_t sizeToCopy = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; - if (sizeToCopy > (size_t)(srcEnd - srcPtr)) sizeToCopy = srcEnd - srcPtr; - memcpy(dctxPtr->header + dctxPtr->tmpInSize, srcPtr, sizeToCopy); - srcPtr += sizeToCopy; - dctxPtr->tmpInSize += sizeToCopy; - if (dctxPtr->tmpInSize < dctxPtr->tmpInTarget) { /* not enough input to get full sBlockSize; wait for more */ - nextSrcSizeHint = dctxPtr->tmpInTarget - dctxPtr->tmpInSize; - doAnotherStage = 0; - break; - } - selectedIn = dctxPtr->header + 4; - } - - /* case dstage_decodeSFrameSize: */ /* no direct access */ - { size_t const SFrameSize = LizardF_readLE32(selectedIn); - dctxPtr->frameInfo.contentSize = SFrameSize; - dctxPtr->tmpInTarget = SFrameSize; - dctxPtr->dStage = dstage_skipSkippable; - break; - } - - case dstage_skipSkippable: - { size_t skipSize = dctxPtr->tmpInTarget; - if (skipSize > (size_t)(srcEnd-srcPtr)) skipSize = srcEnd-srcPtr; - srcPtr += skipSize; - dctxPtr->tmpInTarget -= skipSize; - doAnotherStage = 0; - nextSrcSizeHint = dctxPtr->tmpInTarget; - if (nextSrcSizeHint) break; - dctxPtr->dStage = dstage_getHeader; - break; - } - } - } - - /* preserve dictionary within tmp if necessary */ - if ( (dctxPtr->frameInfo.blockMode==LizardF_blockLinked) - &&(dctxPtr->dict != dctxPtr->tmpOutBuffer) - &&(!decompressOptionsPtr->stableDst) - &&((unsigned)(dctxPtr->dStage-1) < (unsigned)(dstage_getSuffix-1)) - ) - { - if (dctxPtr->dStage == dstage_flushOut) { - size_t preserveSize = dctxPtr->tmpOut - dctxPtr->tmpOutBuffer; - size_t copySize = LIZARD_DICT_SIZE - dctxPtr->tmpOutSize; - const BYTE* oldDictEnd = dctxPtr->dict + dctxPtr->dictSize - dctxPtr->tmpOutStart; - if (dctxPtr->tmpOutSize > LIZARD_DICT_SIZE) copySize = 0; - if (copySize > preserveSize) copySize = preserveSize; - - memcpy(dctxPtr->tmpOutBuffer + preserveSize - copySize, oldDictEnd - copySize, copySize); - - dctxPtr->dict = dctxPtr->tmpOutBuffer; - dctxPtr->dictSize = preserveSize + dctxPtr->tmpOutStart; - } else { - size_t newDictSize = dctxPtr->dictSize; - const BYTE* oldDictEnd = dctxPtr->dict + dctxPtr->dictSize; - if ((newDictSize) > LIZARD_DICT_SIZE) newDictSize = LIZARD_DICT_SIZE; - - memcpy(dctxPtr->tmpOutBuffer, oldDictEnd - newDictSize, newDictSize); - - dctxPtr->dict = dctxPtr->tmpOutBuffer; - dctxPtr->dictSize = newDictSize; - dctxPtr->tmpOut = dctxPtr->tmpOutBuffer + newDictSize; - } - } - - /* require function to be called again from position where it stopped */ - if (srcPtrsrcExpect = srcPtr; - else - dctxPtr->srcExpect = NULL; - - *srcSizePtr = (srcPtr - srcStart); - *dstSizePtr = (dstPtr - dstStart); - return nextSrcSizeHint; -} diff --git a/contrib/lizard/lib/lizard_frame.h b/contrib/lizard/lib/lizard_frame.h deleted file mode 100644 index 7cdd20063c5..00000000000 --- a/contrib/lizard/lib/lizard_frame.h +++ /dev/null @@ -1,303 +0,0 @@ -/* - Lizard auto-framing library - Header File - Copyright (C) 2011-2015, Yann Collet - Copyright (C) 2016-2017, Przemyslaw Skibinski - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -/* LizardF is a stand-alone API to create Lizard-compressed frames - * conformant with specification v1.5.1. - * All related operations, including memory management, are handled internally by the library. - * You don't need lizard_compress.h when using lizard_frame.h. - * */ - -#pragma once - -#if defined (__cplusplus) -extern "C" { -#endif - -/*-************************************ -* Includes -**************************************/ -#include /* size_t */ - - -/*-************************************ -* Error management -**************************************/ -typedef size_t LizardF_errorCode_t; - -unsigned LizardF_isError(LizardF_errorCode_t code); -const char* LizardF_getErrorName(LizardF_errorCode_t code); /* return error code string; useful for debugging */ - - -/*-************************************ -* Frame compression types -**************************************/ -//#define LIZARDF_DISABLE_OBSOLETE_ENUMS -#ifndef LIZARDF_DISABLE_OBSOLETE_ENUMS -# define LIZARDF_OBSOLETE_ENUM(x) ,x -#else -# define LIZARDF_OBSOLETE_ENUM(x) -#endif - -typedef enum { - LizardF_default=0, - LizardF_max128KB=1, - LizardF_max256KB=2, - LizardF_max1MB=3, - LizardF_max4MB=4, - LizardF_max16MB=5, - LizardF_max64MB=6, - LizardF_max256MB=7 -} LizardF_blockSizeID_t; - -typedef enum { - LizardF_blockLinked=0, - LizardF_blockIndependent - LIZARDF_OBSOLETE_ENUM(blockLinked = LizardF_blockLinked) - LIZARDF_OBSOLETE_ENUM(blockIndependent = LizardF_blockIndependent) -} LizardF_blockMode_t; - -typedef enum { - LizardF_noContentChecksum=0, - LizardF_contentChecksumEnabled - LIZARDF_OBSOLETE_ENUM(noContentChecksum = LizardF_noContentChecksum) - LIZARDF_OBSOLETE_ENUM(contentChecksumEnabled = LizardF_contentChecksumEnabled) -} LizardF_contentChecksum_t; - -typedef enum { - LizardF_frame=0, - LizardF_skippableFrame - LIZARDF_OBSOLETE_ENUM(skippableFrame = LizardF_skippableFrame) -} LizardF_frameType_t; - -#ifndef LIZARDF_DISABLE_OBSOLETE_ENUMS -typedef LizardF_blockSizeID_t blockSizeID_t; -typedef LizardF_blockMode_t blockMode_t; -typedef LizardF_frameType_t frameType_t; -typedef LizardF_contentChecksum_t contentChecksum_t; -#endif - -typedef struct { - LizardF_blockSizeID_t blockSizeID; /* max64KB, max256KB, max1MB, max4MB ; 0 == default */ - LizardF_blockMode_t blockMode; /* blockLinked, blockIndependent ; 0 == default */ - LizardF_contentChecksum_t contentChecksumFlag; /* noContentChecksum, contentChecksumEnabled ; 0 == default */ - LizardF_frameType_t frameType; /* LizardF_frame, skippableFrame ; 0 == default */ - unsigned long long contentSize; /* Size of uncompressed (original) content ; 0 == unknown */ - unsigned reserved[2]; /* must be zero for forward compatibility */ -} LizardF_frameInfo_t; - -typedef struct { - LizardF_frameInfo_t frameInfo; - int compressionLevel; /* 0 == default (fast mode); values above 16 count as 16; values below 0 count as 0 */ - unsigned autoFlush; /* 1 == always flush (reduce need for tmp buffer) */ - unsigned reserved[4]; /* must be zero for forward compatibility */ -} LizardF_preferences_t; - - -/*-********************************* -* Simple compression function -***********************************/ -size_t LizardF_compressFrameBound(size_t srcSize, const LizardF_preferences_t* preferencesPtr); - -/*!LizardF_compressFrame() : - * Compress an entire srcBuffer into a valid Lizard frame, as defined by specification v1.5.1 - * The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. - * You can get the minimum value of dstMaxSize by using LizardF_compressFrameBound() - * If this condition is not respected, LizardF_compressFrame() will fail (result is an errorCode) - * The LizardF_preferences_t structure is optional : you can provide NULL as argument. All preferences will be set to default. - * The result of the function is the number of bytes written into dstBuffer. - * The function outputs an error code if it fails (can be tested using LizardF_isError()) - */ -size_t LizardF_compressFrame(void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_preferences_t* preferencesPtr); - - - -/*-*********************************** -* Advanced compression functions -*************************************/ -typedef struct LizardF_cctx_s* LizardF_compressionContext_t; /* must be aligned on 8-bytes */ - -typedef struct { - unsigned stableSrc; /* 1 == src content will remain available on future calls to LizardF_compress(); avoid saving src content within tmp buffer as future dictionary */ - unsigned reserved[3]; -} LizardF_compressOptions_t; - -/* Resource Management */ - -#define LIZARDF_VERSION 100 -LizardF_errorCode_t LizardF_createCompressionContext(LizardF_compressionContext_t* cctxPtr, unsigned version); -LizardF_errorCode_t LizardF_freeCompressionContext(LizardF_compressionContext_t cctx); -/* LizardF_createCompressionContext() : - * The first thing to do is to create a compressionContext object, which will be used in all compression operations. - * This is achieved using LizardF_createCompressionContext(), which takes as argument a version and an LizardF_preferences_t structure. - * The version provided MUST be LIZARDF_VERSION. It is intended to track potential version differences between different binaries. - * The function will provide a pointer to a fully allocated LizardF_compressionContext_t object. - * If the result LizardF_errorCode_t is not zero, there was an error during context creation. - * Object can release its memory using LizardF_freeCompressionContext(); - */ - - -/* Compression */ - -size_t LizardF_compressBegin(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const LizardF_preferences_t* prefsPtr); -/* LizardF_compressBegin() : - * will write the frame header into dstBuffer. - * dstBuffer must be large enough to accommodate a header (dstMaxSize). Maximum header size is 15 bytes. - * The LizardF_preferences_t structure is optional : you can provide NULL as argument, all preferences will then be set to default. - * The result of the function is the number of bytes written into dstBuffer for the header - * or an error code (can be tested using LizardF_isError()) - */ - -size_t LizardF_compressBound(size_t srcSize, const LizardF_preferences_t* prefsPtr); -/* LizardF_compressBound() : - * Provides the minimum size of Dst buffer given srcSize to handle worst case situations. - * Different preferences can produce different results. - * prefsPtr is optional : you can provide NULL as argument, all preferences will then be set to cover worst case. - * This function includes frame termination cost (4 bytes, or 8 if frame checksum is enabled) - */ - -size_t LizardF_compressUpdate(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const void* srcBuffer, size_t srcSize, const LizardF_compressOptions_t* cOptPtr); -/* LizardF_compressUpdate() - * LizardF_compressUpdate() can be called repetitively to compress as much data as necessary. - * The most important rule is that dstBuffer MUST be large enough (dstMaxSize) to ensure compression completion even in worst case. - * You can get the minimum value of dstMaxSize by using LizardF_compressBound(). - * If this condition is not respected, LizardF_compress() will fail (result is an errorCode). - * LizardF_compressUpdate() doesn't guarantee error recovery, so you have to reset compression context when an error occurs. - * The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. - * The result of the function is the number of bytes written into dstBuffer : it can be zero, meaning input data was just buffered. - * The function outputs an error code if it fails (can be tested using LizardF_isError()) - */ - -size_t LizardF_flush(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* cOptPtr); -/* LizardF_flush() - * Should you need to generate compressed data immediately, without waiting for the current block to be filled, - * you can call Lizard_flush(), which will immediately compress any remaining data buffered within cctx. - * Note that dstMaxSize must be large enough to ensure the operation will be successful. - * LizardF_compressOptions_t structure is optional : you can provide NULL as argument. - * The result of the function is the number of bytes written into dstBuffer - * (it can be zero, this means there was no data left within cctx) - * The function outputs an error code if it fails (can be tested using LizardF_isError()) - */ - -size_t LizardF_compressEnd(LizardF_compressionContext_t cctx, void* dstBuffer, size_t dstMaxSize, const LizardF_compressOptions_t* cOptPtr); -/* LizardF_compressEnd() - * When you want to properly finish the compressed frame, just call LizardF_compressEnd(). - * It will flush whatever data remained within compressionContext (like Lizard_flush()) - * but also properly finalize the frame, with an endMark and a checksum. - * The result of the function is the number of bytes written into dstBuffer (necessarily >= 4 (endMark), or 8 if optional frame checksum is enabled) - * The function outputs an error code if it fails (can be tested using LizardF_isError()) - * The LizardF_compressOptions_t structure is optional : you can provide NULL as argument. - * A successful call to LizardF_compressEnd() makes cctx available again for next compression task. - */ - - -/*-********************************* -* Decompression functions -***********************************/ - -typedef struct LizardF_dctx_s* LizardF_decompressionContext_t; /* must be aligned on 8-bytes */ - -typedef struct { - unsigned stableDst; /* guarantee that decompressed data will still be there on next function calls (avoid storage into tmp buffers) */ - unsigned reserved[3]; -} LizardF_decompressOptions_t; - - -/* Resource management */ - -/*!LizardF_createDecompressionContext() : - * Create an LizardF_decompressionContext_t object, which will be used to track all decompression operations. - * The version provided MUST be LIZARDF_VERSION. It is intended to track potential breaking differences between different versions. - * The function will provide a pointer to a fully allocated and initialized LizardF_decompressionContext_t object. - * The result is an errorCode, which can be tested using LizardF_isError(). - * dctx memory can be released using LizardF_freeDecompressionContext(); - * The result of LizardF_freeDecompressionContext() is indicative of the current state of decompressionContext when being released. - * That is, it should be == 0 if decompression has been completed fully and correctly. - */ -LizardF_errorCode_t LizardF_createDecompressionContext(LizardF_decompressionContext_t* dctxPtr, unsigned version); -LizardF_errorCode_t LizardF_freeDecompressionContext(LizardF_decompressionContext_t dctx); - - -/*====== Decompression ======*/ - -/*!LizardF_getFrameInfo() : - * This function decodes frame header information (such as max blockSize, frame checksum, etc.). - * Its usage is optional. The objective is to extract frame header information, typically for allocation purposes. - * A header size is variable and can be from 7 to 15 bytes. It's also possible to input more bytes than that. - * The number of bytes read from srcBuffer will be updated within *srcSizePtr (necessarily <= original value). - * (note that LizardF_getFrameInfo() can also be used anytime *after* starting decompression, in this case 0 input byte is enough) - * Frame header info is *copied into* an already allocated LizardF_frameInfo_t structure. - * The function result is an hint about how many srcSize bytes LizardF_decompress() expects for next call, - * or an error code which can be tested using LizardF_isError() - * (typically, when there is not enough src bytes to fully decode the frame header) - * Decompression is expected to resume from where it stopped (srcBuffer + *srcSizePtr) - */ -size_t LizardF_getFrameInfo(LizardF_decompressionContext_t dctx, - LizardF_frameInfo_t* frameInfoPtr, - const void* srcBuffer, size_t* srcSizePtr); - -/*!LizardF_decompress() : - * Call this function repetitively to regenerate data compressed within srcBuffer. - * The function will attempt to decode *srcSizePtr bytes from srcBuffer, into dstBuffer of maximum size *dstSizePtr. - * - * The number of bytes regenerated into dstBuffer will be provided within *dstSizePtr (necessarily <= original value). - * - * The number of bytes read from srcBuffer will be provided within *srcSizePtr (necessarily <= original value). - * If number of bytes read is < number of bytes provided, then decompression operation is not completed. - * It typically happens when dstBuffer is not large enough to contain all decoded data. - * LizardF_decompress() must be called again, starting from where it stopped (srcBuffer + *srcSizePtr) - * The function will check this condition, and refuse to continue if it is not respected. - * - * `dstBuffer` is expected to be flushed between each call to the function, its content will be overwritten. - * `dst` arguments can be changed at will at each consecutive call to the function. - * - * The function result is an hint of how many `srcSize` bytes LizardF_decompress() expects for next call. - * Schematically, it's the size of the current (or remaining) compressed block + header of next block. - * Respecting the hint provides some boost to performance, since it does skip intermediate buffers. - * This is just a hint though, it's always possible to provide any srcSize. - * When a frame is fully decoded, the function result will be 0 (no more data expected). - * If decompression failed, function result is an error code, which can be tested using LizardF_isError(). - * - * After a frame is fully decoded, dctx can be used again to decompress another frame. - */ -size_t LizardF_decompress(LizardF_decompressionContext_t dctx, - void* dstBuffer, size_t* dstSizePtr, - const void* srcBuffer, size_t* srcSizePtr, - const LizardF_decompressOptions_t* dOptPtr); - - - -#if defined (__cplusplus) -} -#endif diff --git a/contrib/lizard/lib/lizard_frame_static.h b/contrib/lizard/lib/lizard_frame_static.h deleted file mode 100644 index da80756057b..00000000000 --- a/contrib/lizard/lib/lizard_frame_static.h +++ /dev/null @@ -1,81 +0,0 @@ -/* - Lizard auto-framing library - Header File for static linking only - Copyright (C) 2011-2015, Yann Collet. - Copyright (C) 2016-2017, Przemyslaw Skibinski - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -#pragma once - -#if defined (__cplusplus) -extern "C" { -#endif - -/* lizard_frame_static.h should be used solely in the context of static linking. - * It contains definitions which may still change overtime. - * Never use it in the context of DLL linking. - * */ - - -/************************************** -* Includes -**************************************/ -#include "lizard_frame.h" - - -/************************************** - * Error management - * ************************************/ -#define LIZARDF_LIST_ERRORS(ITEM) \ - ITEM(OK_NoError) ITEM(ERROR_GENERIC) \ - ITEM(ERROR_maxBlockSize_invalid) ITEM(ERROR_blockMode_invalid) ITEM(ERROR_contentChecksumFlag_invalid) \ - ITEM(ERROR_compressionLevel_invalid) \ - ITEM(ERROR_headerVersion_wrong) ITEM(ERROR_blockChecksum_unsupported) ITEM(ERROR_reservedFlag_set) \ - ITEM(ERROR_allocation_failed) \ - ITEM(ERROR_srcSize_tooLarge) ITEM(ERROR_dstMaxSize_tooSmall) \ - ITEM(ERROR_frameHeader_incomplete) ITEM(ERROR_frameType_unknown) ITEM(ERROR_frameSize_wrong) \ - ITEM(ERROR_srcPtr_wrong) \ - ITEM(ERROR_decompressionFailed) \ - ITEM(ERROR_headerChecksum_invalid) ITEM(ERROR_contentChecksum_invalid) \ - ITEM(ERROR_maxCode) - -//#define LIZARDF_DISABLE_OLD_ENUMS -#ifndef LIZARDF_DISABLE_OLD_ENUMS -#define LIZARDF_GENERATE_ENUM(ENUM) LizardF_##ENUM, ENUM = LizardF_##ENUM, -#else -#define LIZARDF_GENERATE_ENUM(ENUM) LizardF_##ENUM, -#endif -typedef enum { LIZARDF_LIST_ERRORS(LIZARDF_GENERATE_ENUM) } LizardF_errorCodes; /* enum is exposed, to handle specific errors; compare function result to -enum value */ - - -#if defined (__cplusplus) -} -#endif diff --git a/contrib/lizard/lib/lizard_parser_fast.h b/contrib/lizard/lib/lizard_parser_fast.h deleted file mode 100644 index f9e54b7f475..00000000000 --- a/contrib/lizard/lib/lizard_parser_fast.h +++ /dev/null @@ -1,196 +0,0 @@ -#define LIZARD_FAST_MIN_OFFSET 8 -#define LIZARD_FAST_LONGOFF_MM 0 /* not used with offsets > 1<<16 */ - -/************************************** -* Hash Functions -**************************************/ -static size_t Lizard_hashPosition(const void* p) -{ - if (MEM_64bits()) - return Lizard_hash5Ptr(p, LIZARD_HASHLOG_LZ4); - return Lizard_hash4Ptr(p, LIZARD_HASHLOG_LZ4); -} - -static void Lizard_putPositionOnHash(const BYTE* p, size_t h, U32* hashTable, const BYTE* srcBase) -{ - hashTable[h] = (U32)(p-srcBase); -} - -static void Lizard_putPosition(const BYTE* p, U32* hashTable, const BYTE* srcBase) -{ - size_t const h = Lizard_hashPosition(p); - Lizard_putPositionOnHash(p, h, hashTable, srcBase); -} - -static U32 Lizard_getPositionOnHash(size_t h, U32* hashTable) -{ - return hashTable[h]; -} - -static U32 Lizard_getPosition(const BYTE* p, U32* hashTable) -{ - size_t const h = Lizard_hashPosition(p); - return Lizard_getPositionOnHash(h, hashTable); -} - - -static const U32 Lizard_skipTrigger = 6; /* Increase this value ==> compression run slower on incompressible data */ -static const U32 Lizard_minLength = (MFLIMIT+1); - - -FORCE_INLINE int Lizard_compress_fast( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const U32 acceleration = 1; - const BYTE* base = ctx->base; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictBase = ctx->dictBase; - const BYTE* const dictEnd = dictBase + dictLimit; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = iend - LASTLITERALS; - const BYTE* anchor = ip; - - size_t forwardH, matchIndex; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 lowLimit = (ctx->lowLimit + maxDistance >= (U32)(ip - base)) ? ctx->lowLimit : (U32)(ip - base) - maxDistance; - - /* Init conditions */ - if ((U32)(iend-ip) > (U32)LIZARD_MAX_INPUT_SIZE) goto _output_error; /* Unsupported inputSize, too large (or negative) */ - - if ((U32)(iend-ip) < Lizard_minLength) goto _last_literals; /* Input too small, no compression (all literals) */ - - /* First Byte */ - Lizard_putPosition(ip, ctx->hashTable, base); - ip++; forwardH = Lizard_hashPosition(ip); - - /* Main Loop */ - for ( ; ; ) { - const BYTE* match; - size_t matchLength; - - /* Find a match */ - { const BYTE* forwardIp = ip; - unsigned step = 1; - unsigned searchMatchNb = acceleration << Lizard_skipTrigger; - while (1) { - size_t const h = forwardH; - ip = forwardIp; - forwardIp += step; - step = (searchMatchNb++ >> Lizard_skipTrigger); - - if (unlikely(forwardIp > mflimit)) goto _last_literals; - - matchIndex = Lizard_getPositionOnHash(h, ctx->hashTable); - forwardH = Lizard_hashPosition(forwardIp); - Lizard_putPositionOnHash(ip, h, ctx->hashTable, base); - - if ((matchIndex < lowLimit) || (matchIndex >= (U32)(ip - base)) || (base + matchIndex + maxDistance < ip)) continue; - - if (matchIndex >= dictLimit) { - match = base + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) -#endif - if (MEM_read32(match) == MEM_read32(ip)) - { - int back = 0; - matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); - - while ((ip+back > anchor) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - matchLength -= back; -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - { - ip += back; - match += back; - break; - } - } - } else { - match = dictBase + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - const U32 newLowLimit = (lowLimit + maxDistance >= (U32)(ip-base)) ? lowLimit : (U32)(ip - base) - maxDistance; - int back = 0; - matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); - - while ((ip+back > anchor) && (matchIndex+back > newLowLimit) && (ip[back-1] == match[back-1])) back--; - matchLength -= back; - match = base + matchIndex + back; -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - { - ip += back; - break; - } - } - } - } // while (1) - } - -_next_match: - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, matchLength+MINMATCH, match)) goto _output_error; - - /* Test end of chunk */ - if (ip > mflimit) break; - - /* Fill table */ - Lizard_putPosition(ip-2, ctx->hashTable, base); - - /* Test next position */ - matchIndex = Lizard_getPosition(ip, ctx->hashTable); - Lizard_putPosition(ip, ctx->hashTable, base); - if ((matchIndex >= lowLimit) && (matchIndex < (U32)(ip - base)) && (base + matchIndex + maxDistance >= ip)) - { - if (matchIndex >= dictLimit) { - match = base + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) -#endif - if (MEM_read32(match) == MEM_read32(ip)) - { - matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - goto _next_match; - } - } else { - match = dictBase + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); - match = base + matchIndex; -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - goto _next_match; - } - } - } - - /* Prepare next loop */ - forwardH = Lizard_hashPosition(++ip); - } - -_last_literals: - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} diff --git a/contrib/lizard/lib/lizard_parser_fastbig.h b/contrib/lizard/lib/lizard_parser_fastbig.h deleted file mode 100644 index 30c464fcb90..00000000000 --- a/contrib/lizard/lib/lizard_parser_fastbig.h +++ /dev/null @@ -1,175 +0,0 @@ -#define LIZARD_FASTBIG_LONGOFF_MM MM_LONGOFF - -/************************************** -* Hash Functions -**************************************/ -static size_t Lizard_hashPositionHLog(const void* p, int hashLog) -{ - if (MEM_64bits()) - return Lizard_hash5Ptr(p, hashLog); - return Lizard_hash4Ptr(p, hashLog); -} - -static void Lizard_putPositionOnHashHLog(const BYTE* p, size_t h, U32* hashTable, const BYTE* srcBase) -{ - hashTable[h] = (U32)(p-srcBase); -} - -static void Lizard_putPositionHLog(const BYTE* p, U32* hashTable, const BYTE* srcBase, int hashLog) -{ - size_t const h = Lizard_hashPositionHLog(p, hashLog); - Lizard_putPositionOnHashHLog(p, h, hashTable, srcBase); -} - -static U32 Lizard_getPositionOnHashHLog(size_t h, U32* hashTable) -{ - return hashTable[h]; -} - -static U32 Lizard_getPositionHLog(const BYTE* p, U32* hashTable, int hashLog) -{ - size_t const h = Lizard_hashPositionHLog(p, hashLog); - return Lizard_getPositionOnHashHLog(h, hashTable); -} - -FORCE_INLINE int Lizard_compress_fastBig( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const U32 acceleration = 1; - const BYTE* base = ctx->base; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictBase = ctx->dictBase; - const BYTE* const dictEnd = dictBase + dictLimit; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = iend - LASTLITERALS; - const BYTE* anchor = ip; - - size_t forwardH, matchIndex; - const int hashLog = ctx->params.hashLog; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 lowLimit = (ctx->lowLimit + maxDistance >= (U32)(ip - base)) ? ctx->lowLimit : (U32)(ip - base) - maxDistance; - - /* Init conditions */ - if ((U32)(iend-ip) > (U32)LIZARD_MAX_INPUT_SIZE) goto _output_error; /* Unsupported inputSize, too large (or negative) */ - - if ((U32)(iend-ip) < Lizard_minLength) goto _last_literals; /* Input too small, no compression (all literals) */ - - /* First Byte */ - Lizard_putPositionHLog(ip, ctx->hashTable, base, hashLog); - ip++; forwardH = Lizard_hashPositionHLog(ip, hashLog); - - /* Main Loop */ - for ( ; ; ) { - const BYTE* match; - size_t matchLength; - - /* Find a match */ - { const BYTE* forwardIp = ip; - unsigned step = 1; - unsigned searchMatchNb = acceleration << Lizard_skipTrigger; - while (1) { - size_t const h = forwardH; - ip = forwardIp; - forwardIp += step; - step = (searchMatchNb++ >> Lizard_skipTrigger); - - if (unlikely(forwardIp > mflimit)) goto _last_literals; - - matchIndex = Lizard_getPositionOnHashHLog(h, ctx->hashTable); - forwardH = Lizard_hashPositionHLog(forwardIp, hashLog); - Lizard_putPositionOnHashHLog(ip, h, ctx->hashTable, base); - - if ((matchIndex < lowLimit) || (matchIndex >= (U32)(ip - base)) || (base + matchIndex + maxDistance < ip)) continue; - - if (matchIndex >= dictLimit) { - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) - if (MEM_read32(match) == MEM_read32(ip)) - { - int back = 0; - matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); - - while ((ip+back > anchor) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - matchLength -= back; - if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - { - ip += back; - match += back; - break; - } - } - } else { - match = dictBase + matchIndex; - if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - const U32 newLowLimit = (lowLimit + maxDistance >= (U32)(ip-base)) ? lowLimit : (U32)(ip - base) - maxDistance; - int back = 0; - matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); - - while ((ip+back > anchor) && (matchIndex+back > newLowLimit) && (ip[back-1] == match[back-1])) back--; - matchLength -= back; - match = base + matchIndex + back; - if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - { - ip += back; - break; - } - } - } - } // while (1) - } - -_next_match: - if (Lizard_encodeSequence_LIZv1(ctx, &ip, &anchor, matchLength+MINMATCH, match)) goto _output_error; - - /* Test end of chunk */ - if (ip > mflimit) break; - - /* Fill table */ - Lizard_putPositionHLog(ip-2, ctx->hashTable, base, hashLog); - - /* Test next position */ - matchIndex = Lizard_getPositionHLog(ip, ctx->hashTable, hashLog); - Lizard_putPositionHLog(ip, ctx->hashTable, base, hashLog); - if ((matchIndex >= lowLimit) && (matchIndex < (U32)(ip - base)) && (base + matchIndex + maxDistance >= ip)) - { - if (matchIndex >= dictLimit) { - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) - if (MEM_read32(match) == MEM_read32(ip)) - { - matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); - if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - goto _next_match; - } - } else { - match = dictBase + matchIndex; - if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); - match = base + matchIndex; - if ((matchLength >= LIZARD_FASTBIG_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - goto _next_match; - } - } - } - - /* Prepare next loop */ - forwardH = Lizard_hashPositionHLog(++ip, hashLog); - } - -_last_literals: - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LIZv1(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} diff --git a/contrib/lizard/lib/lizard_parser_fastsmall.h b/contrib/lizard/lib/lizard_parser_fastsmall.h deleted file mode 100644 index 550edc5b201..00000000000 --- a/contrib/lizard/lib/lizard_parser_fastsmall.h +++ /dev/null @@ -1,189 +0,0 @@ -/************************************** -* Hash Functions -**************************************/ -static size_t Lizard_hashPositionSmall(const void* p) -{ - if (MEM_64bits()) - return Lizard_hash5Ptr(p, LIZARD_HASHLOG_LZ4SM); - return Lizard_hash4Ptr(p, LIZARD_HASHLOG_LZ4SM); -} - -static void Lizard_putPositionOnHashSmall(const BYTE* p, size_t h, U32* hashTable, const BYTE* srcBase) -{ - hashTable[h] = (U32)(p-srcBase); -} - -static void Lizard_putPositionSmall(const BYTE* p, U32* hashTable, const BYTE* srcBase) -{ - size_t const h = Lizard_hashPositionSmall(p); - Lizard_putPositionOnHashSmall(p, h, hashTable, srcBase); -} - -static U32 Lizard_getPositionOnHashSmall(size_t h, U32* hashTable) -{ - return hashTable[h]; -} - -static U32 Lizard_getPositionSmall(const BYTE* p, U32* hashTable) -{ - size_t const h = Lizard_hashPositionSmall(p); - return Lizard_getPositionOnHashSmall(h, hashTable); -} - - -FORCE_INLINE int Lizard_compress_fastSmall( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const U32 acceleration = 1; - const BYTE* base = ctx->base; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictBase = ctx->dictBase; - const BYTE* const dictEnd = dictBase + dictLimit; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = iend - LASTLITERALS; - const BYTE* anchor = ip; - - size_t forwardH, matchIndex; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 lowLimit = (ctx->lowLimit + maxDistance >= (U32)(ip - base)) ? ctx->lowLimit : (U32)(ip - base) - maxDistance; - - /* Init conditions */ - if ((U32)(iend-ip) > (U32)LIZARD_MAX_INPUT_SIZE) goto _output_error; /* Unsupported inputSize, too large (or negative) */ - - if ((U32)(iend-ip) < Lizard_minLength) goto _last_literals; /* Input too small, no compression (all literals) */ - - /* First Byte */ - Lizard_putPositionSmall(ip, ctx->hashTable, base); - ip++; forwardH = Lizard_hashPositionSmall(ip); - - /* Main Loop */ - for ( ; ; ) { - const BYTE* match; - size_t matchLength; - - /* Find a match */ - { const BYTE* forwardIp = ip; - unsigned step = 1; - unsigned searchMatchNb = acceleration << Lizard_skipTrigger; - while (1) { - size_t const h = forwardH; - ip = forwardIp; - forwardIp += step; - step = (searchMatchNb++ >> Lizard_skipTrigger); - - if (unlikely(forwardIp > mflimit)) goto _last_literals; - - matchIndex = Lizard_getPositionOnHashSmall(h, ctx->hashTable); - forwardH = Lizard_hashPositionSmall(forwardIp); - Lizard_putPositionOnHashSmall(ip, h, ctx->hashTable, base); - - if ((matchIndex < lowLimit) || (matchIndex >= (U32)(ip - base)) || (base + matchIndex + maxDistance < ip)) continue; - - if (matchIndex >= dictLimit) { - match = base + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) -#endif - if (MEM_read32(match) == MEM_read32(ip)) - { - int back = 0; - matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); - - while ((ip+back > anchor) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - matchLength -= back; -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - { - ip += back; - match += back; - break; - } - } - } else { - match = dictBase + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - const U32 newLowLimit = (lowLimit + maxDistance >= (U32)(ip-base)) ? lowLimit : (U32)(ip - base) - maxDistance; - int back = 0; - matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); - - while ((ip+back > anchor) && (matchIndex+back > newLowLimit) && (ip[back-1] == match[back-1])) back--; - matchLength -= back; - match = base + matchIndex + back; -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - { - ip += back; - break; - } - } - } - } // while (1) - } - -_next_match: - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, matchLength+MINMATCH, match)) goto _output_error; - - /* Test end of chunk */ - if (ip > mflimit) break; - - /* Fill table */ - Lizard_putPositionSmall(ip-2, ctx->hashTable, base); - - /* Test next position */ - matchIndex = Lizard_getPositionSmall(ip, ctx->hashTable); - Lizard_putPositionSmall(ip, ctx->hashTable, base); - if ((matchIndex >= lowLimit) && (matchIndex < (U32)(ip - base)) && (base + matchIndex + maxDistance >= ip)) - { - if (matchIndex >= dictLimit) { - match = base + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_FAST_MIN_OFFSET) -#endif - if (MEM_read32(match) == MEM_read32(ip)) - { - matchLength = Lizard_count(ip+MINMATCH, match+MINMATCH, matchlimit); -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - goto _next_match; - } - } else { - match = dictBase + matchIndex; -#if LIZARD_FAST_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_FAST_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - matchLength = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, matchlimit, dictEnd, lowPrefixPtr); - match = base + matchIndex; -#if LIZARD_FAST_LONGOFF_MM > 0 - if ((matchLength >= LIZARD_FAST_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - goto _next_match; - } - } - } - - /* Prepare next loop */ - forwardH = Lizard_hashPositionSmall(++ip); - } - -_last_literals: - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} diff --git a/contrib/lizard/lib/lizard_parser_hashchain.h b/contrib/lizard/lib/lizard_parser_hashchain.h deleted file mode 100644 index 64b9d76c451..00000000000 --- a/contrib/lizard/lib/lizard_parser_hashchain.h +++ /dev/null @@ -1,369 +0,0 @@ -#define LIZARD_HC_MIN_OFFSET 8 -#define LIZARD_HC_LONGOFF_MM 0 /* not used with offsets > 1<<16 */ -#define OPTIMAL_ML (int)((ML_MASK_LZ4-1)+MINMATCH) -#define GET_MINMATCH(offset) (MINMATCH) - -#if 1 - #define LIZARD_HC_HASH_FUNCTION(ip, hashLog) Lizard_hashPtr(ip, hashLog, ctx->params.searchLength) -#else - #define LIZARD_HC_HASH_FUNCTION(ip, hashLog) Lizard_hash5Ptr(ip, hashLog) -#endif - -/* Update chains up to ip (excluded) */ -FORCE_INLINE void Lizard_Insert (Lizard_stream_t* ctx, const BYTE* ip) -{ - U32* const chainTable = ctx->chainTable; - U32* const hashTable = ctx->hashTable; -#if MINMATCH == 3 - U32* HashTable3 = ctx->hashTable3; -#endif - const BYTE* const base = ctx->base; - U32 const target = (U32)(ip - base); - U32 idx = ctx->nextToUpdate; - const int hashLog = ctx->params.hashLog; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - - while (idx < target) { - size_t const h = Lizard_hashPtr(base+idx, hashLog, ctx->params.searchLength); - size_t delta = idx - hashTable[h]; - if (delta>maxDistance) delta = maxDistance; - DELTANEXT(idx) = (U32)delta; - if ((hashTable[h] >= idx) || (idx >= hashTable[h] + LIZARD_HC_MIN_OFFSET)) - hashTable[h] = idx; -#if MINMATCH == 3 - HashTable3[Lizard_hash3Ptr(base+idx, ctx->params.hashLog3)] = idx; -#endif - idx++; - } - - ctx->nextToUpdate = target; -} - - - -FORCE_INLINE int Lizard_InsertAndFindBestMatch (Lizard_stream_t* ctx, /* Index table will be updated */ - const BYTE* ip, const BYTE* const iLimit, - const BYTE** matchpos) -{ - U32* const chainTable = ctx->chainTable; - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - U32 matchIndex, delta; - const BYTE* match; - int nbAttempts=ctx->params.searchNum; - size_t ml=0; - const int hashLog = ctx->params.hashLog; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 current = (U32)(ip - base); - const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; - - /* HC4 match finder */ - Lizard_Insert(ctx, ip); - matchIndex = HashTable[LIZARD_HC_HASH_FUNCTION(ip, hashLog)]; - - while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { - nbAttempts--; - if (matchIndex >= dictLimit) { - match = base + matchIndex; -#if LIZARD_HC_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_HC_MIN_OFFSET) -#endif - if (*(match+ml) == *(ip+ml) - && (MEM_read32(match) == MEM_read32(ip))) - { - size_t const mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; -#if LIZARD_HC_LONGOFF_MM > 0 - if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - if (mlt > ml) { ml = mlt; *matchpos = match; } - } - } else { - match = dictBase + matchIndex; -#if LIZARD_HC_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_HC_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; -#if LIZARD_HC_LONGOFF_MM > 0 - if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - (base + matchIndex)) < LIZARD_MAX_16BIT_OFFSET)) -#endif - if (mlt > ml) { ml = mlt; *matchpos = base + matchIndex; } /* virtual matchpos */ - } - } - delta = DELTANEXT(matchIndex); - if (delta > matchIndex) break; - matchIndex -= delta; - } - - return (int)ml; -} - - -FORCE_INLINE int Lizard_InsertAndGetWiderMatch ( - Lizard_stream_t* ctx, - const BYTE* const ip, - const BYTE* const iLowLimit, - const BYTE* const iHighLimit, - int longest, - const BYTE** matchpos, - const BYTE** startpos) -{ - U32* const chainTable = ctx->chainTable; - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictBase = ctx->dictBase; - const BYTE* const dictEnd = dictBase + dictLimit; - U32 matchIndex, delta; - int nbAttempts = ctx->params.searchNum; - int LLdelta = (int)(ip-iLowLimit); - const int hashLog = ctx->params.hashLog; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 current = (U32)(ip - base); - const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; - - /* First Match */ - Lizard_Insert(ctx, ip); - matchIndex = HashTable[LIZARD_HC_HASH_FUNCTION(ip, hashLog)]; - - while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { - nbAttempts--; - if (matchIndex >= dictLimit) { - const BYTE* match = base + matchIndex; -#if LIZARD_HC_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_HC_MIN_OFFSET) -#endif - if (*(iLowLimit + longest) == *(match - LLdelta + longest)) { - if (MEM_read32(match) == MEM_read32(ip)) { - int mlt = MINMATCH + Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit); - int back = 0; - while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - -#if LIZARD_HC_LONGOFF_MM > 0 - if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) -#endif - if (mlt > longest) { - longest = (int)mlt; - *matchpos = match+back; - *startpos = ip+back; - } - } - } - } else { - const BYTE* match = dictBase + matchIndex; -#if LIZARD_HC_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_HC_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - int back=0; - size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; - while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == match[back-1])) back--; - mlt -= back; -#if LIZARD_HC_LONGOFF_MM > 0 - if ((mlt >= LIZARD_HC_LONGOFF_MM) || ((U32)(ip - (base + matchIndex)) < LIZARD_MAX_16BIT_OFFSET)) -#endif - if ((int)mlt > longest) { longest = (int)mlt; *matchpos = base + matchIndex + back; *startpos = ip+back; } - } - } - delta = DELTANEXT(matchIndex); - if (delta > matchIndex) break; - matchIndex -= delta; - } - - return longest; -} - - -FORCE_INLINE int Lizard_compress_hashChain ( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const BYTE* anchor = ip; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = (iend - LASTLITERALS); - - int ml, ml2, ml3, ml0; - const BYTE* ref = NULL; - const BYTE* start2 = NULL; - const BYTE* ref2 = NULL; - const BYTE* start3 = NULL; - const BYTE* ref3 = NULL; - const BYTE* start0; - const BYTE* ref0; - - /* init */ - ip++; - - /* Main Loop */ - while (ip < mflimit) { - ml = Lizard_InsertAndFindBestMatch (ctx, ip, matchlimit, (&ref)); - if (!ml) { ip++; continue; } - - /* saved, in case we would skip too much */ - start0 = ip; - ref0 = ref; - ml0 = ml; - -_Search2: - if (ip+ml < mflimit) - ml2 = Lizard_InsertAndGetWiderMatch(ctx, ip + ml - 2, ip + 1, matchlimit, ml, &ref2, &start2); - else ml2 = ml; - - if (ml2 == ml) { /* No better match */ - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - continue; - } - - if (start0 < ip) { - if (start2 < ip + ml0) { /* empirical */ - ip = start0; - ref = ref0; - ml = ml0; - } - } - - /* Here, start0==ip */ - if ((start2 - ip) < 3) { /* First Match too small : removed */ - ml = ml2; - ip = start2; - ref =ref2; - goto _Search2; - } - -_Search3: - /* - * Currently we have : - * ml2 > ml1, and - * ip1+3 <= ip2 (usually < ip1+ml1) - */ - if ((start2 - ip) < OPTIMAL_ML) { - int correction; - int new_ml = ml; - if (new_ml > OPTIMAL_ML) new_ml = OPTIMAL_ML; - if (ip+new_ml > start2 + ml2 - GET_MINMATCH((U32)(start2 - ref2))) { - new_ml = (int)(start2 - ip) + ml2 - GET_MINMATCH((U32)(start2 - ref2)); - if (new_ml < GET_MINMATCH((U32)(ip - ref))) { // match2 doesn't fit - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - continue; - } - } - correction = new_ml - (int)(start2 - ip); - if (correction > 0) { - start2 += correction; - ref2 += correction; - ml2 -= correction; - } - } - /* Now, we have start2 = ip+new_ml, with new_ml = min(ml, OPTIMAL_ML=18) */ - - if (start2 + ml2 < mflimit) - ml3 = Lizard_InsertAndGetWiderMatch(ctx, start2 + ml2 - 3, start2, matchlimit, ml2, &ref3, &start3); - else ml3 = ml2; - - if (ml3 == ml2) { /* No better match : 2 sequences to encode */ - /* ip & ref are known; Now for ml */ - if (start2 < ip+ml) ml = (int)(start2 - ip); - /* Now, encode 2 sequences */ - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - ip = start2; - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml2, ref2)) return 0; - continue; - } - - if (start3 < ip+ml+3) { /* Not enough space for match 2 : remove it */ - if (start3 >= (ip+ml)) { /* can write Seq1 immediately ==> Seq2 is removed, so Seq3 becomes Seq1 */ - if (start2 < ip+ml) { - int correction = (int)(ip+ml - start2); - start2 += correction; - ref2 += correction; - ml2 -= correction; - if (ml2 < GET_MINMATCH((U32)(start2 - ref2))) { - start2 = start3; - ref2 = ref3; - ml2 = ml3; - } - } - - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - ip = start3; - ref = ref3; - ml = ml3; - - start0 = start2; - ref0 = ref2; - ml0 = ml2; - goto _Search2; - } - - start2 = start3; - ref2 = ref3; - ml2 = ml3; - goto _Search3; - } - - /* - * OK, now we have 3 ascending matches; let's write at least the first one - * ip & ref are known; Now for ml - */ - if (start2 < ip+ml) { - if ((start2 - ip) < (int)ML_MASK_LZ4) { - int correction; - if (ml > OPTIMAL_ML) ml = OPTIMAL_ML; - if (ip + ml > start2 + ml2 - GET_MINMATCH((U32)(start2 - ref2))) { - ml = (int)(start2 - ip) + ml2 - GET_MINMATCH((U32)(start2 - ref2)); - if (ml < GET_MINMATCH((U32)(ip - ref))) { // match2 doesn't fit, remove it - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - ip = start3; - ref = ref3; - ml = ml3; - - start0 = start2; - ref0 = ref2; - ml0 = ml2; - goto _Search2; - } - } - correction = ml - (int)(start2 - ip); - if (correction > 0) { - start2 += correction; - ref2 += correction; - ml2 -= correction; - } - } else { - ml = (int)(start2 - ip); - } - } - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - - ip = start2; - ref = ref2; - ml = ml2; - - start2 = start3; - ref2 = ref3; - ml2 = ml3; - - goto _Search3; - } - - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} diff --git a/contrib/lizard/lib/lizard_parser_lowestprice.h b/contrib/lizard/lib/lizard_parser_lowestprice.h deleted file mode 100644 index f8b95dad4ea..00000000000 --- a/contrib/lizard/lib/lizard_parser_lowestprice.h +++ /dev/null @@ -1,376 +0,0 @@ -#define LIZARD_LOWESTPRICE_MIN_OFFSET 8 - - -FORCE_INLINE size_t Lizard_more_profitable(Lizard_stream_t* const ctx, const BYTE *best_ip, size_t best_off, size_t best_common, const BYTE *ip, size_t off, size_t common, size_t literals, int last_off) -{ - size_t sum; - - if (literals > 0) - sum = MAX(common + literals, best_common); - else - sum = MAX(common, best_common - literals); - - if ((int)off == last_off) off = 0; // rep code - if ((int)best_off == last_off) best_off = 0; - - return Lizard_get_price_LIZv1(ctx, last_off, ip, ctx->off24pos, sum - common, (U32)off, common) <= Lizard_get_price_LIZv1(ctx, last_off, best_ip, ctx->off24pos, sum - best_common, (U32)best_off, best_common); -} - - -FORCE_INLINE size_t Lizard_better_price(Lizard_stream_t* const ctx, const BYTE *best_ip, size_t best_off, size_t best_common, const BYTE *ip, size_t off, size_t common, int last_off) -{ - if ((int)off == last_off) off = 0; // rep code - if ((int)best_off == last_off) best_off = 0; - - return Lizard_get_price_LIZv1(ctx, last_off, ip, ctx->off24pos, 0, (U32)off, common) < Lizard_get_price_LIZv1(ctx, last_off, best_ip, ctx->off24pos, common - best_common, (U32)best_off, best_common); -} - - -FORCE_INLINE int Lizard_FindMatchLowestPrice (Lizard_stream_t* ctx, /* Index table will be updated */ - const BYTE* ip, const BYTE* const iLimit, - const BYTE** matchpos) -{ - U32* const chainTable = ctx->chainTable; - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const intptr_t dictLimit = ctx->dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; - const intptr_t current = (ip - base); - const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; - const BYTE* const lowPrefixPtr = base + dictLimit; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - intptr_t matchIndex; - const BYTE* match, *matchDict; - int nbAttempts=ctx->params.searchNum; - size_t ml=0, mlt; - - matchIndex = HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; - - if (ctx->last_off >= LIZARD_LOWESTPRICE_MIN_OFFSET) { - intptr_t matchIndexLO = (ip - ctx->last_off) - base; - if (matchIndexLO >= lowLimit) { - if (matchIndexLO >= dictLimit) { - match = base + matchIndexLO; - mlt = Lizard_count(ip, match, iLimit);// + MINMATCH; - // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) - if (mlt > REPMINMATCH) { - *matchpos = match; - return (int)mlt; - } - } else { - match = dictBase + matchIndexLO; - if ((U32)((dictLimit-1) - matchIndexLO) >= 3) { /* intentional overflow */ - mlt = Lizard_count_2segments(ip, match, iLimit, dictEnd, lowPrefixPtr); - // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) - if (mlt > REPMINMATCH) { - *matchpos = base + matchIndexLO; /* virtual matchpos */ - return (int)mlt; - } - } - } - } - } - - -#if MINMATCH == 3 - { - U32 matchIndex3 = ctx->hashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; - if (matchIndex3 < current && matchIndex3 >= lowLimit) - { - size_t offset = (size_t)current - matchIndex3; - if (offset < LIZARD_MAX_8BIT_OFFSET) - { - match = ip - offset; - if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) - { - ml = 3;//Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - *matchpos = match; - } - } - } - } -#endif - while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { - nbAttempts--; - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_LOWESTPRICE_MIN_OFFSET) { - if (matchIndex >= dictLimit) { - if (*(match+ml) == *(ip+ml) && (MEM_read32(match) == MEM_read32(ip))) { - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (!ml || (mlt > ml && Lizard_better_price(ctx, ip, (ip - *matchpos), ml, ip, (ip - match), mlt, ctx->last_off))) - { ml = mlt; *matchpos = match; } - } - } else { - matchDict = dictBase + matchIndex; - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(matchDict) == MEM_read32(ip)) { - mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (!ml || (mlt > ml && Lizard_better_price(ctx, ip, (ip - *matchpos), ml, ip, (U32)(ip - match), mlt, ctx->last_off))) - { ml = mlt; *matchpos = match; } /* virtual matchpos */ - } - } - } - matchIndex -= chainTable[matchIndex & contentMask]; - } - - return (int)ml; -} - - -FORCE_INLINE size_t Lizard_GetWiderMatch ( - Lizard_stream_t* ctx, - const BYTE* const ip, - const BYTE* const iLowLimit, - const BYTE* const iHighLimit, - size_t longest, - const BYTE** matchpos, - const BYTE** startpos) -{ - U32* const chainTable = ctx->chainTable; - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const intptr_t dictLimit = ctx->dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; - const intptr_t current = (ip - base); - const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; - const BYTE* const lowPrefixPtr = base + dictLimit; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const BYTE* match, *matchDict; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - intptr_t matchIndex; - int nbAttempts = ctx->params.searchNum; - size_t mlt; - - /* First Match */ - matchIndex = HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; - - if (ctx->last_off >= LIZARD_LOWESTPRICE_MIN_OFFSET) { - intptr_t matchIndexLO = (ip - ctx->last_off) - base; - if (matchIndexLO >= lowLimit) { - if (matchIndexLO >= dictLimit) { - match = base + matchIndexLO; - if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { - int back = 0; - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit) + MINMATCH; - while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - - if (mlt > longest) - if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) { - *matchpos = match+back; - *startpos = ip+back; - longest = mlt; - } - } - } else { - match = dictBase + matchIndexLO; - if ((U32)((dictLimit-1) - matchIndexLO) >= 3) /* intentional overflow */ - if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { - int back=0; - mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; - while ((ip+back > iLowLimit) && (matchIndexLO+back > lowLimit) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - - if (mlt > longest) - if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) { - *matchpos = base + matchIndexLO + back; /* virtual matchpos */ - *startpos = ip+back; - longest = mlt; - } - } - } - } - } - -#if MINMATCH == 3 - { - U32 matchIndex3 = ctx->hashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; - if (matchIndex3 < current && matchIndex3 >= lowLimit) { - size_t offset = (size_t)current - matchIndex3; - if (offset < LIZARD_MAX_8BIT_OFFSET) { - match = ip - offset; - if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) { - mlt = Lizard_count(ip + MINMATCH, match + MINMATCH, iHighLimit) + MINMATCH; - - int back = 0; - while ((ip + back > iLowLimit) && (match + back > lowPrefixPtr) && (ip[back - 1] == match[back - 1])) back--; - mlt -= back; - - if (!longest || (mlt > longest && Lizard_better_price(ctx, *startpos, (*startpos - *matchpos), longest, ip, (ip - match), mlt, ctx->last_off))) { - *matchpos = match + back; - *startpos = ip + back; - longest = mlt; - } - } - } - } - } -#endif - - while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { - nbAttempts--; - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_LOWESTPRICE_MIN_OFFSET) { - if (matchIndex >= dictLimit) { - if (MEM_read32(match) == MEM_read32(ip)) { - int back = 0; - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit) + MINMATCH; - while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (!longest || (mlt > longest && Lizard_better_price(ctx, *startpos, (*startpos - *matchpos), longest, ip, (ip - match), mlt, ctx->last_off))) - { longest = mlt; *startpos = ip+back; *matchpos = match+back; } - } - } else { - matchDict = dictBase + matchIndex; - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(matchDict) == MEM_read32(ip)) { - int back=0; - mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; - while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == matchDict[back-1])) back--; - mlt -= back; - - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (!longest || (mlt > longest && Lizard_better_price(ctx, *startpos, (*startpos - *matchpos), longest, ip, (U32)(ip - match), mlt, ctx->last_off))) - { longest = mlt; *startpos = ip+back; *matchpos = match+back; } /* virtual matchpos */ - } - } - } - matchIndex -= chainTable[matchIndex & contentMask]; - } - - return longest; -} - - - - -FORCE_INLINE int Lizard_compress_lowestPrice( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const BYTE* anchor = ip; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = (iend - LASTLITERALS); - - size_t ml, ml2, ml0; - const BYTE* ref=NULL; - const BYTE* start2=NULL; - const BYTE* ref2=NULL; - const BYTE* start0; - const BYTE* ref0; - const BYTE* lowPrefixPtr = ctx->base + ctx->dictLimit; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - const size_t sufficient_len = ctx->params.sufficientLength; - - /* Main Loop */ - while (ip < mflimit) - { - Lizard_Insert(ctx, ip); - ml = Lizard_FindMatchLowestPrice (ctx, ip, matchlimit, (&ref)); - if (!ml) { ip++; continue; } - - { - int back = 0; - while ((ip + back > anchor) && (ref + back > lowPrefixPtr) && (ip[back - 1] == ref[back - 1])) back--; - ml -= back; - ip += back; - ref += back; - } - - /* saved, in case we would skip too much */ - start0 = ip; - ref0 = ref; - ml0 = ml; - // goto _Encode; - -_Search: - if (ip+ml >= mflimit) { goto _Encode; } - if (ml >= sufficient_len) { goto _Encode; } - - Lizard_Insert(ctx, ip); - ml2 = (int)Lizard_GetWiderMatch(ctx, ip + ml - 2, anchor, matchlimit, 0, &ref2, &start2); - if (!ml2) goto _Encode; - - { - U64 price, best_price; - int off0=0, off1=0; - const BYTE *pos, *best_pos; - - // find the lowest price for encoding ml bytes - best_pos = ip; - best_price = LIZARD_MAX_PRICE; - off0 = (int)(ip - ref); - off1 = (int)(start2 - ref2); - - for (pos = ip + ml; pos >= start2; pos--) - { - int common0 = (int)(pos - ip); - if (common0 >= MINMATCH) { - price = (int)Lizard_get_price_LIZv1(ctx, ctx->last_off, ip, ctx->off24pos, ip - anchor, (off0 == ctx->last_off) ? 0 : off0, common0); - - { - int common1 = (int)(start2 + ml2 - pos); - if (common1 >= MINMATCH) - price += Lizard_get_price_LIZv1(ctx, ctx->last_off, pos, ctx->off24pos, 0, (off1 == off0) ? 0 : (off1), common1); - else - price += Lizard_get_price_LIZv1(ctx, ctx->last_off, pos, ctx->off24pos, common1, 0, 0); - } - - if (price < best_price) { - best_price = price; - best_pos = pos; - } - } else { - price = Lizard_get_price_LIZv1(ctx, ctx->last_off, ip, ctx->off24pos, start2 - anchor, (off1 == ctx->last_off) ? 0 : off1, ml2); - - if (price < best_price) - best_pos = pos; - break; - } - } - ml = (int)(best_pos - ip); - } - - - if ((ml < MINMATCH) || ((ml < minMatchLongOff) && ((U32)(ip-ref) >= LIZARD_MAX_16BIT_OFFSET))) - { - ip = start2; - ref = ref2; - ml = ml2; - goto _Search; - } - -_Encode: - if (start0 < ip) - { - if (Lizard_more_profitable(ctx, ip, (ip - ref), ml, start0, (start0 - ref0), ml0, (ref0 - ref), ctx->last_off)) - { - ip = start0; - ref = ref0; - ml = ml0; - } - } - - if (Lizard_encodeSequence_LIZv1(ctx, &ip, &anchor, ml, ((ip - ref == ctx->last_off) ? ip : ref))) return 0; - } - - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LIZv1(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} - diff --git a/contrib/lizard/lib/lizard_parser_nochain.h b/contrib/lizard/lib/lizard_parser_nochain.h deleted file mode 100644 index e8abb586ff4..00000000000 --- a/contrib/lizard/lib/lizard_parser_nochain.h +++ /dev/null @@ -1,318 +0,0 @@ -#define OPTIMAL_ML (int)((ML_MASK_LZ4-1)+MINMATCH) - -//#define LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog) Lizard_hashPtr(ip, hashLog, ctx->params.searchLength) -#define LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog) Lizard_hash5Ptr(ip, hashLog) -#define LIZARD_NOCHAIN_MIN_OFFSET 8 - -/* Update chains up to ip (excluded) */ -FORCE_INLINE void Lizard_InsertNoChain (Lizard_stream_t* ctx, const BYTE* ip) -{ - U32* const hashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - U32 const target = (U32)(ip - base); - U32 idx = ctx->nextToUpdate; - const int hashLog = ctx->params.hashLog; - - while (idx < target) { - size_t const h = LIZARD_NOCHAIN_HASH_FUNCTION(base+idx, hashLog); - if ((hashTable[h] >= idx) || (idx >= hashTable[h] + LIZARD_NOCHAIN_MIN_OFFSET)) - hashTable[h] = idx; - idx++; - } - - ctx->nextToUpdate = target; -} - - -FORCE_INLINE int Lizard_InsertAndFindBestMatchNoChain (Lizard_stream_t* ctx, /* Index table will be updated */ - const BYTE* ip, const BYTE* const iLimit, - const BYTE** matchpos) -{ - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - U32 matchIndex; - const BYTE* match; - size_t ml=0; - const int hashLog = ctx->params.hashLog; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 current = (U32)(ip - base); - const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; - - /* HC4 match finder */ - Lizard_InsertNoChain(ctx, ip); - matchIndex = HashTable[LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog)]; - - if ((matchIndex < current) && (matchIndex >= lowLimit)) { - if (matchIndex >= dictLimit) { - match = base + matchIndex; -#if LIZARD_NOCHAIN_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_NOCHAIN_MIN_OFFSET) -#endif - if (*(match+ml) == *(ip+ml) && (MEM_read32(match) == MEM_read32(ip))) - { - size_t const mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - if (mlt > ml) { ml = mlt; *matchpos = match; } - } - } else { - match = dictBase + matchIndex; -#if LIZARD_NOCHAIN_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_NOCHAIN_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; - if (mlt > ml) { ml = mlt; *matchpos = base + matchIndex; } /* virtual matchpos */ - } - } - } - - return (int)ml; -} - - -FORCE_INLINE int Lizard_InsertAndGetWiderMatchNoChain ( - Lizard_stream_t* ctx, - const BYTE* const ip, - const BYTE* const iLowLimit, - const BYTE* const iHighLimit, - int longest, - const BYTE** matchpos, - const BYTE** startpos) -{ - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const BYTE* const dictBase = ctx->dictBase; - const BYTE* const dictEnd = dictBase + dictLimit; - U32 matchIndex; - int LLdelta = (int)(ip-iLowLimit); - const int hashLog = ctx->params.hashLog; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 current = (U32)(ip - base); - const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; - - /* First Match */ - Lizard_InsertNoChain(ctx, ip); - matchIndex = HashTable[LIZARD_NOCHAIN_HASH_FUNCTION(ip, hashLog)]; - - if ((matchIndex < current) && (matchIndex >= lowLimit)) { - if (matchIndex >= dictLimit) { - const BYTE* match = base + matchIndex; -#if LIZARD_NOCHAIN_MIN_OFFSET > 0 - if ((U32)(ip - match) >= LIZARD_NOCHAIN_MIN_OFFSET) -#endif - if (*(iLowLimit + longest) == *(match - LLdelta + longest)) { - if (MEM_read32(match) == MEM_read32(ip)) { - int mlt = MINMATCH + Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit); - int back = 0; - while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - - if (mlt > longest) { - longest = (int)mlt; - *matchpos = match+back; - *startpos = ip+back; - } - } - } - } else { - const BYTE* match = dictBase + matchIndex; -#if LIZARD_NOCHAIN_MIN_OFFSET > 0 - if ((U32)(ip - (base + matchIndex)) >= LIZARD_NOCHAIN_MIN_OFFSET) -#endif - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(match) == MEM_read32(ip)) { - int back=0; - size_t mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; - while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - if ((int)mlt > longest) { longest = (int)mlt; *matchpos = base + matchIndex + back; *startpos = ip+back; } - } - } - } - - return longest; -} - - -FORCE_INLINE int Lizard_compress_noChain ( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const BYTE* anchor = ip; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = (iend - LASTLITERALS); - - int ml, ml2, ml3, ml0; - const BYTE* ref = NULL; - const BYTE* start2 = NULL; - const BYTE* ref2 = NULL; - const BYTE* start3 = NULL; - const BYTE* ref3 = NULL; - const BYTE* start0; - const BYTE* ref0; - - /* init */ - ip++; - - /* Main Loop */ - while (ip < mflimit) { - ml = Lizard_InsertAndFindBestMatchNoChain (ctx, ip, matchlimit, (&ref)); - if (!ml) { ip++; continue; } - - /* saved, in case we would skip too much */ - start0 = ip; - ref0 = ref; - ml0 = ml; - -_Search2: - if (ip+ml < mflimit) - ml2 = Lizard_InsertAndGetWiderMatchNoChain(ctx, ip + ml - 2, ip + 1, matchlimit, ml, &ref2, &start2); - else ml2 = ml; - - if (ml2 == ml) { /* No better match */ - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - continue; - } - - if (start0 < ip) { - if (start2 < ip + ml0) { /* empirical */ - ip = start0; - ref = ref0; - ml = ml0; - } - } - - /* Here, start0==ip */ - if ((start2 - ip) < 3) { /* First Match too small : removed */ - ml = ml2; - ip = start2; - ref =ref2; - goto _Search2; - } - -_Search3: - /* - * Currently we have : - * ml2 > ml1, and - * ip1+3 <= ip2 (usually < ip1+ml1) - */ - if ((start2 - ip) < OPTIMAL_ML) { - int correction; - int new_ml = ml; - if (new_ml > OPTIMAL_ML) new_ml = OPTIMAL_ML; - if (ip+new_ml > start2 + ml2 - MINMATCH) new_ml = (int)(start2 - ip) + ml2 - MINMATCH; - correction = new_ml - (int)(start2 - ip); - if (correction > 0) { - start2 += correction; - ref2 += correction; - ml2 -= correction; - } - } - /* Now, we have start2 = ip+new_ml, with new_ml = min(ml, OPTIMAL_ML=18) */ - - if (start2 + ml2 < mflimit) - ml3 = Lizard_InsertAndGetWiderMatchNoChain(ctx, start2 + ml2 - 3, start2, matchlimit, ml2, &ref3, &start3); - else ml3 = ml2; - - if (ml3 == ml2) { /* No better match : 2 sequences to encode */ - /* ip & ref are known; Now for ml */ - if (start2 < ip+ml) ml = (int)(start2 - ip); - /* Now, encode 2 sequences */ - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - ip = start2; - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml2, ref2)) return 0; - continue; - } - - if (start3 < ip+ml+3) { /* Not enough space for match 2 : remove it */ - if (start3 >= (ip+ml)) { /* can write Seq1 immediately ==> Seq2 is removed, so Seq3 becomes Seq1 */ - if (start2 < ip+ml) { - int correction = (int)(ip+ml - start2); - start2 += correction; - ref2 += correction; - ml2 -= correction; - if (ml2 < MINMATCH) { - start2 = start3; - ref2 = ref3; - ml2 = ml3; - } - } - - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - ip = start3; - ref = ref3; - ml = ml3; - - start0 = start2; - ref0 = ref2; - ml0 = ml2; - goto _Search2; - } - - start2 = start3; - ref2 = ref3; - ml2 = ml3; - goto _Search3; - } - - /* - * OK, now we have 3 ascending matches; let's write at least the first one - * ip & ref are known; Now for ml - */ - if (start2 < ip+ml) { - if ((start2 - ip) < (int)ML_MASK_LZ4) { - int correction; - if (ml > OPTIMAL_ML) ml = OPTIMAL_ML; - if (ip + ml > start2 + ml2 - MINMATCH) { - ml = (int)(start2 - ip) + ml2 - MINMATCH; - if (ml < MINMATCH) { // match2 doesn't fit, remove it - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - ip = start3; - ref = ref3; - ml = ml3; - - start0 = start2; - ref0 = ref2; - ml0 = ml2; - goto _Search2; - } - } - correction = ml - (int)(start2 - ip); - if (correction > 0) { - start2 += correction; - ref2 += correction; - ml2 -= correction; - } - } else { - ml = (int)(start2 - ip); - } - } - if (Lizard_encodeSequence_LZ4(ctx, &ip, &anchor, ml, ref)) return 0; - - ip = start2; - ref = ref2; - ml = ml2; - - start2 = start3; - ref2 = ref3; - ml2 = ml3; - - goto _Search3; - } - - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LZ4(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} diff --git a/contrib/lizard/lib/lizard_parser_optimal.h b/contrib/lizard/lib/lizard_parser_optimal.h deleted file mode 100644 index 8c9b83e9590..00000000000 --- a/contrib/lizard/lib/lizard_parser_optimal.h +++ /dev/null @@ -1,679 +0,0 @@ -#define LIZARD_LOG_PARSER(fmt, ...) //printf(fmt, __VA_ARGS__) -#define LIZARD_LOG_PRICE(fmt, ...) //printf(fmt, __VA_ARGS__) -#define LIZARD_LOG_ENCODE(fmt, ...) //printf(fmt, __VA_ARGS__) - -#define LIZARD_OPTIMAL_MIN_OFFSET 8 -#define LIZARD_OPT_NUM (1<<12) -#define REPMINMATCH 1 - - -FORCE_INLINE size_t Lizard_get_price(Lizard_stream_t* const ctx, int rep, const BYTE *ip, const BYTE *off24pos, size_t litLength, U32 offset, size_t matchLength) -{ - if (ctx->params.decompressType == Lizard_coderwords_LZ4) - return Lizard_get_price_LZ4(ctx, ip, litLength, offset, matchLength); - - return Lizard_get_price_LIZv1(ctx, rep, ip, off24pos, litLength, offset, matchLength); -} - - - -typedef struct -{ - int off; - int len; - int back; -} Lizard_match_t; - -typedef struct -{ - int price; - int off; - int mlen; - int litlen; - int rep; - const BYTE* off24pos; -} Lizard_optimal_t; - - -/* Update chains up to ip (excluded) */ -FORCE_INLINE void Lizard_BinTree_Insert(Lizard_stream_t* ctx, const BYTE* ip) -{ -#if MINMATCH == 3 - U32* HashTable3 = ctx->hashTable3; - const BYTE* const base = ctx->base; - const U32 target = (U32)(ip - base); - U32 idx = ctx->nextToUpdate; - - while(idx < target) { - HashTable3[Lizard_hash3Ptr(base+idx, ctx->params.hashLog3)] = idx; - idx++; - } - - ctx->nextToUpdate = target; -#else - (void)ctx; (void)ip; -#endif -} - - - -FORCE_INLINE int Lizard_GetAllMatches ( - Lizard_stream_t* ctx, - const BYTE* const ip, - const BYTE* const iLowLimit, - const BYTE* const iHighLimit, - size_t best_mlen, - Lizard_match_t* matches) -{ - U32* const chainTable = ctx->chainTable; - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const intptr_t dictLimit = ctx->dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; - const intptr_t current = (ip - base); - const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const BYTE* match, *matchDict; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - intptr_t matchIndex; - int nbAttempts = ctx->params.searchNum; - // bool fullSearch = (ctx->params.fullSearch >= 2); - int mnum = 0; - U32* HashPos; - size_t mlt; - - if (ip + MINMATCH > iHighLimit) return 0; - - /* First Match */ - HashPos = &HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; - matchIndex = *HashPos; -#if MINMATCH == 3 - { - U32* const HashTable3 = ctx->hashTable3; - U32* HashPos3 = &HashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; - - if ((*HashPos3 < current) && (*HashPos3 >= lowLimit)) { - size_t offset = current - *HashPos3; - if (offset < LIZARD_MAX_8BIT_OFFSET) { - match = ip - offset; - if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) { - size_t mlt = Lizard_count(ip + MINMATCH, match + MINMATCH, iHighLimit) + MINMATCH; - - int back = 0; - while ((ip + back > iLowLimit) && (match + back > lowPrefixPtr) && (ip[back - 1] == match[back - 1])) back--; - mlt -= back; - - matches[mnum].off = (int)offset; - matches[mnum].len = (int)mlt; - matches[mnum].back = -back; - mnum++; - } - } - } - - *HashPos3 = current; - } -#endif - - chainTable[current & contentMask] = (U32)(current - matchIndex); - *HashPos = (U32)current; - ctx->nextToUpdate++; - - if (best_mlen < MINMATCH-1) best_mlen = MINMATCH-1; - - while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { - nbAttempts--; - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_OPTIMAL_MIN_OFFSET) { - if (matchIndex >= dictLimit) { - if ((/*fullSearch ||*/ ip[best_mlen] == match[best_mlen]) && (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip))) { - int back = 0; - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iHighLimit) + MINMATCH; - while ((ip+back > iLowLimit) && (match+back > lowPrefixPtr) && (ip[back-1] == match[back-1])) back--; - mlt -= back; - - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (mlt > best_mlen) { - best_mlen = mlt; - matches[mnum].off = (int)(ip - match); - matches[mnum].len = (int)mlt; - matches[mnum].back = -back; - mnum++; - - if (best_mlen > LIZARD_OPT_NUM) break; - } - } - } else { - matchDict = dictBase + matchIndex; - // fprintf(stderr, "dictBase[%p]+matchIndex[%d]=match[%p] dictLimit=%d base=%p ip=%p iLimit=%p off=%d\n", dictBase, matchIndex, match, dictLimit, base, ip, iLimit, (U32)(ip-match)); - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_readMINMATCH(matchDict) == MEM_readMINMATCH(ip)) { - int back=0; - mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iHighLimit, dictEnd, lowPrefixPtr) + MINMATCH; - while ((ip+back > iLowLimit) && (matchIndex+back > lowLimit) && (ip[back-1] == matchDict[back-1])) back--; - mlt -= back; - - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (mlt > best_mlen) { - best_mlen = mlt; - matches[mnum].off = (int)(ip - match); - matches[mnum].len = (int)mlt; - matches[mnum].back = -back; - mnum++; - - if (best_mlen > LIZARD_OPT_NUM) break; - } - } - } - } - matchIndex -= chainTable[matchIndex & contentMask]; - } - - return mnum; -} - - - - -FORCE_INLINE int Lizard_BinTree_GetAllMatches ( - Lizard_stream_t* ctx, - const BYTE* const ip, - const BYTE* const iHighLimit, - size_t best_mlen, - Lizard_match_t* matches) -{ - U32* const chainTable = ctx->chainTable; - U32* const HashTable = ctx->hashTable; - const BYTE* const base = ctx->base; - const intptr_t dictLimit = ctx->dictLimit; - const BYTE* const dictBase = ctx->dictBase; - const BYTE* const dictEnd = dictBase + dictLimit; - const U32 contentMask = (1 << ctx->params.contentLog) - 1; - const BYTE* const lowPrefixPtr = base + dictLimit; - const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; - const intptr_t current = (ip - base); - const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; - const BYTE* match; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - int nbAttempts = ctx->params.searchNum; - int mnum = 0; - U32 *ptr0, *ptr1, delta0, delta1; - intptr_t matchIndex; - size_t mlt = 0; - U32* HashPos; - - if (ip + MINMATCH > iHighLimit) return 0; - - /* First Match */ - HashPos = &HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; - matchIndex = *HashPos; - - -#if MINMATCH == 3 - { - U32* HashPos3 = &ctx->hashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; - - if ((*HashPos3 < current) && (*HashPos3 >= lowLimit)) { - size_t offset = current - *HashPos3; - if (offset < LIZARD_MAX_8BIT_OFFSET) { - match = ip - offset; - if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) - { - mlt = Lizard_count(ip + MINMATCH, match + MINMATCH, iHighLimit) + MINMATCH; - - matches[mnum].off = (int)offset; - matches[mnum].len = (int)mlt; - matches[mnum].back = 0; - mnum++; - } - } - *HashPos3 = current; - } - } -#endif - - *HashPos = (U32)current; - ctx->nextToUpdate++; - - // check rest of matches - ptr0 = &chainTable[(current*2+1) & contentMask]; - ptr1 = &chainTable[(current*2) & contentMask]; - delta0 = delta1 = (U32)(current - matchIndex); - - if (best_mlen < MINMATCH-1) best_mlen = MINMATCH-1; - - while ((matchIndex < current) && (matchIndex >= lowLimit) && (nbAttempts)) { - nbAttempts--; - if (matchIndex >= dictLimit) { - match = base + matchIndex; - // if (ip[mlt] == match[mlt]) - mlt = Lizard_count(ip, match, iHighLimit); - } else { - match = dictBase + matchIndex; - mlt = Lizard_count_2segments(ip, match, iHighLimit, dictEnd, lowPrefixPtr); - if (matchIndex + (int)mlt >= dictLimit) - match = base + matchIndex; /* to prepare for next usage of match[mlt] */ - } - - if ((U32)(current - matchIndex) >= LIZARD_OPTIMAL_MIN_OFFSET) { - if ((mlt >= minMatchLongOff) || ((U32)(current - matchIndex) < LIZARD_MAX_16BIT_OFFSET)) - if (mlt > best_mlen) { - best_mlen = mlt; - matches[mnum].off = (int)(current - matchIndex); - matches[mnum].len = (int)mlt; - matches[mnum].back = 0; - mnum++; - - if (mlt > LIZARD_OPT_NUM) break; - if (ip + mlt >= iHighLimit) break; - } - } else { -#if 1 - intptr_t newMatchIndex; - size_t newml = 0, newoff = 0; - do { - newoff += (int)(current - matchIndex); - } while (newoff < LIZARD_OPTIMAL_MIN_OFFSET); - newMatchIndex = current - newoff; - if (newMatchIndex >= dictLimit) newml = Lizard_count(ip, base + newMatchIndex, iHighLimit); - - // printf("%d: off=%d mlt=%d\n", (U32)current, (U32)(current - matchIndex), (int)mlt); - // printf("%d: newoff=%d newml=%d\n", (U32)current, (int)newoff, (int)newml); - - if ((newml >= minMatchLongOff) && (newml > best_mlen)) { - best_mlen = newml; - matches[mnum].off = (int)newoff; - matches[mnum].len = (int)newml; - matches[mnum].back = 0; - mnum++; - - if (newml > LIZARD_OPT_NUM) break; - if (ip + newml >= iHighLimit) break; - } -#endif - } - - if (ip[mlt] < match[mlt]) { - *ptr0 = delta0; - ptr0 = &chainTable[(matchIndex*2) & contentMask]; - if (*ptr0 == (U32)-1) break; - delta0 = *ptr0; - delta1 += delta0; - matchIndex -= delta0; - } else { - *ptr1 = delta1; - ptr1 = &chainTable[(matchIndex*2+1) & contentMask]; - if (*ptr1 == (U32)-1) break; - delta1 = *ptr1; - delta0 += delta1; - matchIndex -= delta1; - } - } - - *ptr0 = (U32)-1; - *ptr1 = (U32)-1; - - return mnum; -} - - -#define SET_PRICE(pos, mlen, offset, litlen, price) \ - { \ - while (last_pos < pos) { opt[last_pos+1].price = LIZARD_MAX_PRICE; last_pos++; } \ - opt[pos].mlen = (int)mlen; \ - opt[pos].off = (int)offset; \ - opt[pos].litlen = (int)litlen; \ - opt[pos].price = (int)price; \ - LIZARD_LOG_PARSER("%d: SET price[%d/%d]=%d litlen=%d len=%d off=%d\n", (int)(inr-source), pos, last_pos, opt[pos].price, opt[pos].litlen, opt[pos].mlen, opt[pos].off); \ - } - - -FORCE_INLINE int Lizard_compress_optimalPrice( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - Lizard_optimal_t opt[LIZARD_OPT_NUM + 4]; - Lizard_match_t matches[LIZARD_OPT_NUM + 1]; - const BYTE *inr; - size_t res, cur, cur2, skip_num = 0; - size_t i, llen, litlen, mlen, best_mlen, price, offset, best_off, match_num, last_pos; - - const BYTE* anchor = ip; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = (iend - LASTLITERALS); - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const intptr_t dictLimit = ctx->dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - const BYTE* const lowPrefixPtr = base + dictLimit; - const intptr_t lowLimit = ctx->lowLimit; - const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; - - const size_t sufficient_len = ctx->params.sufficientLength; - const int faster_get_matches = (ctx->params.fullSearch == 0); - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - const int lizardOptimalMinOffset = (ctx->params.decompressType == Lizard_coderwords_LZ4) ? (1<<30) : LIZARD_OPTIMAL_MIN_OFFSET; - const size_t repMinMatch = (ctx->params.decompressType == Lizard_coderwords_LZ4) ? MINMATCH : REPMINMATCH; - - /* Main Loop */ - while (ip < mflimit) { - memset(opt, 0, sizeof(Lizard_optimal_t)); - last_pos = 0; - llen = ip - anchor; - - /* check rep code */ - - if (ctx->last_off >= lizardOptimalMinOffset) { - intptr_t matchIndexLO = (ip - ctx->last_off) - base; - mlen = 0; - if ((matchIndexLO >= lowLimit) && (base + matchIndexLO + maxDistance >= ip)) { - if (matchIndexLO >= dictLimit) { - mlen = Lizard_count(ip, base + matchIndexLO, matchlimit); - } else { - mlen = Lizard_count_2segments(ip, dictBase + matchIndexLO, matchlimit, dictEnd, lowPrefixPtr); - } - } - if (mlen >= REPMINMATCH) { - if (mlen > sufficient_len || mlen >= LIZARD_OPT_NUM) { - best_mlen = mlen; best_off = 0; cur = 0; last_pos = 1; - goto encode; - } - - do - { - litlen = 0; - price = Lizard_get_price(ctx, ctx->last_off, ip, ctx->off24pos, llen, 0, mlen); - if (mlen > last_pos || price < (size_t)opt[mlen].price) - SET_PRICE(mlen, mlen, 0, litlen, price); - mlen--; - } - while (mlen >= REPMINMATCH); - } - } - - if (faster_get_matches && last_pos) - match_num = 0; - else - { - if (ctx->params.parserType == Lizard_parser_optimalPrice) { - Lizard_Insert(ctx, ip); - match_num = Lizard_GetAllMatches(ctx, ip, ip, matchlimit, last_pos, matches); - } else { - Lizard_BinTree_Insert(ctx, ip); - match_num = Lizard_BinTree_GetAllMatches(ctx, ip, matchlimit, last_pos, matches); - } - } - - LIZARD_LOG_PARSER("%d: match_num=%d last_pos=%d\n", (int)(ip-source), match_num, last_pos); - if (!last_pos && !match_num) { ip++; continue; } - - if (match_num && (size_t)matches[match_num-1].len > sufficient_len) { - best_mlen = matches[match_num-1].len; - best_off = matches[match_num-1].off; - cur = 0; - last_pos = 1; - goto encode; - } - - // set prices using matches at position = 0 - best_mlen = (last_pos > MINMATCH) ? last_pos : MINMATCH; - - for (i = 0; i < match_num; i++) { - mlen = (i>0) ? (size_t)matches[i-1].len+1 : best_mlen; - best_mlen = (matches[i].len < LIZARD_OPT_NUM) ? matches[i].len : LIZARD_OPT_NUM; - LIZARD_LOG_PARSER("%d: start Found mlen=%d off=%d best_mlen=%d last_pos=%d\n", (int)(ip-source), matches[i].len, matches[i].off, best_mlen, last_pos); - while (mlen <= best_mlen){ - litlen = 0; - price = Lizard_get_price(ctx, ctx->last_off, ip, ctx->off24pos, llen + litlen, matches[i].off, mlen); - - if ((mlen >= minMatchLongOff) || (matches[i].off < LIZARD_MAX_16BIT_OFFSET)) - if (mlen > last_pos || price < (size_t)opt[mlen].price) - SET_PRICE(mlen, mlen, matches[i].off, litlen, price); - mlen++; - } - } - - if (last_pos < repMinMatch) { ip++; continue; } - - opt[0].off24pos = ctx->off24pos; - opt[0].rep = ctx->last_off; - opt[0].mlen = 1; - opt[0].off = -1; - - // check further positions - for (skip_num = 0, cur = 1; cur <= last_pos; cur++) { - int rep; - inr = ip + cur; - - if (opt[cur-1].off == -1) { // -1 = literals, 0 = rep - litlen = opt[cur-1].litlen + 1; - - if (cur != litlen) { - price = opt[cur - litlen].price + Lizard_get_price(ctx, opt[cur-litlen].rep, inr, ctx->off24pos, litlen, 0, 0); - LIZARD_LOG_PRICE("%d: TRY1 opt[%d].price=%d price=%d cur=%d litlen=%d\n", (int)(inr-source), cur - litlen, opt[cur - litlen].price, price, cur, litlen); - } else { - price = Lizard_get_price(ctx, ctx->last_off, inr, ctx->off24pos, llen + litlen, 0, 0); - LIZARD_LOG_PRICE("%d: TRY2 price=%d cur=%d litlen=%d llen=%d\n", (int)(inr-source), price, cur, litlen, llen); - } - } else { - litlen = 1; - price = opt[cur - 1].price + Lizard_get_price(ctx, opt[cur-1].rep, inr, ctx->off24pos, litlen, 0, 0); - LIZARD_LOG_PRICE("%d: TRY3 price=%d cur=%d litlen=%d litonly=%d\n", (int)(inr-source), price, cur, litlen, Lizard_get_price(ctx, rep, inr, ctx->off24pos, litlen, 0, 0)); - } - - mlen = 1; - best_mlen = 0; - LIZARD_LOG_PARSER("%d: TRY price=%d opt[%d].price=%d\n", (int)(inr-source), price, cur, opt[cur].price); - - if (cur > last_pos || price <= (size_t)opt[cur].price) // || ((price == opt[cur].price) && (opt[cur-1].mlen == 1) && (cur != litlen))) - SET_PRICE(cur, mlen, -1, litlen, price); - - if (cur == last_pos) break; - - - - /* set rep code */ - if (opt[cur].off != -1) { - mlen = opt[cur].mlen; - offset = opt[cur].off; - if (offset < 1) { - opt[cur].rep = opt[cur-mlen].rep; - opt[cur].off24pos = opt[cur-mlen].off24pos; - LIZARD_LOG_PARSER("%d: COPYREP1 cur=%d mlen=%d rep=%d\n", (int)(inr-source), cur, mlen, opt[cur-mlen].rep); - } else { - opt[cur].rep = (int)offset; - opt[cur].off24pos = (offset >= LIZARD_MAX_16BIT_OFFSET) ? inr : opt[cur-mlen].off24pos; - LIZARD_LOG_PARSER("%d: COPYREP2 cur=%d offset=%d rep=%d\n", (int)(inr-source), cur, offset, opt[cur].rep); - } - } else { - opt[cur].rep = opt[cur-1].rep; // copy rep - opt[cur].off24pos = opt[cur-1].off24pos; - } - - rep = opt[cur].rep; - LIZARD_LOG_PARSER("%d: CURRENT price[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(inr-source), cur, last_pos, opt[cur].price, opt[cur].off, opt[cur].mlen, opt[cur].litlen, opt[cur].rep); - - - /* check rep code */ - if (opt[cur].rep >= lizardOptimalMinOffset) { - intptr_t matchIndexLO = (inr - opt[cur].rep) - base; - mlen = 0; - if ((matchIndexLO >= lowLimit) && (base + matchIndexLO + maxDistance >= inr)) { - if (matchIndexLO >= dictLimit) { - mlen = Lizard_count(inr, base + matchIndexLO, matchlimit); - } else { - mlen = Lizard_count_2segments(inr, dictBase + matchIndexLO, matchlimit, dictEnd, lowPrefixPtr); - } - } - if (mlen >= REPMINMATCH/* && mlen > best_mlen*/) { - LIZARD_LOG_PARSER("%d: try REP rep=%d mlen=%d\n", (int)(inr-source), opt[cur].rep, mlen); - LIZARD_LOG_PARSER("%d: Found REP mlen=%d off=%d rep=%d opt[%d].off=%d\n", (int)(inr-source), mlen, 0, opt[cur].rep, cur, opt[cur].off); - - if (mlen > sufficient_len || cur + mlen >= LIZARD_OPT_NUM) { - best_mlen = mlen; - best_off = 0; - LIZARD_LOG_PARSER("%d: REP sufficient_len=%d best_mlen=%d best_off=%d last_pos=%d\n", (int)(inr-source), sufficient_len, best_mlen, best_off, last_pos); - last_pos = cur + 1; - goto encode; - } - - best_mlen = mlen; - if (faster_get_matches) - skip_num = best_mlen; - - do - { - //if (opt[cur].mlen == 1) - if (opt[cur].off == -1) { - litlen = opt[cur].litlen; - - if (cur != litlen) { - price = opt[cur - litlen].price + Lizard_get_price(ctx, rep, inr, opt[cur].off24pos, litlen, 0, mlen); - LIZARD_LOG_PRICE("%d: TRY1 opt[%d].price=%d price=%d cur=%d litlen=%d\n", (int)(inr-source), cur - litlen, opt[cur - litlen].price, price, cur, litlen); - } else { - price = Lizard_get_price(ctx, rep, inr, ctx->off24pos, llen + litlen, 0, mlen); - LIZARD_LOG_PRICE("%d: TRY2 price=%d cur=%d litlen=%d llen=%d\n", (int)(inr-source), price, cur, litlen, llen); - } - } else { - litlen = 0; - price = opt[cur].price + Lizard_get_price(ctx, rep, inr, opt[cur].off24pos, litlen, 0, mlen); - LIZARD_LOG_PRICE("%d: TRY3 price=%d cur=%d litlen=%d getprice=%d\n", (int)(inr-source), price, cur, litlen, Lizard_get_price(ctx, rep, inr, opt[cur].off24pos, litlen, 0, mlen - MINMATCH)); - } - - LIZARD_LOG_PARSER("%d: Found REP mlen=%d off=%d price=%d litlen=%d price[%d]=%d\n", (int)(inr-source), mlen, 0, price, litlen, cur - litlen, opt[cur - litlen].price); - - if (cur + mlen > last_pos || price <= (size_t)opt[cur + mlen].price) // || ((price == opt[cur + mlen].price) && (opt[cur].mlen == 1) && (cur != litlen))) // at equal price prefer REP instead of MATCH - SET_PRICE(cur + mlen, mlen, 0, litlen, price); - mlen--; - } - while (mlen >= REPMINMATCH); - } - } - - if (faster_get_matches && skip_num > 0) { - skip_num--; - continue; - } - - if (ctx->params.parserType == Lizard_parser_optimalPrice) { - Lizard_Insert(ctx, inr); - match_num = Lizard_GetAllMatches(ctx, inr, ip, matchlimit, best_mlen, matches); - LIZARD_LOG_PARSER("%d: Lizard_GetAllMatches match_num=%d\n", (int)(inr-source), match_num); - } else { - Lizard_BinTree_Insert(ctx, inr); - match_num = Lizard_BinTree_GetAllMatches(ctx, inr, matchlimit, best_mlen, matches); - LIZARD_LOG_PARSER("%d: Lizard_BinTree_GetAllMatches match_num=%d\n", (int)(inr-source), match_num); - } - - - if (match_num > 0 && (size_t)matches[match_num-1].len > sufficient_len) { - cur -= matches[match_num-1].back; - best_mlen = matches[match_num-1].len; - best_off = matches[match_num-1].off; - last_pos = cur + 1; - goto encode; - } - - // set prices using matches at position = cur - best_mlen = (best_mlen > MINMATCH) ? best_mlen : MINMATCH; - - for (i = 0; i < match_num; i++) { - mlen = (i>0) ? (size_t)matches[i-1].len+1 : best_mlen; - cur2 = cur - matches[i].back; - best_mlen = (cur2 + matches[i].len < LIZARD_OPT_NUM) ? (size_t)matches[i].len : LIZARD_OPT_NUM - cur2; - LIZARD_LOG_PARSER("%d: Found1 cur=%d cur2=%d mlen=%d off=%d best_mlen=%d last_pos=%d\n", (int)(inr-source), cur, cur2, matches[i].len, matches[i].off, best_mlen, last_pos); - - if (mlen < (size_t)matches[i].back + 1) - mlen = matches[i].back + 1; - - while (mlen <= best_mlen) { - // if (opt[cur2].mlen == 1) - if (opt[cur2].off == -1) - { - litlen = opt[cur2].litlen; - - if (cur2 != litlen) - price = opt[cur2 - litlen].price + Lizard_get_price(ctx, rep, inr, opt[cur2].off24pos, litlen, matches[i].off, mlen); - else - price = Lizard_get_price(ctx, rep, inr, ctx->off24pos, llen + litlen, matches[i].off, mlen); - } else { - litlen = 0; - price = opt[cur2].price + Lizard_get_price(ctx, rep, inr, opt[cur2].off24pos, litlen, matches[i].off, mlen); - } - - LIZARD_LOG_PARSER("%d: Found2 pred=%d mlen=%d best_mlen=%d off=%d price=%d litlen=%d price[%d]=%d\n", (int)(inr-source), matches[i].back, mlen, best_mlen, matches[i].off, price, litlen, cur - litlen, opt[cur - litlen].price); - // if (cur2 + mlen > last_pos || ((matches[i].off != opt[cur2 + mlen].off) && (price < opt[cur2 + mlen].price))) - - if ((mlen >= minMatchLongOff) || (matches[i].off < LIZARD_MAX_16BIT_OFFSET)) - if (cur2 + mlen > last_pos || price < (size_t)opt[cur2 + mlen].price) - { - SET_PRICE(cur2 + mlen, mlen, matches[i].off, litlen, price); - } - - mlen++; - } - } - } // for (skip_num = 0, cur = 1; cur <= last_pos; cur++) - - - best_mlen = opt[last_pos].mlen; - best_off = opt[last_pos].off; - cur = last_pos - best_mlen; - - encode: // cur, last_pos, best_mlen, best_off have to be set - for (i = 1; i <= last_pos; i++) { - LIZARD_LOG_PARSER("%d: price[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(ip-source+i), i, last_pos, opt[i].price, opt[i].off, opt[i].mlen, opt[i].litlen, opt[i].rep); - } - - LIZARD_LOG_PARSER("%d: cur=%d/%d best_mlen=%d best_off=%d rep=%d\n", (int)(ip-source+cur), cur, last_pos, best_mlen, best_off, opt[cur].rep); - - opt[0].mlen = 1; - - while (1) { - mlen = opt[cur].mlen; - offset = opt[cur].off; - opt[cur].mlen = (int)best_mlen; - opt[cur].off = (int)best_off; - best_mlen = mlen; - best_off = offset; - if (mlen > cur) break; - cur -= mlen; - } - - for (i = 0; i <= last_pos;) { - LIZARD_LOG_PARSER("%d: price2[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(ip-source+i), i, last_pos, opt[i].price, opt[i].off, opt[i].mlen, opt[i].litlen, opt[i].rep); - i += opt[i].mlen; - } - - cur = 0; - - while (cur < last_pos) { - LIZARD_LOG_PARSER("%d: price3[%d/%d]=%d off=%d mlen=%d litlen=%d rep=%d\n", (int)(ip-source+cur), cur, last_pos, opt[cur].price, opt[cur].off, opt[cur].mlen, opt[cur].litlen, opt[cur].rep); - mlen = opt[cur].mlen; - // if (mlen == 1) { ip++; cur++; continue; } - if (opt[cur].off == -1) { ip++; cur++; continue; } - offset = opt[cur].off; - cur += mlen; - - LIZARD_LOG_ENCODE("%d: ENCODE literals=%d off=%d mlen=%d ", (int)(ip-source), (int)(ip-anchor), (int)(offset), mlen); - res = Lizard_encodeSequence(ctx, &ip, &anchor, mlen, ip - offset); - if (res) return 0; - - LIZARD_LOG_PARSER("%d: offset=%d rep=%d\n", (int)(ip-source), offset, ctx->last_off); - } - } - - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} - diff --git a/contrib/lizard/lib/lizard_parser_pricefast.h b/contrib/lizard/lib/lizard_parser_pricefast.h deleted file mode 100644 index 9818e2c4212..00000000000 --- a/contrib/lizard/lib/lizard_parser_pricefast.h +++ /dev/null @@ -1,250 +0,0 @@ -#define LIZARD_PRICEFAST_MIN_OFFSET 8 - -FORCE_INLINE int Lizard_FindMatchFast(Lizard_stream_t* ctx, intptr_t matchIndex, intptr_t matchIndex3, /* Index table will be updated */ - const BYTE* ip, const BYTE* const iLimit, - const BYTE** matchpos) -{ - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const intptr_t dictLimit = ctx->dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - const intptr_t maxDistance = (1 << ctx->params.windowLog) - 1; - const intptr_t current = (U32)(ip - base); - const intptr_t lowLimit = ((intptr_t)ctx->lowLimit + maxDistance >= current) ? (intptr_t)ctx->lowLimit : current - maxDistance; - const BYTE* const lowPrefixPtr = base + dictLimit; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - const BYTE* match, *matchDict; - size_t ml=0, mlt; - - if (ctx->last_off >= LIZARD_PRICEFAST_MIN_OFFSET) { - intptr_t matchIndexLO = (ip - ctx->last_off) - base; - if (matchIndexLO >= lowLimit) { - if (matchIndexLO >= dictLimit) { - match = base + matchIndexLO; - if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) - { - *matchpos = match; - return (int)mlt; - } - } - } else { - match = dictBase + matchIndexLO; - if ((U32)((dictLimit-1) - matchIndexLO) >= 3) /* intentional overflow */ - if (MEM_readMINMATCH(match) == MEM_readMINMATCH(ip)) { - mlt = Lizard_count_2segments(ip+MINMATCH, match+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; - // if ((mlt >= minMatchLongOff) || (ctx->last_off < LIZARD_MAX_16BIT_OFFSET)) - { - *matchpos = base + matchIndexLO; /* virtual matchpos */ - return (int)mlt; - } - } - } - } - } - - -#if MINMATCH == 3 - if (matchIndex3 < current && matchIndex3 >= lowLimit) { - intptr_t offset = current - matchIndex3; - if (offset < LIZARD_MAX_8BIT_OFFSET) { - match = ip - offset; - if (match > base && MEM_readMINMATCH(ip) == MEM_readMINMATCH(match)) { - ml = 3;//Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - *matchpos = match; - } - } - } -#else - (void)matchIndex3; -#endif - - if ((matchIndex < current) && (matchIndex >= lowLimit)) { - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_PRICEFAST_MIN_OFFSET) { - if (matchIndex >= dictLimit) { - if (*(match+ml) == *(ip+ml) && (MEM_read32(match) == MEM_read32(ip))) { - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (!ml || (mlt > ml)) // && Lizard_better_price((ip - *matchpos), ml, (ip - match), mlt, ctx->last_off))) - { ml = mlt; *matchpos = match; } - } - } else { - matchDict = dictBase + matchIndex; - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(matchDict) == MEM_read32(ip)) { - mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - if (!ml || (mlt > ml)) // && Lizard_better_price((ip - *matchpos), ml, (U32)(ip - match), mlt, ctx->last_off))) - { ml = mlt; *matchpos = match; } /* virtual matchpos */ - } - } - } - } - - return (int)ml; -} - - -FORCE_INLINE int Lizard_FindMatchFaster (Lizard_stream_t* ctx, U32 matchIndex, /* Index table will be updated */ - const BYTE* ip, const BYTE* const iLimit, - const BYTE** matchpos) -{ - const BYTE* const base = ctx->base; - const BYTE* const dictBase = ctx->dictBase; - const U32 dictLimit = ctx->dictLimit; - const BYTE* const dictEnd = dictBase + dictLimit; - const U32 maxDistance = (1 << ctx->params.windowLog) - 1; - const U32 current = (U32)(ip - base); - const U32 lowLimit = (ctx->lowLimit + maxDistance >= current) ? ctx->lowLimit : current - maxDistance; - const BYTE* const lowPrefixPtr = base + dictLimit; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - const BYTE* match, *matchDict; - size_t ml=0, mlt; - - if (matchIndex < current && matchIndex >= lowLimit) { - match = base + matchIndex; - if ((U32)(ip - match) >= LIZARD_PRICEFAST_MIN_OFFSET) { - if (matchIndex >= dictLimit) { - if (MEM_read32(match) == MEM_read32(ip)) { - mlt = Lizard_count(ip+MINMATCH, match+MINMATCH, iLimit) + MINMATCH; - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - { ml = mlt; *matchpos = match; } - } - } else { - matchDict = dictBase + matchIndex; - if ((U32)((dictLimit-1) - matchIndex) >= 3) /* intentional overflow */ - if (MEM_read32(matchDict) == MEM_read32(ip)) { - mlt = Lizard_count_2segments(ip+MINMATCH, matchDict+MINMATCH, iLimit, dictEnd, lowPrefixPtr) + MINMATCH; - if ((mlt >= minMatchLongOff) || ((U32)(ip - match) < LIZARD_MAX_16BIT_OFFSET)) - { ml = mlt; *matchpos = match; } /* virtual matchpos */ - } - } - } - } - - return (int)ml; -} - - - -FORCE_INLINE int Lizard_compress_priceFast( - Lizard_stream_t* const ctx, - const BYTE* ip, - const BYTE* const iend) -{ - const BYTE* anchor = ip; - const BYTE* const mflimit = iend - MFLIMIT; - const BYTE* const matchlimit = (iend - LASTLITERALS); - - size_t ml, ml2=0; - const BYTE* ref=NULL; - const BYTE* start2=NULL; - const BYTE* ref2=NULL; - const BYTE* lowPrefixPtr = ctx->base + ctx->dictLimit; - U32* HashTable = ctx->hashTable; -#if MINMATCH == 3 - U32* HashTable3 = ctx->hashTable3; -#endif - const BYTE* const base = ctx->base; - const size_t minMatchLongOff = ctx->params.minMatchLongOff; - U32* HashPos; - - /* init */ - ip++; - - /* Main Loop */ - while (ip < mflimit) - { - HashPos = &HashTable[Lizard_hashPtr(ip, ctx->params.hashLog, ctx->params.searchLength)]; -#if MINMATCH == 3 - { - U32* HashPos3 = &HashTable3[Lizard_hash3Ptr(ip, ctx->params.hashLog3)]; - ml = Lizard_FindMatchFast (ctx, *HashPos, *HashPos3, ip, matchlimit, (&ref)); - *HashPos3 = (U32)(ip - base); - } -#else - ml = Lizard_FindMatchFast (ctx, *HashPos, 0, ip, matchlimit, (&ref)); -#endif - if ((*HashPos >= (U32)(ip - base)) || ((U32)(ip - base) >= *HashPos + LIZARD_PRICEFAST_MIN_OFFSET)) - *HashPos = (U32)(ip - base); - - if (!ml) { ip++; continue; } - if ((int)(ip - ref) == ctx->last_off) { ml2=0; ref=ip; goto _Encode; } - - { - int back = 0; - while ((ip+back>anchor) && (ref+back > lowPrefixPtr) && (ip[back-1] == ref[back-1])) back--; - ml -= back; - ip += back; - ref += back; - } - -_Search: - if (ip+ml >= mflimit) goto _Encode; - - start2 = ip + ml - 2; - HashPos = &HashTable[Lizard_hashPtr(start2, ctx->params.hashLog, ctx->params.searchLength)]; - ml2 = Lizard_FindMatchFaster(ctx, *HashPos, start2, matchlimit, (&ref2)); - if ((*HashPos >= (U32)(start2 - base)) || ((U32)(start2 - base) >= *HashPos + LIZARD_PRICEFAST_MIN_OFFSET)) - *HashPos = (U32)(start2 - base); - - if (!ml2) goto _Encode; - - { - int back = 0; - while ((start2+back>ip) && (ref2+back > lowPrefixPtr) && (start2[back-1] == ref2[back-1])) back--; - ml2 -= back; - start2 += back; - ref2 += back; - } - - if (ml2 <= ml) { ml2 = 0; goto _Encode; } - - if (start2 <= ip) - { - - ip = start2; ref = ref2; ml = ml2; - ml2 = 0; - goto _Encode; - } - - if (start2 - ip < 3) - { - ip = start2; ref = ref2; ml = ml2; - ml2 = 0; - goto _Search; - } - - if (start2 < ip + ml) - { - size_t correction = ml - (int)(start2 - ip); - start2 += correction; - ref2 += correction; - ml2 -= correction; - if (ml2 < 3) { ml2 = 0; } - if ((ml2 < minMatchLongOff) && ((U32)(start2 - ref2) >= LIZARD_MAX_16BIT_OFFSET)) { ml2 = 0; } - } - -_Encode: - if (Lizard_encodeSequence_LIZv1(ctx, &ip, &anchor, ml, ref)) goto _output_error; - - if (ml2) - { - ip = start2; ref = ref2; ml = ml2; - ml2 = 0; - goto _Search; - } - } - - /* Encode Last Literals */ - ip = iend; - if (Lizard_encodeLastLiterals_LIZv1(ctx, &ip, &anchor)) goto _output_error; - - /* End */ - return 1; -_output_error: - return 0; -} - diff --git a/contrib/lizard/lib/xxhash/xxhash.c b/contrib/lizard/lib/xxhash/xxhash.c deleted file mode 100644 index 833b99f3b21..00000000000 --- a/contrib/lizard/lib/xxhash/xxhash.c +++ /dev/null @@ -1,888 +0,0 @@ -/* -* xxHash - Fast Hash algorithm -* Copyright (C) 2012-2016, Yann Collet -* -* BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) -* -* Redistribution and use in source and binary forms, with or without -* modification, are permitted provided that the following conditions are -* met: -* -* * Redistributions of source code must retain the above copyright -* notice, this list of conditions and the following disclaimer. -* * Redistributions in binary form must reproduce the above -* copyright notice, this list of conditions and the following disclaimer -* in the documentation and/or other materials provided with the -* distribution. -* -* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -* -* You can contact the author at : -* - xxHash homepage: http://www.xxhash.com -* - xxHash source repository : https://github.com/Cyan4973/xxHash -*/ - - -/* ************************************* -* Tuning parameters -***************************************/ -/*!XXH_FORCE_MEMORY_ACCESS : - * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable. - * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal. - * The below switch allow to select different access method for improved performance. - * Method 0 (default) : use `memcpy()`. Safe and portable. - * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable). - * This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`. - * Method 2 : direct access. This method doesn't depend on compiler but violate C standard. - * It can generate buggy code on targets which do not support unaligned memory accesses. - * But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6) - * See http://stackoverflow.com/a/32095106/646947 for details. - * Prefer these methods in priority order (0 > 1 > 2) - */ -#ifndef XXH_FORCE_MEMORY_ACCESS /* can be defined externally, on command line for example */ -# if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) ) -# define XXH_FORCE_MEMORY_ACCESS 2 -# elif defined(__INTEL_COMPILER) || \ - (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) )) -# define XXH_FORCE_MEMORY_ACCESS 1 -# endif -#endif - -/*!XXH_ACCEPT_NULL_INPUT_POINTER : - * If the input pointer is a null pointer, xxHash default behavior is to trigger a memory access error, since it is a bad pointer. - * When this option is enabled, xxHash output for null input pointers will be the same as a null-length input. - * By default, this option is disabled. To enable it, uncomment below define : - */ -/* #define XXH_ACCEPT_NULL_INPUT_POINTER 1 */ - -/*!XXH_FORCE_NATIVE_FORMAT : - * By default, xxHash library provides endian-independent Hash values, based on little-endian convention. - * Results are therefore identical for little-endian and big-endian CPU. - * This comes at a performance cost for big-endian CPU, since some swapping is required to emulate little-endian format. - * Should endian-independence be of no importance for your application, you may set the #define below to 1, - * to improve speed for Big-endian CPU. - * This option has no impact on Little_Endian CPU. - */ -#ifndef XXH_FORCE_NATIVE_FORMAT /* can be defined externally */ -# define XXH_FORCE_NATIVE_FORMAT 0 -#endif - -/*!XXH_FORCE_ALIGN_CHECK : - * This is a minor performance trick, only useful with lots of very small keys. - * It means : check for aligned/unaligned input. - * The check costs one initial branch per hash; - * set it to 0 when the input is guaranteed to be aligned, - * or when alignment doesn't matter for performance. - */ -#ifndef XXH_FORCE_ALIGN_CHECK /* can be defined externally */ -# if defined(__i386) || defined(_M_IX86) || defined(__x86_64__) || defined(_M_X64) -# define XXH_FORCE_ALIGN_CHECK 0 -# else -# define XXH_FORCE_ALIGN_CHECK 1 -# endif -#endif - - -/* ************************************* -* Includes & Memory related functions -***************************************/ -/*! Modify the local functions below should you wish to use some other memory routines -* for malloc(), free() */ -#include -static void* XXH_malloc(size_t s) { return malloc(s); } -static void XXH_free (void* p) { free(p); } -/*! and for memcpy() */ -#include -static void* XXH_memcpy(void* dest, const void* src, size_t size) { return memcpy(dest,src,size); } - -#define XXH_STATIC_LINKING_ONLY -#include "xxhash.h" - - -/* ************************************* -* Compiler Specific Options -***************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# define FORCE_INLINE static __forceinline -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - - -/* ************************************* -* Basic Types -***************************************/ -#ifndef MEM_MODULE -# if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) -# include - typedef uint8_t BYTE; - typedef uint16_t U16; - typedef uint32_t U32; -# else - typedef unsigned char BYTE; - typedef unsigned short U16; - typedef unsigned int U32; -# endif -#endif - -#if (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==2)) - -/* Force direct memory access. Only works on CPU which support unaligned memory access in hardware */ -static U32 XXH_read32(const void* memPtr) { return *(const U32*) memPtr; } - -#elif (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==1)) - -/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */ -/* currently only defined for gcc and icc */ -typedef union { U32 u32; } __attribute__((packed)) unalign; -static U32 XXH_read32(const void* ptr) { return ((const unalign*)ptr)->u32; } - -#else - -/* portable and safe solution. Generally efficient. - * see : http://stackoverflow.com/a/32095106/646947 - */ -static U32 XXH_read32(const void* memPtr) -{ - U32 val; - memcpy(&val, memPtr, sizeof(val)); - return val; -} - -#endif /* XXH_FORCE_DIRECT_MEMORY_ACCESS */ - - -/* **************************************** -* Compiler-specific Functions and Macros -******************************************/ -#define XXH_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) - -/* Note : although _rotl exists for minGW (GCC under windows), performance seems poor */ -#if defined(_MSC_VER) -# define XXH_rotl32(x,r) _rotl(x,r) -# define XXH_rotl64(x,r) _rotl64(x,r) -#else -# define XXH_rotl32(x,r) ((x << r) | (x >> (32 - r))) -# define XXH_rotl64(x,r) ((x << r) | (x >> (64 - r))) -#endif - -#if defined(_MSC_VER) /* Visual Studio */ -# define XXH_swap32 _byteswap_ulong -#elif XXH_GCC_VERSION >= 403 -# define XXH_swap32 __builtin_bswap32 -#else -static U32 XXH_swap32 (U32 x) -{ - return ((x << 24) & 0xff000000 ) | - ((x << 8) & 0x00ff0000 ) | - ((x >> 8) & 0x0000ff00 ) | - ((x >> 24) & 0x000000ff ); -} -#endif - - -/* ************************************* -* Architecture Macros -***************************************/ -typedef enum { XXH_bigEndian=0, XXH_littleEndian=1 } XXH_endianess; - -/* XXH_CPU_LITTLE_ENDIAN can be defined externally, for example on the compiler command line */ -#ifndef XXH_CPU_LITTLE_ENDIAN - static const int g_one = 1; -# define XXH_CPU_LITTLE_ENDIAN (*(const char*)(&g_one)) -#endif - - -/* *************************** -* Memory reads -*****************************/ -typedef enum { XXH_aligned, XXH_unaligned } XXH_alignment; - -FORCE_INLINE U32 XXH_readLE32_align(const void* ptr, XXH_endianess endian, XXH_alignment align) -{ - if (align==XXH_unaligned) - return endian==XXH_littleEndian ? XXH_read32(ptr) : XXH_swap32(XXH_read32(ptr)); - else - return endian==XXH_littleEndian ? *(const U32*)ptr : XXH_swap32(*(const U32*)ptr); -} - -FORCE_INLINE U32 XXH_readLE32(const void* ptr, XXH_endianess endian) -{ - return XXH_readLE32_align(ptr, endian, XXH_unaligned); -} - -static U32 XXH_readBE32(const void* ptr) -{ - return XXH_CPU_LITTLE_ENDIAN ? XXH_swap32(XXH_read32(ptr)) : XXH_read32(ptr); -} - - -/* ************************************* -* Macros -***************************************/ -#define XXH_STATIC_ASSERT(c) { enum { XXH_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ -XXH_PUBLIC_API unsigned XXH_versionNumber (void) { return XXH_VERSION_NUMBER; } - - -/* ******************************************************************* -* 32-bits hash functions -*********************************************************************/ -static const U32 PRIME32_1 = 2654435761U; -static const U32 PRIME32_2 = 2246822519U; -static const U32 PRIME32_3 = 3266489917U; -static const U32 PRIME32_4 = 668265263U; -static const U32 PRIME32_5 = 374761393U; - -static U32 XXH32_round(U32 seed, U32 input) -{ - seed += input * PRIME32_2; - seed = XXH_rotl32(seed, 13); - seed *= PRIME32_1; - return seed; -} - -FORCE_INLINE U32 XXH32_endian_align(const void* input, size_t len, U32 seed, XXH_endianess endian, XXH_alignment align) -{ - const BYTE* p = (const BYTE*)input; - const BYTE* bEnd = p + len; - U32 h32; -#define XXH_get32bits(p) XXH_readLE32_align(p, endian, align) - -#ifdef XXH_ACCEPT_NULL_INPUT_POINTER - if (p==NULL) { - len=0; - bEnd=p=(const BYTE*)(size_t)16; - } -#endif - - if (len>=16) { - const BYTE* const limit = bEnd - 16; - U32 v1 = seed + PRIME32_1 + PRIME32_2; - U32 v2 = seed + PRIME32_2; - U32 v3 = seed + 0; - U32 v4 = seed - PRIME32_1; - - do { - v1 = XXH32_round(v1, XXH_get32bits(p)); p+=4; - v2 = XXH32_round(v2, XXH_get32bits(p)); p+=4; - v3 = XXH32_round(v3, XXH_get32bits(p)); p+=4; - v4 = XXH32_round(v4, XXH_get32bits(p)); p+=4; - } while (p<=limit); - - h32 = XXH_rotl32(v1, 1) + XXH_rotl32(v2, 7) + XXH_rotl32(v3, 12) + XXH_rotl32(v4, 18); - } else { - h32 = seed + PRIME32_5; - } - - h32 += (U32) len; - - while (p+4<=bEnd) { - h32 += XXH_get32bits(p) * PRIME32_3; - h32 = XXH_rotl32(h32, 17) * PRIME32_4 ; - p+=4; - } - - while (p> 15; - h32 *= PRIME32_2; - h32 ^= h32 >> 13; - h32 *= PRIME32_3; - h32 ^= h32 >> 16; - - return h32; -} - - -XXH_PUBLIC_API unsigned int XXH32 (const void* input, size_t len, unsigned int seed) -{ -#if 0 - /* Simple version, good for code maintenance, but unfortunately slow for small inputs */ - XXH32_state_t state; - XXH32_reset(&state, seed); - XXH32_update(&state, input, len); - return XXH32_digest(&state); -#else - XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; - - if (XXH_FORCE_ALIGN_CHECK) { - if ((((size_t)input) & 3) == 0) { /* Input is 4-bytes aligned, leverage the speed benefit */ - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned); - else - return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned); - } } - - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned); - else - return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned); -#endif -} - - - -/*====== Hash streaming ======*/ - -XXH_PUBLIC_API XXH32_state_t* XXH32_createState(void) -{ - return (XXH32_state_t*)XXH_malloc(sizeof(XXH32_state_t)); -} -XXH_PUBLIC_API XXH_errorcode XXH32_freeState(XXH32_state_t* statePtr) -{ - XXH_free(statePtr); - return XXH_OK; -} - -XXH_PUBLIC_API void XXH32_copyState(XXH32_state_t* dstState, const XXH32_state_t* srcState) -{ - memcpy(dstState, srcState, sizeof(*dstState)); -} - -XXH_PUBLIC_API XXH_errorcode XXH32_reset(XXH32_state_t* statePtr, unsigned int seed) -{ - XXH32_state_t state; /* using a local state to memcpy() in order to avoid strict-aliasing warnings */ - memset(&state, 0, sizeof(state)-4); /* do not write into reserved, for future removal */ - state.v1 = seed + PRIME32_1 + PRIME32_2; - state.v2 = seed + PRIME32_2; - state.v3 = seed + 0; - state.v4 = seed - PRIME32_1; - memcpy(statePtr, &state, sizeof(state)); - return XXH_OK; -} - - -FORCE_INLINE XXH_errorcode XXH32_update_endian (XXH32_state_t* state, const void* input, size_t len, XXH_endianess endian) -{ - const BYTE* p = (const BYTE*)input; - const BYTE* const bEnd = p + len; - -#ifdef XXH_ACCEPT_NULL_INPUT_POINTER - if (input==NULL) return XXH_ERROR; -#endif - - state->total_len_32 += (unsigned)len; - state->large_len |= (len>=16) | (state->total_len_32>=16); - - if (state->memsize + len < 16) { /* fill in tmp buffer */ - XXH_memcpy((BYTE*)(state->mem32) + state->memsize, input, len); - state->memsize += (unsigned)len; - return XXH_OK; - } - - if (state->memsize) { /* some data left from previous update */ - XXH_memcpy((BYTE*)(state->mem32) + state->memsize, input, 16-state->memsize); - { const U32* p32 = state->mem32; - state->v1 = XXH32_round(state->v1, XXH_readLE32(p32, endian)); p32++; - state->v2 = XXH32_round(state->v2, XXH_readLE32(p32, endian)); p32++; - state->v3 = XXH32_round(state->v3, XXH_readLE32(p32, endian)); p32++; - state->v4 = XXH32_round(state->v4, XXH_readLE32(p32, endian)); - } - p += 16-state->memsize; - state->memsize = 0; - } - - if (p <= bEnd-16) { - const BYTE* const limit = bEnd - 16; - U32 v1 = state->v1; - U32 v2 = state->v2; - U32 v3 = state->v3; - U32 v4 = state->v4; - - do { - v1 = XXH32_round(v1, XXH_readLE32(p, endian)); p+=4; - v2 = XXH32_round(v2, XXH_readLE32(p, endian)); p+=4; - v3 = XXH32_round(v3, XXH_readLE32(p, endian)); p+=4; - v4 = XXH32_round(v4, XXH_readLE32(p, endian)); p+=4; - } while (p<=limit); - - state->v1 = v1; - state->v2 = v2; - state->v3 = v3; - state->v4 = v4; - } - - if (p < bEnd) { - XXH_memcpy(state->mem32, p, (size_t)(bEnd-p)); - state->memsize = (unsigned)(bEnd-p); - } - - return XXH_OK; -} - -XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* state_in, const void* input, size_t len) -{ - XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; - - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH32_update_endian(state_in, input, len, XXH_littleEndian); - else - return XXH32_update_endian(state_in, input, len, XXH_bigEndian); -} - - - -FORCE_INLINE U32 XXH32_digest_endian (const XXH32_state_t* state, XXH_endianess endian) -{ - const BYTE * p = (const BYTE*)state->mem32; - const BYTE* const bEnd = (const BYTE*)(state->mem32) + state->memsize; - U32 h32; - - if (state->large_len) { - h32 = XXH_rotl32(state->v1, 1) + XXH_rotl32(state->v2, 7) + XXH_rotl32(state->v3, 12) + XXH_rotl32(state->v4, 18); - } else { - h32 = state->v3 /* == seed */ + PRIME32_5; - } - - h32 += state->total_len_32; - - while (p+4<=bEnd) { - h32 += XXH_readLE32(p, endian) * PRIME32_3; - h32 = XXH_rotl32(h32, 17) * PRIME32_4; - p+=4; - } - - while (p> 15; - h32 *= PRIME32_2; - h32 ^= h32 >> 13; - h32 *= PRIME32_3; - h32 ^= h32 >> 16; - - return h32; -} - - -XXH_PUBLIC_API unsigned int XXH32_digest (const XXH32_state_t* state_in) -{ - XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; - - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH32_digest_endian(state_in, XXH_littleEndian); - else - return XXH32_digest_endian(state_in, XXH_bigEndian); -} - - -/*====== Canonical representation ======*/ - -/*! Default XXH result types are basic unsigned 32 and 64 bits. -* The canonical representation follows human-readable write convention, aka big-endian (large digits first). -* These functions allow transformation of hash result into and from its canonical format. -* This way, hash values can be written into a file or buffer, and remain comparable across different systems and programs. -*/ - -XXH_PUBLIC_API void XXH32_canonicalFromHash(XXH32_canonical_t* dst, XXH32_hash_t hash) -{ - XXH_STATIC_ASSERT(sizeof(XXH32_canonical_t) == sizeof(XXH32_hash_t)); - if (XXH_CPU_LITTLE_ENDIAN) hash = XXH_swap32(hash); - memcpy(dst, &hash, sizeof(*dst)); -} - -XXH_PUBLIC_API XXH32_hash_t XXH32_hashFromCanonical(const XXH32_canonical_t* src) -{ - return XXH_readBE32(src); -} - - -#ifndef XXH_NO_LONG_LONG - -/* ******************************************************************* -* 64-bits hash functions -*********************************************************************/ - -/*====== Memory access ======*/ - -#ifndef MEM_MODULE -# define MEM_MODULE -# if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) -# include - typedef uint64_t U64; -# else - typedef unsigned long long U64; /* if your compiler doesn't support unsigned long long, replace by another 64-bit type here. Note that xxhash.h will also need to be updated. */ -# endif -#endif - - -#if (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==2)) - -/* Force direct memory access. Only works on CPU which support unaligned memory access in hardware */ -static U64 XXH_read64(const void* memPtr) { return *(const U64*) memPtr; } - -#elif (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==1)) - -/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */ -/* currently only defined for gcc and icc */ -typedef union { U32 u32; U64 u64; } __attribute__((packed)) unalign64; -static U64 XXH_read64(const void* ptr) { return ((const unalign64*)ptr)->u64; } - -#else - -/* portable and safe solution. Generally efficient. - * see : http://stackoverflow.com/a/32095106/646947 - */ - -static U64 XXH_read64(const void* memPtr) -{ - U64 val; - memcpy(&val, memPtr, sizeof(val)); - return val; -} - -#endif /* XXH_FORCE_DIRECT_MEMORY_ACCESS */ - -#if defined(_MSC_VER) /* Visual Studio */ -# define XXH_swap64 _byteswap_uint64 -#elif XXH_GCC_VERSION >= 403 -# define XXH_swap64 __builtin_bswap64 -#else -static U64 XXH_swap64 (U64 x) -{ - return ((x << 56) & 0xff00000000000000ULL) | - ((x << 40) & 0x00ff000000000000ULL) | - ((x << 24) & 0x0000ff0000000000ULL) | - ((x << 8) & 0x000000ff00000000ULL) | - ((x >> 8) & 0x00000000ff000000ULL) | - ((x >> 24) & 0x0000000000ff0000ULL) | - ((x >> 40) & 0x000000000000ff00ULL) | - ((x >> 56) & 0x00000000000000ffULL); -} -#endif - -FORCE_INLINE U64 XXH_readLE64_align(const void* ptr, XXH_endianess endian, XXH_alignment align) -{ - if (align==XXH_unaligned) - return endian==XXH_littleEndian ? XXH_read64(ptr) : XXH_swap64(XXH_read64(ptr)); - else - return endian==XXH_littleEndian ? *(const U64*)ptr : XXH_swap64(*(const U64*)ptr); -} - -FORCE_INLINE U64 XXH_readLE64(const void* ptr, XXH_endianess endian) -{ - return XXH_readLE64_align(ptr, endian, XXH_unaligned); -} - -static U64 XXH_readBE64(const void* ptr) -{ - return XXH_CPU_LITTLE_ENDIAN ? XXH_swap64(XXH_read64(ptr)) : XXH_read64(ptr); -} - - -/*====== xxh64 ======*/ - -static const U64 PRIME64_1 = 11400714785074694791ULL; -static const U64 PRIME64_2 = 14029467366897019727ULL; -static const U64 PRIME64_3 = 1609587929392839161ULL; -static const U64 PRIME64_4 = 9650029242287828579ULL; -static const U64 PRIME64_5 = 2870177450012600261ULL; - -static U64 XXH64_round(U64 acc, U64 input) -{ - acc += input * PRIME64_2; - acc = XXH_rotl64(acc, 31); - acc *= PRIME64_1; - return acc; -} - -static U64 XXH64_mergeRound(U64 acc, U64 val) -{ - val = XXH64_round(0, val); - acc ^= val; - acc = acc * PRIME64_1 + PRIME64_4; - return acc; -} - -FORCE_INLINE U64 XXH64_endian_align(const void* input, size_t len, U64 seed, XXH_endianess endian, XXH_alignment align) -{ - const BYTE* p = (const BYTE*)input; - const BYTE* bEnd = p + len; - U64 h64; -#define XXH_get64bits(p) XXH_readLE64_align(p, endian, align) - -#ifdef XXH_ACCEPT_NULL_INPUT_POINTER - if (p==NULL) { - len=0; - bEnd=p=(const BYTE*)(size_t)32; - } -#endif - - if (len>=32) { - const BYTE* const limit = bEnd - 32; - U64 v1 = seed + PRIME64_1 + PRIME64_2; - U64 v2 = seed + PRIME64_2; - U64 v3 = seed + 0; - U64 v4 = seed - PRIME64_1; - - do { - v1 = XXH64_round(v1, XXH_get64bits(p)); p+=8; - v2 = XXH64_round(v2, XXH_get64bits(p)); p+=8; - v3 = XXH64_round(v3, XXH_get64bits(p)); p+=8; - v4 = XXH64_round(v4, XXH_get64bits(p)); p+=8; - } while (p<=limit); - - h64 = XXH_rotl64(v1, 1) + XXH_rotl64(v2, 7) + XXH_rotl64(v3, 12) + XXH_rotl64(v4, 18); - h64 = XXH64_mergeRound(h64, v1); - h64 = XXH64_mergeRound(h64, v2); - h64 = XXH64_mergeRound(h64, v3); - h64 = XXH64_mergeRound(h64, v4); - - } else { - h64 = seed + PRIME64_5; - } - - h64 += (U64) len; - - while (p+8<=bEnd) { - U64 const k1 = XXH64_round(0, XXH_get64bits(p)); - h64 ^= k1; - h64 = XXH_rotl64(h64,27) * PRIME64_1 + PRIME64_4; - p+=8; - } - - if (p+4<=bEnd) { - h64 ^= (U64)(XXH_get32bits(p)) * PRIME64_1; - h64 = XXH_rotl64(h64, 23) * PRIME64_2 + PRIME64_3; - p+=4; - } - - while (p> 33; - h64 *= PRIME64_2; - h64 ^= h64 >> 29; - h64 *= PRIME64_3; - h64 ^= h64 >> 32; - - return h64; -} - - -XXH_PUBLIC_API unsigned long long XXH64 (const void* input, size_t len, unsigned long long seed) -{ -#if 0 - /* Simple version, good for code maintenance, but unfortunately slow for small inputs */ - XXH64_state_t state; - XXH64_reset(&state, seed); - XXH64_update(&state, input, len); - return XXH64_digest(&state); -#else - XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; - - if (XXH_FORCE_ALIGN_CHECK) { - if ((((size_t)input) & 7)==0) { /* Input is aligned, let's leverage the speed advantage */ - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH64_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned); - else - return XXH64_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned); - } } - - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH64_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned); - else - return XXH64_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned); -#endif -} - -/*====== Hash Streaming ======*/ - -XXH_PUBLIC_API XXH64_state_t* XXH64_createState(void) -{ - return (XXH64_state_t*)XXH_malloc(sizeof(XXH64_state_t)); -} -XXH_PUBLIC_API XXH_errorcode XXH64_freeState(XXH64_state_t* statePtr) -{ - XXH_free(statePtr); - return XXH_OK; -} - -XXH_PUBLIC_API void XXH64_copyState(XXH64_state_t* dstState, const XXH64_state_t* srcState) -{ - memcpy(dstState, srcState, sizeof(*dstState)); -} - -XXH_PUBLIC_API XXH_errorcode XXH64_reset(XXH64_state_t* statePtr, unsigned long long seed) -{ - XXH64_state_t state; /* using a local state to memcpy() in order to avoid strict-aliasing warnings */ - memset(&state, 0, sizeof(state)-8); /* do not write into reserved, for future removal */ - state.v1 = seed + PRIME64_1 + PRIME64_2; - state.v2 = seed + PRIME64_2; - state.v3 = seed + 0; - state.v4 = seed - PRIME64_1; - memcpy(statePtr, &state, sizeof(state)); - return XXH_OK; -} - -FORCE_INLINE XXH_errorcode XXH64_update_endian (XXH64_state_t* state, const void* input, size_t len, XXH_endianess endian) -{ - const BYTE* p = (const BYTE*)input; - const BYTE* const bEnd = p + len; - -#ifdef XXH_ACCEPT_NULL_INPUT_POINTER - if (input==NULL) return XXH_ERROR; -#endif - - state->total_len += len; - - if (state->memsize + len < 32) { /* fill in tmp buffer */ - XXH_memcpy(((BYTE*)state->mem64) + state->memsize, input, len); - state->memsize += (U32)len; - return XXH_OK; - } - - if (state->memsize) { /* tmp buffer is full */ - XXH_memcpy(((BYTE*)state->mem64) + state->memsize, input, 32-state->memsize); - state->v1 = XXH64_round(state->v1, XXH_readLE64(state->mem64+0, endian)); - state->v2 = XXH64_round(state->v2, XXH_readLE64(state->mem64+1, endian)); - state->v3 = XXH64_round(state->v3, XXH_readLE64(state->mem64+2, endian)); - state->v4 = XXH64_round(state->v4, XXH_readLE64(state->mem64+3, endian)); - p += 32-state->memsize; - state->memsize = 0; - } - - if (p+32 <= bEnd) { - const BYTE* const limit = bEnd - 32; - U64 v1 = state->v1; - U64 v2 = state->v2; - U64 v3 = state->v3; - U64 v4 = state->v4; - - do { - v1 = XXH64_round(v1, XXH_readLE64(p, endian)); p+=8; - v2 = XXH64_round(v2, XXH_readLE64(p, endian)); p+=8; - v3 = XXH64_round(v3, XXH_readLE64(p, endian)); p+=8; - v4 = XXH64_round(v4, XXH_readLE64(p, endian)); p+=8; - } while (p<=limit); - - state->v1 = v1; - state->v2 = v2; - state->v3 = v3; - state->v4 = v4; - } - - if (p < bEnd) { - XXH_memcpy(state->mem64, p, (size_t)(bEnd-p)); - state->memsize = (unsigned)(bEnd-p); - } - - return XXH_OK; -} - -XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* state_in, const void* input, size_t len) -{ - XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; - - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH64_update_endian(state_in, input, len, XXH_littleEndian); - else - return XXH64_update_endian(state_in, input, len, XXH_bigEndian); -} - -FORCE_INLINE U64 XXH64_digest_endian (const XXH64_state_t* state, XXH_endianess endian) -{ - const BYTE * p = (const BYTE*)state->mem64; - const BYTE* const bEnd = (const BYTE*)state->mem64 + state->memsize; - U64 h64; - - if (state->total_len >= 32) { - U64 const v1 = state->v1; - U64 const v2 = state->v2; - U64 const v3 = state->v3; - U64 const v4 = state->v4; - - h64 = XXH_rotl64(v1, 1) + XXH_rotl64(v2, 7) + XXH_rotl64(v3, 12) + XXH_rotl64(v4, 18); - h64 = XXH64_mergeRound(h64, v1); - h64 = XXH64_mergeRound(h64, v2); - h64 = XXH64_mergeRound(h64, v3); - h64 = XXH64_mergeRound(h64, v4); - } else { - h64 = state->v3 + PRIME64_5; - } - - h64 += (U64) state->total_len; - - while (p+8<=bEnd) { - U64 const k1 = XXH64_round(0, XXH_readLE64(p, endian)); - h64 ^= k1; - h64 = XXH_rotl64(h64,27) * PRIME64_1 + PRIME64_4; - p+=8; - } - - if (p+4<=bEnd) { - h64 ^= (U64)(XXH_readLE32(p, endian)) * PRIME64_1; - h64 = XXH_rotl64(h64, 23) * PRIME64_2 + PRIME64_3; - p+=4; - } - - while (p> 33; - h64 *= PRIME64_2; - h64 ^= h64 >> 29; - h64 *= PRIME64_3; - h64 ^= h64 >> 32; - - return h64; -} - -XXH_PUBLIC_API unsigned long long XXH64_digest (const XXH64_state_t* state_in) -{ - XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; - - if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) - return XXH64_digest_endian(state_in, XXH_littleEndian); - else - return XXH64_digest_endian(state_in, XXH_bigEndian); -} - - -/*====== Canonical representation ======*/ - -XXH_PUBLIC_API void XXH64_canonicalFromHash(XXH64_canonical_t* dst, XXH64_hash_t hash) -{ - XXH_STATIC_ASSERT(sizeof(XXH64_canonical_t) == sizeof(XXH64_hash_t)); - if (XXH_CPU_LITTLE_ENDIAN) hash = XXH_swap64(hash); - memcpy(dst, &hash, sizeof(*dst)); -} - -XXH_PUBLIC_API XXH64_hash_t XXH64_hashFromCanonical(const XXH64_canonical_t* src) -{ - return XXH_readBE64(src); -} - -#endif /* XXH_NO_LONG_LONG */ diff --git a/contrib/lizard/lib/xxhash/xxhash.h b/contrib/lizard/lib/xxhash/xxhash.h deleted file mode 100644 index 9d831e03b35..00000000000 --- a/contrib/lizard/lib/xxhash/xxhash.h +++ /dev/null @@ -1,293 +0,0 @@ -/* - xxHash - Extremely Fast Hash algorithm - Header File - Copyright (C) 2012-2016, Yann Collet. - - BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are - met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - You can contact the author at : - - xxHash source repository : https://github.com/Cyan4973/xxHash -*/ - -/* Notice extracted from xxHash homepage : - -xxHash is an extremely fast Hash algorithm, running at RAM speed limits. -It also successfully passes all tests from the SMHasher suite. - -Comparison (single thread, Windows Seven 32 bits, using SMHasher on a Core 2 Duo @3GHz) - -Name Speed Q.Score Author -xxHash 5.4 GB/s 10 -CrapWow 3.2 GB/s 2 Andrew -MumurHash 3a 2.7 GB/s 10 Austin Appleby -SpookyHash 2.0 GB/s 10 Bob Jenkins -SBox 1.4 GB/s 9 Bret Mulvey -Lookup3 1.2 GB/s 9 Bob Jenkins -SuperFastHash 1.2 GB/s 1 Paul Hsieh -CityHash64 1.05 GB/s 10 Pike & Alakuijala -FNV 0.55 GB/s 5 Fowler, Noll, Vo -CRC32 0.43 GB/s 9 -MD5-32 0.33 GB/s 10 Ronald L. Rivest -SHA1-32 0.28 GB/s 10 - -Q.Score is a measure of quality of the hash function. -It depends on successfully passing SMHasher test set. -10 is a perfect score. - -A 64-bits version, named XXH64, is available since r35. -It offers much better speed, but for 64-bits applications only. -Name Speed on 64 bits Speed on 32 bits -XXH64 13.8 GB/s 1.9 GB/s -XXH32 6.8 GB/s 6.0 GB/s -*/ - -#ifndef XXHASH_H_5627135585666179 -#define XXHASH_H_5627135585666179 1 - -#if defined (__cplusplus) -extern "C" { -#endif - - -/* **************************** -* Definitions -******************************/ -#include /* size_t */ -typedef enum { XXH_OK=0, XXH_ERROR } XXH_errorcode; - - -/* **************************** -* API modifier -******************************/ -/** XXH_PRIVATE_API -* This is useful to include xxhash functions in `static` mode -* in order to inline them, and remove their symbol from the public list. -* Methodology : -* #define XXH_PRIVATE_API -* #include "xxhash.h" -* `xxhash.c` is automatically included. -* It's not useful to compile and link it as a separate module. -*/ -#ifdef XXH_PRIVATE_API -# ifndef XXH_STATIC_LINKING_ONLY -# define XXH_STATIC_LINKING_ONLY -# endif -# if defined(__GNUC__) -# define XXH_PUBLIC_API static __inline __attribute__((unused)) -# elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) -# define XXH_PUBLIC_API static inline -# elif defined(_MSC_VER) -# define XXH_PUBLIC_API static __inline -# else -# define XXH_PUBLIC_API static /* this version may generate warnings for unused static functions; disable the relevant warning */ -# endif -#else -# define XXH_PUBLIC_API /* do nothing */ -#endif /* XXH_PRIVATE_API */ - -/*!XXH_NAMESPACE, aka Namespace Emulation : - -If you want to include _and expose_ xxHash functions from within your own library, -but also want to avoid symbol collisions with other libraries which may also include xxHash, - -you can use XXH_NAMESPACE, to automatically prefix any public symbol from xxhash library -with the value of XXH_NAMESPACE (therefore, avoid NULL and numeric values). - -Note that no change is required within the calling program as long as it includes `xxhash.h` : -regular symbol name will be automatically translated by this header. -*/ -#ifdef XXH_NAMESPACE -# define XXH_CAT(A,B) A##B -# define XXH_NAME2(A,B) XXH_CAT(A,B) -# define XXH_versionNumber XXH_NAME2(XXH_NAMESPACE, XXH_versionNumber) -# define XXH32 XXH_NAME2(XXH_NAMESPACE, XXH32) -# define XXH32_createState XXH_NAME2(XXH_NAMESPACE, XXH32_createState) -# define XXH32_freeState XXH_NAME2(XXH_NAMESPACE, XXH32_freeState) -# define XXH32_reset XXH_NAME2(XXH_NAMESPACE, XXH32_reset) -# define XXH32_update XXH_NAME2(XXH_NAMESPACE, XXH32_update) -# define XXH32_digest XXH_NAME2(XXH_NAMESPACE, XXH32_digest) -# define XXH32_copyState XXH_NAME2(XXH_NAMESPACE, XXH32_copyState) -# define XXH32_canonicalFromHash XXH_NAME2(XXH_NAMESPACE, XXH32_canonicalFromHash) -# define XXH32_hashFromCanonical XXH_NAME2(XXH_NAMESPACE, XXH32_hashFromCanonical) -# define XXH64 XXH_NAME2(XXH_NAMESPACE, XXH64) -# define XXH64_createState XXH_NAME2(XXH_NAMESPACE, XXH64_createState) -# define XXH64_freeState XXH_NAME2(XXH_NAMESPACE, XXH64_freeState) -# define XXH64_reset XXH_NAME2(XXH_NAMESPACE, XXH64_reset) -# define XXH64_update XXH_NAME2(XXH_NAMESPACE, XXH64_update) -# define XXH64_digest XXH_NAME2(XXH_NAMESPACE, XXH64_digest) -# define XXH64_copyState XXH_NAME2(XXH_NAMESPACE, XXH64_copyState) -# define XXH64_canonicalFromHash XXH_NAME2(XXH_NAMESPACE, XXH64_canonicalFromHash) -# define XXH64_hashFromCanonical XXH_NAME2(XXH_NAMESPACE, XXH64_hashFromCanonical) -#endif - - -/* ************************************* -* Version -***************************************/ -#define XXH_VERSION_MAJOR 0 -#define XXH_VERSION_MINOR 6 -#define XXH_VERSION_RELEASE 2 -#define XXH_VERSION_NUMBER (XXH_VERSION_MAJOR *100*100 + XXH_VERSION_MINOR *100 + XXH_VERSION_RELEASE) -XXH_PUBLIC_API unsigned XXH_versionNumber (void); - - -/*-********************************************************************** -* 32-bits hash -************************************************************************/ -typedef unsigned int XXH32_hash_t; - -/*! XXH32() : - Calculate the 32-bits hash of sequence "length" bytes stored at memory address "input". - The memory between input & input+length must be valid (allocated and read-accessible). - "seed" can be used to alter the result predictably. - Speed on Core 2 Duo @ 3 GHz (single thread, SMHasher benchmark) : 5.4 GB/s */ -XXH_PUBLIC_API XXH32_hash_t XXH32 (const void* input, size_t length, unsigned int seed); - -/*====== Streaming ======*/ -typedef struct XXH32_state_s XXH32_state_t; /* incomplete type */ -XXH_PUBLIC_API XXH32_state_t* XXH32_createState(void); -XXH_PUBLIC_API XXH_errorcode XXH32_freeState(XXH32_state_t* statePtr); -XXH_PUBLIC_API void XXH32_copyState(XXH32_state_t* dst_state, const XXH32_state_t* src_state); - -XXH_PUBLIC_API XXH_errorcode XXH32_reset (XXH32_state_t* statePtr, unsigned int seed); -XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* statePtr, const void* input, size_t length); -XXH_PUBLIC_API XXH32_hash_t XXH32_digest (const XXH32_state_t* statePtr); - -/* -These functions generate the xxHash of an input provided in multiple segments. -Note that, for small input, they are slower than single-call functions, due to state management. -For small input, prefer `XXH32()` and `XXH64()` . - -XXH state must first be allocated, using XXH*_createState() . - -Start a new hash by initializing state with a seed, using XXH*_reset(). - -Then, feed the hash state by calling XXH*_update() as many times as necessary. -Obviously, input must be allocated and read accessible. -The function returns an error code, with 0 meaning OK, and any other value meaning there is an error. - -Finally, a hash value can be produced anytime, by using XXH*_digest(). -This function returns the nn-bits hash as an int or long long. - -It's still possible to continue inserting input into the hash state after a digest, -and generate some new hashes later on, by calling again XXH*_digest(). - -When done, free XXH state space if it was allocated dynamically. -*/ - -/*====== Canonical representation ======*/ - -typedef struct { unsigned char digest[4]; } XXH32_canonical_t; -XXH_PUBLIC_API void XXH32_canonicalFromHash(XXH32_canonical_t* dst, XXH32_hash_t hash); -XXH_PUBLIC_API XXH32_hash_t XXH32_hashFromCanonical(const XXH32_canonical_t* src); - -/* Default result type for XXH functions are primitive unsigned 32 and 64 bits. -* The canonical representation uses human-readable write convention, aka big-endian (large digits first). -* These functions allow transformation of hash result into and from its canonical format. -* This way, hash values can be written into a file / memory, and remain comparable on different systems and programs. -*/ - - -#ifndef XXH_NO_LONG_LONG -/*-********************************************************************** -* 64-bits hash -************************************************************************/ -typedef unsigned long long XXH64_hash_t; - -/*! XXH64() : - Calculate the 64-bits hash of sequence of length "len" stored at memory address "input". - "seed" can be used to alter the result predictably. - This function runs faster on 64-bits systems, but slower on 32-bits systems (see benchmark). -*/ -XXH_PUBLIC_API XXH64_hash_t XXH64 (const void* input, size_t length, unsigned long long seed); - -/*====== Streaming ======*/ -typedef struct XXH64_state_s XXH64_state_t; /* incomplete type */ -XXH_PUBLIC_API XXH64_state_t* XXH64_createState(void); -XXH_PUBLIC_API XXH_errorcode XXH64_freeState(XXH64_state_t* statePtr); -XXH_PUBLIC_API void XXH64_copyState(XXH64_state_t* dst_state, const XXH64_state_t* src_state); - -XXH_PUBLIC_API XXH_errorcode XXH64_reset (XXH64_state_t* statePtr, unsigned long long seed); -XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* statePtr, const void* input, size_t length); -XXH_PUBLIC_API XXH64_hash_t XXH64_digest (const XXH64_state_t* statePtr); - -/*====== Canonical representation ======*/ -typedef struct { unsigned char digest[8]; } XXH64_canonical_t; -XXH_PUBLIC_API void XXH64_canonicalFromHash(XXH64_canonical_t* dst, XXH64_hash_t hash); -XXH_PUBLIC_API XXH64_hash_t XXH64_hashFromCanonical(const XXH64_canonical_t* src); -#endif /* XXH_NO_LONG_LONG */ - - -#ifdef XXH_STATIC_LINKING_ONLY - -/* ================================================================================================ - This section contains definitions which are not guaranteed to remain stable. - They may change in future versions, becoming incompatible with a different version of the library. - They shall only be used with static linking. - Never use these definitions in association with dynamic linking ! -=================================================================================================== */ - -/* These definitions are only meant to make possible - static allocation of XXH state, on stack or in a struct for example. - Never use members directly. */ - -struct XXH32_state_s { - unsigned total_len_32; - unsigned large_len; - unsigned v1; - unsigned v2; - unsigned v3; - unsigned v4; - unsigned mem32[4]; /* buffer defined as U32 for alignment */ - unsigned memsize; - unsigned reserved; /* never read nor write, will be removed in a future version */ -}; /* typedef'd to XXH32_state_t */ - -#ifndef XXH_NO_LONG_LONG /* remove 64-bits support */ -struct XXH64_state_s { - unsigned long long total_len; - unsigned long long v1; - unsigned long long v2; - unsigned long long v3; - unsigned long long v4; - unsigned long long mem64[4]; /* buffer defined as U64 for alignment */ - unsigned memsize; - unsigned reserved[2]; /* never read nor write, will be removed in a future version */ -}; /* typedef'd to XXH64_state_t */ -#endif - -#ifdef XXH_PRIVATE_API -# include "xxhash.c" /* include xxhash function bodies as `static`, for inlining */ -#endif - -#endif /* XXH_STATIC_LINKING_ONLY */ - - -#if defined (__cplusplus) -} -#endif - -#endif /* XXHASH_H_5627135585666179 */ diff --git a/contrib/lizard/programs/.gitignore b/contrib/lizard/programs/.gitignore deleted file mode 100644 index c2bfcafa9ee..00000000000 --- a/contrib/lizard/programs/.gitignore +++ /dev/null @@ -1,15 +0,0 @@ -# local binary (Makefile) -lizard -lizardc32 -datagen -frametest -frametest32 -fullbench -fullbench32 -fuzzer -fuzzer32 -*.exe - -# tests files -tmp* -_* \ No newline at end of file diff --git a/contrib/lizard/programs/COPYING b/contrib/lizard/programs/COPYING deleted file mode 100644 index d159169d105..00000000000 --- a/contrib/lizard/programs/COPYING +++ /dev/null @@ -1,339 +0,0 @@ - GNU GENERAL PUBLIC LICENSE - Version 2, June 1991 - - Copyright (C) 1989, 1991 Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The licenses for most software are designed to take away your -freedom to share and change it. By contrast, the GNU General Public -License is intended to guarantee your freedom to share and change free -software--to make sure the software is free for all its users. This -General Public License applies to most of the Free Software -Foundation's software and to any other program whose authors commit to -using it. (Some other Free Software Foundation software is covered by -the GNU Lesser General Public License instead.) You can apply it to -your programs, too. - - When we speak of free software, we are referring to freedom, not -price. Our General Public Licenses are designed to make sure that you -have the freedom to distribute copies of free software (and charge for -this service if you wish), that you receive source code or can get it -if you want it, that you can change the software or use pieces of it -in new free programs; and that you know you can do these things. - - To protect your rights, we need to make restrictions that forbid -anyone to deny you these rights or to ask you to surrender the rights. -These restrictions translate to certain responsibilities for you if you -distribute copies of the software, or if you modify it. - - For example, if you distribute copies of such a program, whether -gratis or for a fee, you must give the recipients all the rights that -you have. You must make sure that they, too, receive or can get the -source code. And you must show them these terms so they know their -rights. - - We protect your rights with two steps: (1) copyright the software, and -(2) offer you this license which gives you legal permission to copy, -distribute and/or modify the software. - - Also, for each author's protection and ours, we want to make certain -that everyone understands that there is no warranty for this free -software. If the software is modified by someone else and passed on, we -want its recipients to know that what they have is not the original, so -that any problems introduced by others will not reflect on the original -authors' reputations. - - Finally, any free program is threatened constantly by software -patents. We wish to avoid the danger that redistributors of a free -program will individually obtain patent licenses, in effect making the -program proprietary. To prevent this, we have made it clear that any -patent must be licensed for everyone's free use or not licensed at all. - - The precise terms and conditions for copying, distribution and -modification follow. - - GNU GENERAL PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License applies to any program or other work which contains -a notice placed by the copyright holder saying it may be distributed -under the terms of this General Public License. The "Program", below, -refers to any such program or work, and a "work based on the Program" -means either the Program or any derivative work under copyright law: -that is to say, a work containing the Program or a portion of it, -either verbatim or with modifications and/or translated into another -language. (Hereinafter, translation is included without limitation in -the term "modification".) Each licensee is addressed as "you". - -Activities other than copying, distribution and modification are not -covered by this License; they are outside its scope. The act of -running the Program is not restricted, and the output from the Program -is covered only if its contents constitute a work based on the -Program (independent of having been made by running the Program). -Whether that is true depends on what the Program does. - - 1. You may copy and distribute verbatim copies of the Program's -source code as you receive it, in any medium, provided that you -conspicuously and appropriately publish on each copy an appropriate -copyright notice and disclaimer of warranty; keep intact all the -notices that refer to this License and to the absence of any warranty; -and give any other recipients of the Program a copy of this License -along with the Program. - -You may charge a fee for the physical act of transferring a copy, and -you may at your option offer warranty protection in exchange for a fee. - - 2. You may modify your copy or copies of the Program or any portion -of it, thus forming a work based on the Program, and copy and -distribute such modifications or work under the terms of Section 1 -above, provided that you also meet all of these conditions: - - a) You must cause the modified files to carry prominent notices - stating that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in - whole or in part contains or is derived from the Program or any - part thereof, to be licensed as a whole at no charge to all third - parties under the terms of this License. - - c) If the modified program normally reads commands interactively - when run, you must cause it, when started running for such - interactive use in the most ordinary way, to print or display an - announcement including an appropriate copyright notice and a - notice that there is no warranty (or else, saying that you provide - a warranty) and that users may redistribute the program under - these conditions, and telling the user how to view a copy of this - License. (Exception: if the Program itself is interactive but - does not normally print such an announcement, your work based on - the Program is not required to print an announcement.) - -These requirements apply to the modified work as a whole. If -identifiable sections of that work are not derived from the Program, -and can be reasonably considered independent and separate works in -themselves, then this License, and its terms, do not apply to those -sections when you distribute them as separate works. But when you -distribute the same sections as part of a whole which is a work based -on the Program, the distribution of the whole must be on the terms of -this License, whose permissions for other licensees extend to the -entire whole, and thus to each and every part regardless of who wrote it. - -Thus, it is not the intent of this section to claim rights or contest -your rights to work written entirely by you; rather, the intent is to -exercise the right to control the distribution of derivative or -collective works based on the Program. - -In addition, mere aggregation of another work not based on the Program -with the Program (or with a work based on the Program) on a volume of -a storage or distribution medium does not bring the other work under -the scope of this License. - - 3. You may copy and distribute the Program (or a work based on it, -under Section 2) in object code or executable form under the terms of -Sections 1 and 2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable - source code, which must be distributed under the terms of Sections - 1 and 2 above on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three - years, to give any third party, for a charge no more than your - cost of physically performing source distribution, a complete - machine-readable copy of the corresponding source code, to be - distributed under the terms of Sections 1 and 2 above on a medium - customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer - to distribute corresponding source code. (This alternative is - allowed only for noncommercial distribution and only if you - received the program in object code or executable form with such - an offer, in accord with Subsection b above.) - -The source code for a work means the preferred form of the work for -making modifications to it. For an executable work, complete source -code means all the source code for all modules it contains, plus any -associated interface definition files, plus the scripts used to -control compilation and installation of the executable. However, as a -special exception, the source code distributed need not include -anything that is normally distributed (in either source or binary -form) with the major components (compiler, kernel, and so on) of the -operating system on which the executable runs, unless that component -itself accompanies the executable. - -If distribution of executable or object code is made by offering -access to copy from a designated place, then offering equivalent -access to copy the source code from the same place counts as -distribution of the source code, even though third parties are not -compelled to copy the source along with the object code. - - 4. You may not copy, modify, sublicense, or distribute the Program -except as expressly provided under this License. Any attempt -otherwise to copy, modify, sublicense or distribute the Program is -void, and will automatically terminate your rights under this License. -However, parties who have received copies, or rights, from you under -this License will not have their licenses terminated so long as such -parties remain in full compliance. - - 5. You are not required to accept this License, since you have not -signed it. However, nothing else grants you permission to modify or -distribute the Program or its derivative works. These actions are -prohibited by law if you do not accept this License. Therefore, by -modifying or distributing the Program (or any work based on the -Program), you indicate your acceptance of this License to do so, and -all its terms and conditions for copying, distributing or modifying -the Program or works based on it. - - 6. Each time you redistribute the Program (or any work based on the -Program), the recipient automatically receives a license from the -original licensor to copy, distribute or modify the Program subject to -these terms and conditions. You may not impose any further -restrictions on the recipients' exercise of the rights granted herein. -You are not responsible for enforcing compliance by third parties to -this License. - - 7. If, as a consequence of a court judgment or allegation of patent -infringement or for any other reason (not limited to patent issues), -conditions are imposed on you (whether by court order, agreement or -otherwise) that contradict the conditions of this License, they do not -excuse you from the conditions of this License. If you cannot -distribute so as to satisfy simultaneously your obligations under this -License and any other pertinent obligations, then as a consequence you -may not distribute the Program at all. For example, if a patent -license would not permit royalty-free redistribution of the Program by -all those who receive copies directly or indirectly through you, then -the only way you could satisfy both it and this License would be to -refrain entirely from distribution of the Program. - -If any portion of this section is held invalid or unenforceable under -any particular circumstance, the balance of the section is intended to -apply and the section as a whole is intended to apply in other -circumstances. - -It is not the purpose of this section to induce you to infringe any -patents or other property right claims or to contest validity of any -such claims; this section has the sole purpose of protecting the -integrity of the free software distribution system, which is -implemented by public license practices. Many people have made -generous contributions to the wide range of software distributed -through that system in reliance on consistent application of that -system; it is up to the author/donor to decide if he or she is willing -to distribute software through any other system and a licensee cannot -impose that choice. - -This section is intended to make thoroughly clear what is believed to -be a consequence of the rest of this License. - - 8. If the distribution and/or use of the Program is restricted in -certain countries either by patents or by copyrighted interfaces, the -original copyright holder who places the Program under this License -may add an explicit geographical distribution limitation excluding -those countries, so that distribution is permitted only in or among -countries not thus excluded. In such case, this License incorporates -the limitation as if written in the body of this License. - - 9. The Free Software Foundation may publish revised and/or new versions -of the General Public License from time to time. Such new versions will -be similar in spirit to the present version, but may differ in detail to -address new problems or concerns. - -Each version is given a distinguishing version number. If the Program -specifies a version number of this License which applies to it and "any -later version", you have the option of following the terms and conditions -either of that version or of any later version published by the Free -Software Foundation. If the Program does not specify a version number of -this License, you may choose any version ever published by the Free Software -Foundation. - - 10. If you wish to incorporate parts of the Program into other free -programs whose distribution conditions are different, write to the author -to ask for permission. For software which is copyrighted by the Free -Software Foundation, write to the Free Software Foundation; we sometimes -make exceptions for this. Our decision will be guided by the two goals -of preserving the free status of all derivatives of our free software and -of promoting the sharing and reuse of software generally. - - NO WARRANTY - - 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY -FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN -OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES -PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED -OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF -MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS -TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE -PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, -REPAIR OR CORRECTION. - - 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING -WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR -REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, -INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING -OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED -TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY -YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER -PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE -POSSIBILITY OF SUCH DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest -possible use to the public, the best way to achieve this is to make it -free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest -to attach them to the start of each source file to most effectively -convey the exclusion of warranty; and each file should have at least -the "copyright" line and a pointer to where the full notice is found. - - - Copyright (C) - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - -Also add information on how to contact you by electronic and paper mail. - -If the program is interactive, make it output a short notice like this -when it starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author - Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. - This is free software, and you are welcome to redistribute it - under certain conditions; type `show c' for details. - -The hypothetical commands `show w' and `show c' should show the appropriate -parts of the General Public License. Of course, the commands you use may -be called something other than `show w' and `show c'; they could even be -mouse-clicks or menu items--whatever suits your program. - -You should also get your employer (if you work as a programmer) or your -school, if any, to sign a "copyright disclaimer" for the program, if -necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the program - `Gnomovision' (which makes passes at compilers) written by James Hacker. - - , 1 April 1989 - Ty Coon, President of Vice - -This General Public License does not permit incorporating your program into -proprietary programs. If your program is a subroutine library, you may -consider it more useful to permit linking proprietary applications with the -library. If this is what you want to do, use the GNU Lesser General -Public License instead of this License. diff --git a/contrib/lizard/programs/README.md b/contrib/lizard/programs/README.md deleted file mode 100644 index ee116c116c9..00000000000 --- a/contrib/lizard/programs/README.md +++ /dev/null @@ -1,73 +0,0 @@ -Command Line Interface for LZ4 library -============================================ - -Command Line Interface (CLI) can be created using the `make` command without any additional parameters. -There are also multiple targets that create different variations of CLI: -- `lizard` : default CLI, with a command line syntax close to gzip -- `lizardc32` : Same as `lizard`, but forced to compile in 32-bits mode - - -#### Aggregation of parameters -CLI supports aggregation of parameters i.e. `-b1`, `-e18`, and `-i1` can be joined into `-b1e18i1`. - - - -#### Benchmark in Command Line Interface -CLI includes in-memory compression benchmark module for lizard. -The benchmark is conducted using a given filename. -The file is read into memory. -It makes benchmark more precise as it eliminates I/O overhead. - -The benchmark measures ratio, compressed size, compression and decompression speed. -One can select compression levels starting from `-b` and ending with `-e`. -The `-i` parameter selects a number of seconds used for each of tested levels. - - - -#### Usage of Command Line Interface -The full list of commands can be obtained with `-h` or `-H` parameter: -``` -Usage : - lizard [arg] [input] [output] - -input : a filename - with no FILE, or when FILE is - or stdin, read standard input -Arguments : - -10...-19 : compression method fastLZ4 = 16-bit bytewise codewords - higher number == more compression but slower - -20...-29 : compression method LIZv1 = 24-bit bytewise codewords - -30...-39 : compression method fastLZ4 + Huffman - -40...-49 : compression method LIZv1 + Huffman - -d : decompression (default for .liz extension) - -z : force compression - -f : overwrite output without prompting ---rm : remove source file(s) after successful de/compression - -h/-H : display help/long help and exit - -Advanced arguments : - -V : display Version number and exit - -v : verbose mode - -q : suppress warnings; specify twice to suppress errors too - -c : force write to standard output, even if it is the console - -t : test compressed file integrity - -m : multiple input files (implies automatic output filenames) - -r : operate recursively on directories (sets also -m) - -l : compress using Legacy format (Linux kernel compression) - -B# : Block size [1-7] = 128KB, 256KB, 1MB, 4MB, 16MB, 64MB, 256MB (default : 4) - -BD : Block dependency (improve compression ratio) ---no-frame-crc : disable stream checksum (default:enabled) ---content-size : compressed frame includes original size (default:not present) ---[no-]sparse : sparse mode (default:enabled on file, disabled on stdout) -Benchmark arguments : - -b# : benchmark file(s), using # compression level (default : 1) - -e# : test all compression levels from -bX to # (default : 1) - -i# : minimum evaluation time in seconds (default : 3s) - -B# : cut file into independent blocks of size # bytes [32+] - or predefined block size [1-7] (default: 4) -``` - -#### License - -All files in this directory are licensed under GPL-v2. -See [COPYING](COPYING) for details. -The text of the license is also included at the top of each source file. diff --git a/contrib/lizard/programs/bench.c b/contrib/lizard/programs/bench.c deleted file mode 100644 index 3fe56d1c060..00000000000 --- a/contrib/lizard/programs/bench.c +++ /dev/null @@ -1,502 +0,0 @@ -/* - bench.c - Demo program to benchmark open-source compression algorithms - Copyright (C) Yann Collet 2012-2016 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - - -/*-************************************ -* Compiler options -**************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -#endif -#if defined(__MINGW32__) && !defined(_POSIX_SOURCE) -# define _POSIX_SOURCE 1 /* disable %llu warnings with MinGW on Windows */ -#endif - - -/* ************************************* -* Includes -***************************************/ -#include "platform.h" /* Compiler options */ -#include /* malloc, free */ -#include /* memset */ -#include /* fprintf, fopen, ftello64 */ -#include /* clock_t, clock, CLOCKS_PER_SEC */ - -#include "datagen.h" /* RDG_genBuffer */ -#include "xxhash/xxhash.h" -#include "lizard_common.h" -#include "lizard_decompress.h" -#define UTIL_WITHOUT_BASIC_TYPES -#include "util.h" /* UTIL_GetFileSize, UTIL_sleep */ - -#define Lizard_isError(errcode) (errcode==0) - - -/* ************************************* -* Constants -***************************************/ -#ifndef LIZARD_GIT_COMMIT_STRING -# define LIZARD_GIT_COMMIT_STRING "" -#else -# define LIZARD_GIT_COMMIT_STRING LIZARD_EXPAND_AND_QUOTE(Lizard_GIT_COMMIT) -#endif - -#define NBSECONDS 3 -#define TIMELOOP_MICROSEC 1*1000000ULL /* 1 second */ -#define ACTIVEPERIOD_MICROSEC 70*1000000ULL /* 70 seconds */ -#define COOLPERIOD_SEC 10 -#define DECOMP_MULT 2 /* test decompression DECOMP_MULT times longer than compression */ - -#define KB *(1 <<10) -#define MB *(1 <<20) -#define GB *(1U<<30) - -static const size_t maxMemory = (sizeof(size_t)==4) ? (2 GB - 64 MB) : (size_t)(1ULL << ((sizeof(size_t)*8)-31)); - -static U32 g_compressibilityDefault = 50; - - -/* ************************************* -* console display -***************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define DISPLAYLEVEL(l, ...) if (g_displayLevel>=l) { DISPLAY(__VA_ARGS__); } -static U32 g_displayLevel = 2; /* 0 : no display; 1: errors; 2 : + result + interaction + warnings; 3 : + progression; 4 : + information */ - -#define DISPLAYUPDATE(l, ...) if (g_displayLevel>=l) { \ - if ((clock() - g_time > refreshRate) || (g_displayLevel>=4)) \ - { g_time = clock(); DISPLAY(__VA_ARGS__); \ - if (g_displayLevel>=4) fflush(stdout); } } -static const clock_t refreshRate = CLOCKS_PER_SEC * 15 / 100; -static clock_t g_time = 0; - - -/* ************************************* -* Exceptions -***************************************/ -#ifndef DEBUG -# define DEBUG 0 -#endif -#define DEBUGOUTPUT(...) if (DEBUG) DISPLAY(__VA_ARGS__); -#define EXM_THROW(error, ...) \ -{ \ - DEBUGOUTPUT("Error defined at %s, line %i : \n", __FILE__, __LINE__); \ - DISPLAYLEVEL(1, "Error %i : ", error); \ - DISPLAYLEVEL(1, __VA_ARGS__); \ - DISPLAYLEVEL(1, "\n"); \ - exit(error); \ -} - - -/* ************************************* -* Benchmark Parameters -***************************************/ -static U32 g_nbSeconds = NBSECONDS; -static size_t g_blockSize = 0; -int g_additionalParam = 0; - -void BMK_setNotificationLevel(unsigned level) { g_displayLevel=level; } - -void BMK_setAdditionalParam(int additionalParam) { g_additionalParam=additionalParam; } - -void BMK_SetNbSeconds(unsigned nbSeconds) -{ - g_nbSeconds = nbSeconds; - DISPLAYLEVEL(3, "- test >= %u seconds per compression / decompression -\n", g_nbSeconds); -} - -void BMK_SetBlockSize(size_t blockSize) -{ - g_blockSize = blockSize; -} - - -/* ******************************************************** -* Bench functions -**********************************************************/ -typedef struct { - const char* srcPtr; - size_t srcSize; - char* cPtr; - size_t cRoom; - size_t cSize; - char* resPtr; - size_t resSize; -} blockParam_t; - - - -static int BMK_benchMem(const void* srcBuffer, size_t srcSize, - const char* displayName, int cLevel, - const size_t* fileSizes, U32 nbFiles) -{ - size_t const blockSize = (g_blockSize>=32 ? g_blockSize : srcSize) + (!srcSize) /* avoid div by 0 */ ; - U32 const maxNbBlocks = (U32) ((srcSize + (blockSize-1)) / blockSize) + nbFiles; - blockParam_t* const blockTable = (blockParam_t*) malloc(maxNbBlocks * sizeof(blockParam_t)); - size_t const maxCompressedSize = Lizard_compressBound((int)srcSize) + (maxNbBlocks * 1024); /* add some room for safety */ - void* const compressedBuffer = malloc(maxCompressedSize); - void* const resultBuffer = malloc(srcSize); - U32 nbBlocks; - UTIL_time_t ticksPerSecond; - - /* checks */ - if (!compressedBuffer || !resultBuffer || !blockTable) - EXM_THROW(31, "allocation error : not enough memory"); - - /* init */ - if (strlen(displayName)>17) displayName += strlen(displayName)-17; /* can only display 17 characters */ - UTIL_initTimer(&ticksPerSecond); - - /* Init blockTable data */ - { const char* srcPtr = (const char*)srcBuffer; - char* cPtr = (char*)compressedBuffer; - char* resPtr = (char*)resultBuffer; - U32 fileNb; - for (nbBlocks=0, fileNb=0; fileNb ACTIVEPERIOD_MICROSEC) { - DISPLAYLEVEL(2, "\rcooling down ... \r"); - UTIL_sleep(COOLPERIOD_SEC); - UTIL_getTime(&coolTime); - } - - /* Compression */ - DISPLAYLEVEL(2, "%2s-%-17.17s :%10u ->\r", marks[markNb], displayName, (U32)srcSize); - if (!cCompleted) memset(compressedBuffer, 0xE5, maxCompressedSize); /* warm up and erase result buffer */ - - UTIL_sleepMilli(1); /* give processor time to other processes */ - UTIL_waitForNextTick(ticksPerSecond); - UTIL_getTime(&clockStart); - - if (!cCompleted) { /* still some time to do compression tests */ - U32 nbLoops = 0; - do { - U32 blockNb; - for (blockNb=0; blockNbmaxTime; - } } - - cSize = 0; - { U32 blockNb; for (blockNb=0; blockNb%10u (%5.3f),%6.1f MB/s\r", - marks[markNb], displayName, (U32)srcSize, (U32)cSize, ratio, - (double)srcSize / fastestC ); - - (void)fastestD; (void)crcOrig; /* unused when decompression disabled */ -#if 1 - /* Decompression */ - if (!dCompleted) memset(resultBuffer, 0xD6, srcSize); /* warm result buffer */ - - UTIL_sleepMilli(1); /* give processor time to other processes */ - UTIL_waitForNextTick(ticksPerSecond); - UTIL_getTime(&clockStart); - - if (!dCompleted) { - U32 nbLoops = 0; - do { - U32 blockNb; - for (blockNb=0; blockNb(DECOMP_MULT*maxTime); - } } - - markNb = (markNb+1) % NB_MARKS; - DISPLAYLEVEL(2, "%2s-%-17.17s :%10u ->%10u (%5.3f),%6.1f MB/s ,%6.1f MB/s\r", - marks[markNb], displayName, (U32)srcSize, (U32)cSize, ratio, - (double)srcSize / fastestC, - (double)srcSize / fastestD ); - - /* CRC Checking */ - { U64 const crcCheck = XXH64(resultBuffer, srcSize, 0); - if (crcOrig!=crcCheck) { - size_t u; - DISPLAY("!!! WARNING !!! %14s : Invalid Checksum : %x != %x \n", displayName, (unsigned)crcOrig, (unsigned)crcCheck); - for (u=0; u u) break; - bacc += blockTable[segNb].srcSize; - } - pos = (U32)(u - bacc); - bNb = pos / (128 KB); - DISPLAY("(block %u, sub %u, pos %u) \n", segNb, bNb, pos); - break; - } - if (u==srcSize-1) { /* should never happen */ - DISPLAY("no difference detected\n"); - } } - break; - } } /* CRC Checking */ -#endif - } /* for (testNb = 1; testNb <= (g_nbSeconds + !g_nbSeconds); testNb++) */ - - if (g_displayLevel == 1) { - double cSpeed = (double)srcSize / fastestC; - double dSpeed = (double)srcSize / fastestD; - if (g_additionalParam) - DISPLAY("-%-3i%11i (%5.3f) %6.2f MB/s %6.1f MB/s %s (param=%d)\n", cLevel, (int)cSize, ratio, cSpeed, dSpeed, displayName, g_additionalParam); - else - DISPLAY("-%-3i%11i (%5.3f) %6.2f MB/s %6.1f MB/s %s\n", cLevel, (int)cSize, ratio, cSpeed, dSpeed, displayName); - } - DISPLAYLEVEL(2, "%2i#\n", cLevel); - } /* Bench */ - - /* clean up */ - free(blockTable); - free(compressedBuffer); - free(resultBuffer); - return 0; -} - - -static size_t BMK_findMaxMem(U64 requiredMem) -{ - size_t step = 64 MB; - BYTE* testmem=NULL; - - requiredMem = (((requiredMem >> 26) + 1) << 26); - requiredMem += 2*step; - if (requiredMem > maxMemory) requiredMem = maxMemory; - - while (!testmem) { - if (requiredMem > step) requiredMem -= step; - else requiredMem >>= 1; - testmem = (BYTE*) malloc ((size_t)requiredMem); - } - free (testmem); - - /* keep some space available */ - if (requiredMem > step) requiredMem -= step; - else requiredMem >>= 1; - - return (size_t)requiredMem; -} - - -static void BMK_benchCLevel(void* srcBuffer, size_t benchedSize, - const char* displayName, int cLevel, int cLevelLast, - const size_t* fileSizes, unsigned nbFiles) -{ - int l; - - const char* pch = strrchr(displayName, '\\'); /* Windows */ - if (!pch) pch = strrchr(displayName, '/'); /* Linux */ - if (pch) displayName = pch+1; - - SET_REALTIME_PRIORITY; - - if (g_displayLevel == 1 && !g_additionalParam) - DISPLAY("bench %s %s: input %u bytes, %u seconds, %u KB blocks\n", LIZARD_VERSION_STRING, LIZARD_GIT_COMMIT_STRING, (U32)benchedSize, g_nbSeconds, (U32)(g_blockSize>>10)); - - if (cLevelLast < cLevel) cLevelLast = cLevel; - - for (l=cLevel; l <= cLevelLast; l++) { - BMK_benchMem(srcBuffer, benchedSize, - displayName, l, - fileSizes, nbFiles); - } -} - - -/*! BMK_loadFiles() : - Loads `buffer` with content of files listed within `fileNamesTable`. - At most, fills `buffer` entirely */ -static void BMK_loadFiles(void* buffer, size_t bufferSize, - size_t* fileSizes, - const char** fileNamesTable, unsigned nbFiles) -{ - size_t pos = 0, totalSize = 0; - unsigned n; - for (n=0; n bufferSize-pos) fileSize = bufferSize-pos, nbFiles=n; /* buffer too small - stop after this file */ - { size_t const readSize = fread(((char*)buffer)+pos, 1, (size_t)fileSize, f); - if (readSize != (size_t)fileSize) EXM_THROW(11, "could not read %s", fileNamesTable[n]); - DISPLAYLEVEL(4, "readSize=%llu\n", (unsigned long long)readSize); - pos += readSize; } - fileSizes[n] = (size_t)fileSize; - totalSize += (size_t)fileSize; - fclose(f); - } - - if (totalSize == 0) EXM_THROW(12, "no data to bench"); -} - -static void BMK_benchFileTable(const char** fileNamesTable, unsigned nbFiles, - int cLevel, int cLevelLast) -{ - void* srcBuffer; - size_t benchedSize; - size_t* fileSizes = (size_t*)malloc(nbFiles * sizeof(size_t)); - U64 const totalSizeToLoad = UTIL_getTotalFileSize(fileNamesTable, nbFiles); - char mfName[20] = {0}; - - if (!fileSizes) EXM_THROW(12, "not enough memory for fileSizes"); - - /* Memory allocation & restrictions */ - benchedSize = BMK_findMaxMem(totalSizeToLoad * 3) / 3; - if (benchedSize==0) EXM_THROW(12, "not enough memory"); - if ((U64)benchedSize > totalSizeToLoad) benchedSize = (size_t)totalSizeToLoad; - if (benchedSize > LIZARD_MAX_INPUT_SIZE) { - benchedSize = LIZARD_MAX_INPUT_SIZE; - DISPLAY("File(s) bigger than Lizard's max input size; testing %u MB only...\n", (U32)(benchedSize >> 20)); - } - else - if (benchedSize < totalSizeToLoad) { - DISPLAY("Not enough memory; testing %u MB only...\n", (U32)(benchedSize >> 20)); - } - srcBuffer = malloc(benchedSize + !benchedSize); /* avoid alloc of zero */ - if (!srcBuffer) EXM_THROW(12, "not enough memory"); - - /* Load input buffer */ - BMK_loadFiles(srcBuffer, benchedSize, fileSizes, fileNamesTable, nbFiles); - - /* Bench */ - snprintf (mfName, sizeof(mfName), " %u files", nbFiles); - { const char* displayName = (nbFiles > 1) ? mfName : fileNamesTable[0]; - BMK_benchCLevel(srcBuffer, benchedSize, - displayName, cLevel, cLevelLast, - fileSizes, nbFiles); - } - - /* clean up */ - free(srcBuffer); - free(fileSizes); -} - - -static void BMK_syntheticTest(int cLevel, int cLevelLast, double compressibility) -{ - char name[20] = {0}; - size_t benchedSize = 10000000; - void* const srcBuffer = malloc(benchedSize); - - /* Memory allocation */ - if (!srcBuffer) EXM_THROW(21, "not enough memory"); - - /* Fill input buffer */ - RDG_genBuffer(srcBuffer, benchedSize, compressibility, 0.0, 0); - - /* Bench */ - snprintf (name, sizeof(name), "Synthetic %2u%%", (unsigned)(compressibility*100)); - BMK_benchCLevel(srcBuffer, benchedSize, name, cLevel, cLevelLast, &benchedSize, 1); - - /* clean up */ - free(srcBuffer); -} - - -int BMK_benchFiles(const char** fileNamesTable, unsigned nbFiles, - int cLevel, int cLevelLast) -{ - double const compressibility = (double)g_compressibilityDefault / 100; - - if (cLevel < LIZARD_MIN_CLEVEL) cLevel = LIZARD_MIN_CLEVEL; - if (cLevel > LIZARD_MAX_CLEVEL) cLevel = LIZARD_MAX_CLEVEL; - if (cLevelLast > LIZARD_MAX_CLEVEL) cLevelLast = LIZARD_MAX_CLEVEL; - if (cLevelLast < cLevel) cLevelLast = cLevel; - if (cLevelLast > cLevel) DISPLAYLEVEL(2, "Benchmarking levels from %d to %d\n", cLevel, cLevelLast); - - if (nbFiles == 0) - BMK_syntheticTest(cLevel, cLevelLast, compressibility); - else - BMK_benchFileTable(fileNamesTable, nbFiles, cLevel, cLevelLast); - return 0; -} diff --git a/contrib/lizard/programs/bench.h b/contrib/lizard/programs/bench.h deleted file mode 100644 index feff51293f3..00000000000 --- a/contrib/lizard/programs/bench.h +++ /dev/null @@ -1,37 +0,0 @@ -/* - bench.h - Demo program to benchmark open-source compression algorithm - Copyright (C) Yann Collet 2012-2016 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ -#ifndef BENCH_H_12 -#define BENCH_H_12 - -#include - -int BMK_benchFiles(const char** fileNamesTable, unsigned nbFiles, - int cLevel, int cLevelLast); - -/* Set Parameters */ -void BMK_SetNbSeconds(unsigned nbLoops); -void BMK_SetBlockSize(size_t blockSize); -void BMK_setAdditionalParam(int additionalParam); -void BMK_setNotificationLevel(unsigned level); - -#endif /* BENCH_H_125623623633 */ diff --git a/contrib/lizard/programs/datagen.c b/contrib/lizard/programs/datagen.c deleted file mode 100644 index 87d6044830a..00000000000 --- a/contrib/lizard/programs/datagen.c +++ /dev/null @@ -1,188 +0,0 @@ -/* - datagen.c - compressible data generator test tool - Copyright (C) Yann Collet 2012-2016 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -/************************************** -* Includes -**************************************/ -#include "platform.h" /* Compiler options, SET_BINARY_MODE */ -#include "util.h" /* U32 */ -#include /* malloc */ -#include /* FILE, fwrite */ -#include /* memcpy */ - - -/************************************** -* Constants -**************************************/ -#define KB *(1 <<10) - -#define PRIME1 2654435761U -#define PRIME2 2246822519U - - -/************************************** -* Local types -**************************************/ -#define LTLOG 13 -#define LTSIZE (1<> (32 - r))) -static unsigned int RDG_rand(U32* src) -{ - U32 rand32 = *src; - rand32 *= PRIME1; - rand32 ^= PRIME2; - rand32 = RDG_rotl32(rand32, 13); - *src = rand32; - return rand32; -} - - -static void RDG_fillLiteralDistrib(litDistribTable lt, double ld) -{ - BYTE const firstChar = ld <= 0.0 ? 0 : '('; - BYTE const lastChar = ld <= 0.0 ? 255 : '}'; - BYTE character = ld <= 0.0 ? 0 : '0'; - U32 u = 0; - - while (u lastChar) character = firstChar; - } -} - - -static BYTE RDG_genChar(U32* seed, const litDistribTable lt) -{ - U32 id = RDG_rand(seed) & LTMASK; - return (lt[id]); -} - - -#define RDG_DICTSIZE (32 KB) -#define RDG_RAND15BITS ((RDG_rand(seed) >> 3) & 32767) -#define RDG_RANDLENGTH ( ((RDG_rand(seed) >> 7) & 7) ? (RDG_rand(seed) & 15) : (RDG_rand(seed) & 511) + 15) -void RDG_genBlock(void* buffer, size_t buffSize, size_t prefixSize, double matchProba, litDistribTable lt, unsigned* seedPtr) -{ - BYTE* buffPtr = (BYTE*)buffer; - const U32 matchProba32 = (U32)(32768 * matchProba); - size_t pos = prefixSize; - U32* seed = seedPtr; - - /* special case */ - while (matchProba >= 1.0) - { - size_t size0 = RDG_rand(seed) & 3; - size0 = (size_t)1 << (16 + size0 * 2); - size0 += RDG_rand(seed) & (size0-1); /* because size0 is power of 2*/ - if (buffSize < pos + size0) - { - memset(buffPtr+pos, 0, buffSize-pos); - return; - } - memset(buffPtr+pos, 0, size0); - pos += size0; - buffPtr[pos-1] = RDG_genChar(seed, lt); - } - - /* init */ - if (pos==0) buffPtr[0] = RDG_genChar(seed, lt), pos=1; - - /* Generate compressible data */ - while (pos < buffSize) - { - /* Select : Literal (char) or Match (within 32K) */ - if (RDG_RAND15BITS < matchProba32) - { - /* Copy (within 32K) */ - size_t match; - size_t d; - int length = RDG_RANDLENGTH + 4; - U32 offset = RDG_RAND15BITS + 1; - if (offset > pos) offset = (U32)pos; - match = pos - offset; - d = pos + length; - if (d > buffSize) d = buffSize; - while (pos < d) buffPtr[pos++] = buffPtr[match++]; - } - else - { - /* Literal (noise) */ - size_t d; - size_t length = RDG_RANDLENGTH; - d = pos + length; - if (d > buffSize) d = buffSize; - while (pos < d) buffPtr[pos++] = RDG_genChar(seed, lt); - } - } -} - - -void RDG_genBuffer(void* buffer, size_t size, double matchProba, double litProba, unsigned seed) -{ - litDistribTable lt; - if (litProba==0.0) litProba = matchProba / 4.5; - RDG_fillLiteralDistrib(lt, litProba); - RDG_genBlock(buffer, size, 0, matchProba, lt, &seed); -} - - -#define RDG_BLOCKSIZE (128 KB) -void RDG_genOut(unsigned long long size, double matchProba, double litProba, unsigned seed) -{ - BYTE buff[RDG_DICTSIZE + RDG_BLOCKSIZE]; - U64 total = 0; - size_t genBlockSize = RDG_BLOCKSIZE; - litDistribTable lt; - - /* init */ - if (litProba==0.0) litProba = matchProba / 4.5; - RDG_fillLiteralDistrib(lt, litProba); - SET_BINARY_MODE(stdout); - - /* Generate dict */ - RDG_genBlock(buff, RDG_DICTSIZE, 0, matchProba, lt, &seed); - - /* Generate compressible data */ - while (total < size) - { - RDG_genBlock(buff, RDG_DICTSIZE+RDG_BLOCKSIZE, RDG_DICTSIZE, matchProba, lt, &seed); - if (size-total < RDG_BLOCKSIZE) genBlockSize = (size_t)(size-total); - total += genBlockSize; - fwrite(buff, 1, genBlockSize, stdout); - /* update dict */ - memcpy(buff, buff + RDG_BLOCKSIZE, RDG_DICTSIZE); - } -} diff --git a/contrib/lizard/programs/datagen.h b/contrib/lizard/programs/datagen.h deleted file mode 100644 index 2cc41d2a3ff..00000000000 --- a/contrib/lizard/programs/datagen.h +++ /dev/null @@ -1,39 +0,0 @@ -/* - datagen.h - compressible data generator header - Copyright (C) Yann Collet 2012-2015 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - - -#include /* size_t */ - -void RDG_genOut(unsigned long long size, double matchProba, double litProba, unsigned seed); -void RDG_genBuffer(void* buffer, size_t size, double matchProba, double litProba, unsigned seed); -/* RDG_genOut - Generate 'size' bytes of compressible data into stdout. - Compressibility can be controlled using 'matchProba'. - 'LitProba' is optional, and affect variability of bytes. If litProba==0.0, default value is used. - Generated data can be selected using 'seed'. - If (matchProba, litProba and seed) are equal, the function always generate the same content. - - RDG_genBuffer - Same as RDG_genOut, but generate data into provided buffer -*/ diff --git a/contrib/lizard/programs/lizard.1 b/contrib/lizard/programs/lizard.1 deleted file mode 100644 index b5d84213442..00000000000 --- a/contrib/lizard/programs/lizard.1 +++ /dev/null @@ -1,229 +0,0 @@ -\." -\." lizard.1: This is a manual page for 'lizard' program. This file is part of the -\." lizard project. -\." Author: Yann Collet -\." -. -\." No hyphenation -.hy 0 -.nr HY 0 -. -.TH lizard "1" "2015-03-21" "lizard" "User Commands" -.SH NAME -\fBlizard, unlizard, lizardcat\fR \- Compress or decompress .liz files - -.SH SYNOPSIS -.TP 5 -\fBlizard\fR [\fBOPTIONS\fR] [-|INPUT-FILE] -.PP -.B unlizard -is equivalent to -.BR "lizard \-d" -.br -.B lizardcat -is equivalent to -.BR "lizard \-dc" -.br -.PP -When writing scripts that need to decompress files, -it is recommended to always use the name -.B lizard -with appropriate arguments -.RB ( "lizard \-d" -or -.BR "lizard \-dc" ) -instead of the names -.B unlizard -and -.BR lizardcat . - - -.SH DESCRIPTION -.PP -\fBlizard\fR is an extremely fast lossless compression algorithm, -based on \fBbyte-aligned LZ77\fR family of compression scheme. -\fBlizard\fR offers compression speeds of 400 MB/s per core, linearly scalable with multi-core CPUs. -It features an extremely fast decoder, with speed in multiple GB/s per core, -typically reaching RAM speed limit on multi-core systems. -The native file format is the -.B .liz -format. - -.B lizard -supports a command line syntax similar but not identical to -.BR gzip (1). -Differences are : -\fBlizard\fR preserve original files ; -\fBlizard file1 file2\fR means : compress file1 \fIinto\fR file2 ; -\fBlizard file\fR shows real-time statistics during compression . - -Default behaviors can be modified by opt-in commands, described below. -\fBlizard --quiet --multiple\fR more closely mimics \fBgzip\fR behavior. - -.SS "Concatenation of .liz files" -It is possible to concatenate -.B .liz -files as is. -.B lizard -will decompress such files as if they were a single -.B .liz -file. For example: - lizard file1 > foo.liz - lizard file2 >> foo.liz - then - lizardcat foo.liz - is equivalent to : - cat file1 file2 - -.PP - -.SH OPTIONS -. -.SS "Short commands concatenation" -In some cases, some options can be expressed using short command -.B "-x" -or long command -.B "--long-word" . -Short commands can be concatenated together. For example, -.B "-d -c" -is equivalent to -.B "-dc" . -Long commands cannot be concatenated. -They must be clearly separated by a space. -.SS "Multiple commands" -When multiple contradictory commands are issued on a same command line, -only the latest one will be applied. -. -.SS "Operation mode" -.TP -.BR \-z ", " \-\-compress -Compress. -This is the default operation mode -when no operation mode option is specified , -no other operation mode is implied from the command name -(for example, -.B unlizard -implies -.B \-\-decompress ), -nor from the input file name -(for example, a file extension -.B .liz -implies -.B \-\-decompress -by default). -.B -z -can also be used to force compression of an already compressed -.B .liz -file. -.TP -.BR \-d ", " \-\-decompress ", " \-\-uncompress -Decompress. -.B --decompress -is also the default operation when the input filename has an -.B .liz -extensionq -.TP -.BR \-t ", " \-\-test -Test the integrity of compressed -.B .liz -files. -The decompressed data is discarded. -No files are created nor removed. -. -.SS "Operation modifiers" -.TP -.B \-1 - fast compression (default) -.TP -.B \-9 - high compression - -.TP -.BR \-f ", " --[no-]force - This option has several effects: -.RS -.IP \(bu 3 -If the target file already exists, -overwrite it without prompting. -.IP \(bu 3 -When used with -.B \-\-decompress -and -.B lizard -cannot recognize the type of the source file, -copy the source file as is to standard output. -This allows -.B lizardcat -.B \-\-force -to be used like -.BR cat (1) -for files that have not been compressed with -.BR lizard . -.RE - -.TP -.BR \-c ", " \--stdout ", " \--to-stdout - force write to standard output, even if it is the console - -.TP -.BR \-m ", " \--multiple - Multiple file names. - By default, the second filename is used as the destination filename for the compressed file. - With -.B -m -, you can specify any number of input filenames. Each of them will be compressed -independently, and the resulting name of each compressed file will be -.B filename.liz -. - -.TP -.B \-B# - block size [4-7](default : 7) - B1 = 128KB, B2=256KB, B3=1MB, B4=4MB, B5=16MB, B6=64MB, B7=256MB -.TP -.B \-BD - block dependency (improves compression ratio on small blocks) -.TP -.B \--[no-]frame-crc - select frame checksum (default:enabled) -.TP -.B \--[no-]content-size - header includes original size (default:not present) - Note : this option can only be activated when the original size can be determined, -hence for a file. It won't work with unknown source size, such as stdin or pipe. -.TP -.B \--[no-]sparse - sparse mode support (default:enabled on file, disabled on stdout) -.TP -.B \-l - use Legacy format (useful for Linux Kernel compression) -. -.SS "Other options" -.TP -.BR \-v ", " --verbose - verbose mode -.TP -.BR \-q ", " --quiet - suppress warnings and real-time statistics; specify twice to suppress errors too -.TP -.B \-h/\-H - display help/long help and exit -.TP -.BR \-V ", " \--version - display Version number and exit -.TP -.BR \-k ", " \--keep - Don't delete source file. -This is default behavior anyway, so this option is just for compatibility with gzip/xz. -.TP -.B \-b - benchmark file(s) -.TP -.B \-i# - iteration loops [1-9](default : 3), benchmark mode only - -.SH BUGS -Report bugs at: https://github.com/inikep/lizard/issues - -.SH AUTHOR -Yann Collet diff --git a/contrib/lizard/programs/lizardcli.c b/contrib/lizard/programs/lizardcli.c deleted file mode 100644 index 2a63fe8e001..00000000000 --- a/contrib/lizard/programs/lizardcli.c +++ /dev/null @@ -1,581 +0,0 @@ -/* - Lizardcli - Lizard Command Line Interface - Copyright (C) Yann Collet 2011-2016 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ -/* - Note : this is stand-alone program. - It is not part of Lizard compression library, it is a user program of the Lizard library. - The license of Lizard library is BSD. - The license of xxHash library is BSD. - The license of this compression CLI program is GPLv2. -*/ - - -/**************************** -* Includes -*****************************/ -#include "platform.h" /* Compiler options, IS_CONSOLE */ -#include "util.h" /* UTIL_HAS_CREATEFILELIST, UTIL_createFileList */ -#include /* fprintf, getchar */ -#include /* exit, calloc, free */ -#include /* strcmp, strlen */ -#include "bench.h" /* BMK_benchFile, BMK_SetNbIterations, BMK_SetBlocksize, BMK_SetPause */ -#include "lizardio.h" /* LizardIO_compressFilename, LizardIO_decompressFilename, LizardIO_compressMultipleFilenames */ -#include "lizard_compress.h" /* LizardHC_DEFAULT_CLEVEL, LIZARD_VERSION_STRING */ - - - -/***************************** -* Constants -******************************/ -#define COMPRESSOR_NAME "Lizard command line interface" -#define AUTHOR "Y.Collet & P.Skibinski" -#define WELCOME_MESSAGE "%s %i-bit %s by %s (%s)\n", COMPRESSOR_NAME, (int)(sizeof(void*)*8), LIZARD_VERSION_STRING, AUTHOR, __DATE__ -#define LIZARD_EXTENSION ".liz" -#define LIZARDCAT "lizardcat" -#define UNLIZARD "unlizard" - -#define KB *(1U<<10) -#define MB *(1U<<20) -#define GB *(1U<<30) - -#define LIZARD_BLOCKSIZEID_DEFAULT 4 - - -/*-************************************ -* Macros -***************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define DISPLAYLEVEL(l, ...) if (displayLevel>=l) { DISPLAY(__VA_ARGS__); } -static unsigned displayLevel = 2; /* 0 : no display ; 1: errors only ; 2 : downgradable normal ; 3 : non-downgradable normal; 4 : + information */ - - -/*-************************************ -* Exceptions -***************************************/ -#define DEBUG 0 -#define DEBUGOUTPUT(...) if (DEBUG) DISPLAY(__VA_ARGS__); -#define EXM_THROW(error, ...) \ -{ \ - DEBUGOUTPUT("Error defined at %s, line %i : \n", __FILE__, __LINE__); \ - DISPLAYLEVEL(1, "Error %i : ", error); \ - DISPLAYLEVEL(1, __VA_ARGS__); \ - DISPLAYLEVEL(1, "\n"); \ - exit(error); \ -} - - -/*-************************************ -* Version modifiers -***************************************/ -#define EXTENDED_ARGUMENTS -#define EXTENDED_HELP -#define EXTENDED_FORMAT -#define DEFAULT_COMPRESSOR LizardIO_compressFilename -#define DEFAULT_DECOMPRESSOR LizardIO_decompressFilename -int LizardIO_compressFilename_Legacy(const char* input_filename, const char* output_filename, int compressionlevel); /* hidden function */ - - -/*-*************************** -* Functions -*****************************/ -static int usage(const char* exeName) -{ - DISPLAY( "Usage :\n"); - DISPLAY( " %s [arg] [input] [output]\n", exeName); - DISPLAY( "\n"); - DISPLAY( "input : a filename\n"); - DISPLAY( " with no FILE, or when FILE is - or %s, read standard input\n", stdinmark); - DISPLAY( "Arguments :\n"); - DISPLAY( " -10...-19 : compression method fastLZ4 = 16-bit bytewise codewords\n"); - DISPLAY( " higher number == more compression but slower\n"); - DISPLAY( " -20...-29 : compression method LIZv1 = 24-bit bytewise codewords\n"); -#ifndef LIZARD_NO_HUFFMAN - DISPLAY( " -30...-39 : compression method fastLZ4 + Huffman\n"); - DISPLAY( " -40...-49 : compression method LIZv1 + Huffman\n"); -#endif - DISPLAY( " -d : decompression (default for %s extension)\n", LIZARD_EXTENSION); - DISPLAY( " -z : force compression\n"); - DISPLAY( " -f : overwrite output without prompting \n"); - DISPLAY( "--rm : remove source file(s) after successful de/compression \n"); - DISPLAY( " -h/-H : display help/long help and exit\n"); - return 0; -} - -static int usage_advanced(const char* exeName) -{ - DISPLAY(WELCOME_MESSAGE); - usage(exeName); - DISPLAY( "\n"); - DISPLAY( "Advanced arguments :\n"); - DISPLAY( " -V : display Version number and exit\n"); - DISPLAY( " -v : verbose mode\n"); - DISPLAY( " -q : suppress warnings; specify twice to suppress errors too\n"); - DISPLAY( " -c : force write to standard output, even if it is the console\n"); - DISPLAY( " -t : test compressed file integrity\n"); - DISPLAY( " -m : multiple input files (implies automatic output filenames)\n"); -#ifdef UTIL_HAS_CREATEFILELIST - DISPLAY( " -r : operate recursively on directories (sets also -m)\n"); -#endif - DISPLAY( " -B# : Block size [1-7] = 128KB, 256KB, 1MB, 4MB, 16MB, 64MB, 256MB (default : 4)\n"); - DISPLAY( " -BD : Block dependency (improve compression ratio)\n"); - /* DISPLAY( " -BX : enable block checksum (default:disabled)\n"); *//* Option currently inactive */ - DISPLAY( "--no-frame-crc : disable stream checksum (default:enabled)\n"); - DISPLAY( "--content-size : compressed frame includes original size (default:not present)\n"); - DISPLAY( "--[no-]sparse : sparse mode (default:enabled on file, disabled on stdout)\n"); - DISPLAY( "Benchmark arguments :\n"); - DISPLAY( " -b# : benchmark file(s), using # compression level (default : 1) \n"); - DISPLAY( " -e# : test all compression levels from -bX to # (default : 1)\n"); - DISPLAY( " -i# : minimum evaluation time in seconds (default : 3s)\n"); - DISPLAY( " -B# : cut file into independent blocks of size # bytes [32+]\n"); - DISPLAY( " or predefined block size [1-7] (default: 4)\n"); - EXTENDED_HELP; - return 0; -} - -static int usage_longhelp(const char* exeName) -{ - usage_advanced(exeName); - DISPLAY( "\n"); - DISPLAY( "****************************\n"); - DISPLAY( "***** Advanced comment *****\n"); - DISPLAY( "****************************\n"); - DISPLAY( "\n"); - DISPLAY( "Which values can [output] have ? \n"); - DISPLAY( "---------------------------------\n"); - DISPLAY( "[output] : a filename \n"); - DISPLAY( " '%s', or '-' for standard output (pipe mode)\n", stdoutmark); - DISPLAY( " '%s' to discard output (test mode) \n", NULL_OUTPUT); - DISPLAY( "[output] can be left empty. In this case, it receives the following value :\n"); - DISPLAY( " - if stdout is not the console, then [output] = stdout \n"); - DISPLAY( " - if stdout is console : \n"); - DISPLAY( " + for compression, output to filename%s \n", LIZARD_EXTENSION); - DISPLAY( " + for decompression, output to filename without '%s'\n", LIZARD_EXTENSION); - DISPLAY( " > if input filename has no '%s' extension : error \n", LIZARD_EXTENSION); - DISPLAY( "\n"); - DISPLAY( "stdin, stdout and the console : \n"); - DISPLAY( "--------------------------------\n"); - DISPLAY( "To protect the console from binary flooding (bad argument mistake)\n"); - DISPLAY( "%s will refuse to read from console, or write to console \n", exeName); - DISPLAY( "except if '-c' command is specified, to force output to console \n"); - DISPLAY( "\n"); - DISPLAY( "Simple example :\n"); - DISPLAY( "----------------\n"); - DISPLAY( "1 : compress 'filename' fast, using default output name 'filename.liz'\n"); - DISPLAY( " %s filename\n", exeName); - DISPLAY( "\n"); - DISPLAY( "Short arguments can be aggregated. For example :\n"); - DISPLAY( "----------------------------------\n"); - DISPLAY( "2 : compress 'filename' in high compression mode, overwrite output if exists\n"); - DISPLAY( " %s -9 -f filename \n", exeName); - DISPLAY( " is equivalent to :\n"); - DISPLAY( " %s -9f filename \n", exeName); - DISPLAY( "\n"); - DISPLAY( "%s can be used in 'pure pipe mode'. For example :\n", exeName); - DISPLAY( "-------------------------------------\n"); - DISPLAY( "3 : compress data stream from 'generator', send result to 'consumer'\n"); - DISPLAY( " generator | %s | consumer \n", exeName); - return 0; -} - -static int badusage(const char* exeName) -{ - DISPLAYLEVEL(1, "Incorrect parameters\n"); - if (displayLevel >= 1) usage(exeName); - exit(1); -} - - -static void waitEnter(void) -{ - DISPLAY("Press enter to continue...\n"); - (void)getchar(); -} - - -static const char* lastNameFromPath(const char* path) -{ - const char* name = strrchr(path, '/'); - if (name==NULL) name = strrchr(path, '\\'); /* windows */ - if (name==NULL) return path; - return name+1; -} - - -/*! readU32FromChar() : - @return : unsigned integer value reach from input in `char` format - Will also modify `*stringPtr`, advancing it to position where it stopped reading. - Note : this function can overflow if result > MAX_UINT */ -static unsigned readU32FromChar(const char** stringPtr) -{ - unsigned result = 0; - while ((**stringPtr >='0') && (**stringPtr <='9')) - result *= 10, result += **stringPtr - '0', (*stringPtr)++ ; - return result; -} - -typedef enum { om_auto, om_compress, om_decompress, om_test, om_bench } operationMode_e; - -int main(int argc, const char** argv) -{ - int i, - cLevel=1, - cLevelLast=1, - forceStdout=0, - main_pause=0, - multiple_inputs=0, - operationResult=0; - operationMode_e mode = om_auto; - const char* input_filename = NULL; - const char* output_filename= NULL; - char* dynNameSpace = NULL; - const char** inFileNames = (const char**) calloc(argc, sizeof(char*)); - unsigned ifnIdx=0; - const char nullOutput[] = NULL_OUTPUT; - const char extension[] = LIZARD_EXTENSION; - size_t blockSize = LizardIO_setBlockSizeID(LIZARD_BLOCKSIZEID_DEFAULT); - const char* const exeName = lastNameFromPath(argv[0]); -#ifdef UTIL_HAS_CREATEFILELIST - const char** extendedFileList = NULL; - char* fileNamesBuf = NULL; - unsigned fileNamesNb, recursive=0; -#endif - - /* Init */ - if (inFileNames==NULL) { - DISPLAY("Allocation error : not enough memory \n"); - return 1; - } - inFileNames[0] = stdinmark; - LizardIO_setOverwrite(0); - - /* lizardcat predefined behavior */ - if (!strcmp(exeName, LIZARDCAT)) { - mode = om_decompress; - LizardIO_setOverwrite(1); - forceStdout=1; - output_filename=stdoutmark; - displayLevel=1; - multiple_inputs=1; - } - if (!strcmp(exeName, UNLIZARD)) { mode = om_decompress; } - - /* command switches */ - for(i=1; i='0') && (*argument<='9')) { - cLevel = readU32FromChar(&argument); - argument--; - continue; - } - - - switch(argument[0]) - { - /* Display help */ - case 'V': DISPLAY(WELCOME_MESSAGE); goto _cleanup; /* Version */ - case 'h': usage_advanced(exeName); goto _cleanup; - case 'H': usage_longhelp(exeName); goto _cleanup; - - case 'e': - argument++; - cLevelLast = readU32FromChar(&argument); - argument--; - break; - - /* Compression (default) */ - case 'z': mode = om_compress; break; - - /* Decoding */ - case 'd': mode = om_decompress; break; - - /* Force stdout, even if stdout==console */ - case 'c': forceStdout=1; output_filename=stdoutmark; break; - - /* Test integrity */ - case 't': mode = om_test; break; - - /* Overwrite */ - case 'f': LizardIO_setOverwrite(1); break; - - /* Verbose mode */ - case 'v': displayLevel++; break; - - /* Quiet mode */ - case 'q': if (displayLevel) displayLevel--; break; - - /* keep source file (default anyway, so useless) (for xz/lzma compatibility) */ - case 'k': LizardIO_setRemoveSrcFile(0); break; - - /* Modify Block Properties */ - case 'B': - while (argument[1]!=0) { - int exitBlockProperties=0; - switch(argument[1]) - { - case 'D': LizardIO_setBlockMode(LizardIO_blockLinked); argument++; break; - case 'X': LizardIO_setBlockChecksumMode(1); argument ++; break; /* disabled by default */ - default : - if (argument[1] < '0' || argument[1] > '9') { - exitBlockProperties=1; - break; - } else { - unsigned B; - argument++; - B = readU32FromChar(&argument); - argument--; - if (B < 1) badusage(exeName); - if (B <= 7) { - blockSize = LizardIO_setBlockSizeID(B); - BMK_SetBlockSize(blockSize); - DISPLAYLEVEL(2, "using blocks of size %u KB \n", (U32)(blockSize>>10)); - } else { - if (B < 32) badusage(exeName); - BMK_SetBlockSize(B); - if (B >= 1024) { - DISPLAYLEVEL(2, "bench: using blocks of size %u KB \n", (U32)(B>>10)); - } else { - DISPLAYLEVEL(2, "bench: using blocks of size %u bytes \n", (U32)(B)); - } - } - break; - } - } - if (exitBlockProperties) break; - } - break; - - /* Benchmark */ - case 'b': mode = om_bench; multiple_inputs=1; - break; - -#ifdef UTIL_HAS_CREATEFILELIST - /* recursive */ - case 'r': recursive=1; - /* fallthrough */ -#endif - /* Treat non-option args as input files. See https://code.google.com/p/lizard/issues/detail?id=151 */ - /* fallthrough */ - case 'm': multiple_inputs=1; - break; - - /* Modify Nb Seconds (benchmark only) */ - case 'i': - { unsigned iters; - argument++; - iters = readU32FromChar(&argument); - argument--; - BMK_setNotificationLevel(displayLevel); - BMK_SetNbSeconds(iters); /* notification if displayLevel >= 3 */ - } - break; - - /* Pause at the end (hidden option) */ - case 'p': main_pause=1; break; - - /* Specific commands for customized versions */ - EXTENDED_ARGUMENTS; - - /* Unrecognised command */ - default : badusage(exeName); - } - } - continue; - } - - /* Store in *inFileNames[] if -m is used. */ - if (multiple_inputs) { inFileNames[ifnIdx++]=argument; continue; } - - /* Store first non-option arg in input_filename to preserve original cli logic. */ - if (!input_filename) { input_filename=argument; continue; } - - /* Second non-option arg in output_filename to preserve original cli logic. */ - if (!output_filename) { - output_filename=argument; - if (!strcmp (output_filename, nullOutput)) output_filename = nulmark; - continue; - } - - /* 3rd non-option arg should not exist */ - DISPLAYLEVEL(1, "Warning : %s won't be used ! Do you want multiple input files (-m) ? \n", argument); - } - - DISPLAYLEVEL(3, WELCOME_MESSAGE); - if ((mode == om_compress) || (mode == om_bench)) DISPLAYLEVEL(4, "Blocks size : %i KB\n", (U32)(blockSize>>10)); - - if (multiple_inputs) { - input_filename = inFileNames[0]; -#ifdef UTIL_HAS_CREATEFILELIST - if (recursive) { /* at this stage, filenameTable is a list of paths, which can contain both files and directories */ - extendedFileList = UTIL_createFileList(inFileNames, ifnIdx, &fileNamesBuf, &fileNamesNb); - if (extendedFileList) { - unsigned u; - for (u=0; u try to select one automatically (when possible) */ - while ((!output_filename) && (multiple_inputs==0)) { - if (!IS_CONSOLE(stdout)) { output_filename=stdoutmark; break; } /* Default to stdout whenever possible (i.e. not a console) */ - if (mode == om_auto) { /* auto-determine compression or decompression, based on file extension */ - size_t const inSize = strlen(input_filename); - size_t const extSize = strlen(LIZARD_EXTENSION); - size_t const extStart= (inSize > extSize) ? inSize-extSize : 0; - if (!strcmp(input_filename+extStart, LIZARD_EXTENSION)) mode = om_decompress; - else mode = om_compress; - } - if (mode == om_compress) { /* compression to file */ - size_t const l = strlen(input_filename); - dynNameSpace = (char*)calloc(1,l+5); - if (dynNameSpace==NULL) { perror(exeName); exit(1); } - strcpy(dynNameSpace, input_filename); - strcat(dynNameSpace, LIZARD_EXTENSION); - output_filename = dynNameSpace; - DISPLAYLEVEL(2, "Compressed filename will be : %s \n", output_filename); - break; - } - if (mode == om_decompress) {/* decompression to file (automatic name will work only if input filename has correct format extension) */ - size_t outl; - size_t const inl = strlen(input_filename); - dynNameSpace = (char*)calloc(1,inl+1); - if (dynNameSpace==NULL) { perror(exeName); exit(1); } - strcpy(dynNameSpace, input_filename); - outl = inl; - if (inl>4) - while ((outl >= inl-4) && (input_filename[outl] == extension[outl-inl+4])) dynNameSpace[outl--]=0; - if (outl != inl-5) { DISPLAYLEVEL(1, "Cannot determine an output filename\n"); badusage(exeName); } - output_filename = dynNameSpace; - DISPLAYLEVEL(2, "Decoding file %s \n", output_filename); - } - break; - } - - /* Check if output is defined as console; trigger an error in this case */ - if (!output_filename) output_filename = "*\\dummy^!//"; - if (!strcmp(output_filename,stdoutmark) && IS_CONSOLE(stdout) && !forceStdout) { - DISPLAYLEVEL(1, "refusing to write to console without -c\n"); - exit(1); - } - - /* Downgrade notification level in stdout and multiple file mode */ - if (!strcmp(output_filename,stdoutmark) && (displayLevel==2)) displayLevel=1; - if ((multiple_inputs) && (displayLevel==2)) displayLevel=1; - - /* IO Stream/File */ - LizardIO_setNotificationLevel(displayLevel); - if (ifnIdx == 0) multiple_inputs = 0; - if (mode == om_decompress) { - if (multiple_inputs) - operationResult = LizardIO_decompressMultipleFilenames(inFileNames, ifnIdx, !strcmp(output_filename,stdoutmark) ? stdoutmark : LIZARD_EXTENSION); - else - operationResult = LizardIO_decompressFilename(input_filename, output_filename); - } else { /* compression is default action */ - { - if (multiple_inputs) - operationResult = LizardIO_compressMultipleFilenames(inFileNames, ifnIdx, LIZARD_EXTENSION, cLevel); - else - operationResult = LizardIO_compressFilename(input_filename, output_filename, cLevel); - } - } - -_cleanup: - if (main_pause) waitEnter(); - if (dynNameSpace) free(dynNameSpace); -#ifdef UTIL_HAS_CREATEFILELIST - if (extendedFileList) - UTIL_freeFileList(extendedFileList, fileNamesBuf); - else -#endif - free((void*)inFileNames); - return operationResult; -} diff --git a/contrib/lizard/programs/lizardio.c b/contrib/lizard/programs/lizardio.c deleted file mode 100644 index 7638a55178a..00000000000 --- a/contrib/lizard/programs/lizardio.c +++ /dev/null @@ -1,896 +0,0 @@ -/* - Lizardio.c - Lizard File/Stream Interface - Copyright (C) Yann Collet 2011-2015 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ -/* - Note : this is stand-alone program. - It is not part of Lizard compression library, it is a user code of the Lizard library. - - The license of Lizard library is BSD. - - The license of xxHash library is BSD. - - The license of this source file is GPLv2. -*/ - -/*-************************************ -* Compiler options -**************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -#endif -#if defined(__MINGW32__) && !defined(_POSIX_SOURCE) -# define _POSIX_SOURCE 1 /* disable %llu warnings with MinGW on Windows */ -#endif - - -/***************************** -* Includes -*****************************/ -#include "platform.h" /* Large File Support, SET_BINARY_MODE, SET_SPARSE_FILE_MODE, PLATFORM_POSIX_VERSION, __64BIT__ */ -#include "util.h" /* UTIL_getFileStat, UTIL_setFileStat */ -#include /* fprintf, fopen, fread, stdin, stdout, fflush, getchar */ -#include /* malloc, free */ -#include /* strcmp, strlen */ -#include /* clock */ -#include /* stat64 */ -#include /* stat64 */ -#include "lizardio.h" -#include "lizard_frame.h" - - - -/***************************** -* Constants -*****************************/ -#define KB *(1 <<10) -#define MB *(1 <<20) -#define GB *(1U<<30) - -#define _1BIT 0x01 -#define _2BITS 0x03 -#define _3BITS 0x07 -#define _4BITS 0x0F -#define _8BITS 0xFF - -#define MAGICNUMBER_SIZE 4 -#define LIZARDIO_MAGICNUMBER 0x184D2206U -#define LIZARDIO_SKIPPABLE0 0x184D2A50U -#define LIZARDIO_SKIPPABLEMASK 0xFFFFFFF0U - -#define CACHELINE 64 -#define MIN_STREAM_BUFSIZE (192 KB) -#define LIZARDIO_BLOCKSIZEID_DEFAULT 7 - -#define sizeT sizeof(size_t) -#define maskT (sizeT - 1) - - -/************************************** -* Macros -**************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define DISPLAYLEVEL(l, ...) if (g_displayLevel>=l) { DISPLAY(__VA_ARGS__); } -static int g_displayLevel = 0; /* 0 : no display ; 1: errors ; 2 : + result + interaction + warnings ; 3 : + progression; 4 : + information */ - -#define DISPLAYUPDATE(l, ...) if (g_displayLevel>=l) { \ - if (((clock_t)(g_time - clock()) > refreshRate) || (g_displayLevel>=4)) \ - { g_time = clock(); DISPLAY(__VA_ARGS__); \ - if (g_displayLevel>=4) fflush(stderr); } } -static const clock_t refreshRate = CLOCKS_PER_SEC / 6; -static clock_t g_time = 0; - - -/************************************** -* Local Parameters -**************************************/ -static int g_overwrite = 1; -static int g_testMode = 0; -static int g_blockSizeId = LIZARDIO_BLOCKSIZEID_DEFAULT; -static int g_blockChecksum = 0; -static int g_streamChecksum = 1; -static int g_blockIndependence = 1; -static int g_sparseFileSupport = 1; -static int g_contentSizeFlag = 0; - - -/************************************** -* Exceptions -***************************************/ -#ifndef DEBUG -# define DEBUG 0 -#endif -#define DEBUGOUTPUT(...) if (DEBUG) DISPLAY(__VA_ARGS__); -#define EXM_THROW(error, ...) \ -{ \ - DEBUGOUTPUT("Error defined at %s, line %i : \n", __FILE__, __LINE__); \ - DISPLAYLEVEL(1, "Error %i : ", error); \ - DISPLAYLEVEL(1, __VA_ARGS__); \ - DISPLAYLEVEL(1, " \n"); \ - exit(error); \ -} - - -/************************************** -* Version modifiers -**************************************/ -#define EXTENDED_ARGUMENTS -#define EXTENDED_HELP -#define EXTENDED_FORMAT -#define DEFAULT_DECOMPRESSOR LizardIO_decompressLizardF - - -/* ************************************************** */ -/* ****************** Parameters ******************** */ -/* ************************************************** */ - -/* Default setting : overwrite = 1; return : overwrite mode (0/1) */ -int LizardIO_setOverwrite(int yes) -{ - g_overwrite = (yes!=0); - return g_overwrite; -} - -/* Default setting : testMode = 0; return : testMode (0/1) */ -int LizardIO_setTestMode(int yes) -{ - g_testMode = (yes!=0); - return g_testMode; -} - -/* blockSizeID : valid values : 1-7 */ -size_t LizardIO_setBlockSizeID(unsigned bsid) -{ - static const int blockSizeTable[] = { 128 KB, 256 KB, 1 MB, 4 MB, 16 MB, 64 MB, 256 MB }; - static const unsigned minBlockSizeID = 1; - static const unsigned maxBlockSizeID = 7; - if ((bsid < minBlockSizeID) || (bsid > maxBlockSizeID)) return 0; - - g_blockSizeId = bsid; - return blockSizeTable[g_blockSizeId-minBlockSizeID]; -} - - -static size_t LizardIO_GetBlockSize_FromBlockId(unsigned blockSizeID) -{ - static const size_t blockSizes[7] = { 128 KB, 256 KB, 1 MB, 4 MB, 16 MB, 64 MB, 256 MB }; - - if (blockSizeID == 0) blockSizeID = LIZARDIO_BLOCKSIZEID_DEFAULT; - blockSizeID -= 1; - if (blockSizeID >= 7) blockSizeID = LIZARDIO_BLOCKSIZEID_DEFAULT - 1; - - return blockSizes[blockSizeID]; -} - - - -int LizardIO_setBlockMode(LizardIO_blockMode_t blockMode) -{ - g_blockIndependence = (blockMode == LizardIO_blockIndependent); - return g_blockIndependence; -} - -/* Default setting : no checksum */ -int LizardIO_setBlockChecksumMode(int xxhash) -{ - g_blockChecksum = (xxhash != 0); - return g_blockChecksum; -} - -/* Default setting : checksum enabled */ -int LizardIO_setStreamChecksumMode(int xxhash) -{ - g_streamChecksum = (xxhash != 0); - return g_streamChecksum; -} - -/* Default setting : 0 (no notification) */ -int LizardIO_setNotificationLevel(int level) -{ - g_displayLevel = level; - return g_displayLevel; -} - -/* Default setting : 0 (disabled) */ -int LizardIO_setSparseFile(int enable) -{ - g_sparseFileSupport = (enable!=0); - return g_sparseFileSupport; -} - -/* Default setting : 0 (disabled) */ -int LizardIO_setContentSize(int enable) -{ - g_contentSizeFlag = (enable!=0); - return g_contentSizeFlag; -} - -static U32 g_removeSrcFile = 0; -void LizardIO_setRemoveSrcFile(unsigned flag) { g_removeSrcFile = (flag>0); } - - - -/* ************************************************************************ ** -** ********************** Lizard File / Pipe compression ********************* ** -** ************************************************************************ */ - -static int LizardIO_isSkippableMagicNumber(unsigned int magic) { return (magic & LIZARDIO_SKIPPABLEMASK) == LIZARDIO_SKIPPABLE0; } - - -/** LizardIO_openSrcFile() : - * condition : `dstFileName` must be non-NULL. - * @result : FILE* to `dstFileName`, or NULL if it fails */ -static FILE* LizardIO_openSrcFile(const char* srcFileName) -{ - FILE* f; - - if (!strcmp (srcFileName, stdinmark)) { - DISPLAYLEVEL(4,"Using stdin for input\n"); - f = stdin; - SET_BINARY_MODE(stdin); - } else { - f = fopen(srcFileName, "rb"); - if ( f==NULL ) DISPLAYLEVEL(1, "%s: %s \n", srcFileName, strerror(errno)); - } - - return f; -} - -/** FIO_openDstFile() : - * condition : `dstFileName` must be non-NULL. - * @result : FILE* to `dstFileName`, or NULL if it fails */ -static FILE* LizardIO_openDstFile(const char* dstFileName) -{ - FILE* f; - - if (!strcmp (dstFileName, stdoutmark)) { - DISPLAYLEVEL(4,"Using stdout for output\n"); - f = stdout; - SET_BINARY_MODE(stdout); - if (g_sparseFileSupport==1) { - g_sparseFileSupport = 0; - DISPLAYLEVEL(4, "Sparse File Support is automatically disabled on stdout ; try --sparse \n"); - } - } else { - if (!g_overwrite && strcmp (dstFileName, nulmark)) { /* Check if destination file already exists */ - f = fopen( dstFileName, "rb" ); - if (f != NULL) { /* dest exists, prompt for overwrite authorization */ - fclose(f); - if (g_displayLevel <= 1) { /* No interaction possible */ - DISPLAY("%s already exists; not overwritten \n", dstFileName); - return NULL; - } - DISPLAY("%s already exists; do you wish to overwrite (y/N) ? ", dstFileName); - { int ch = getchar(); - if ((ch!='Y') && (ch!='y')) { - DISPLAY(" not overwritten \n"); - return NULL; - } - while ((ch!=EOF) && (ch!='\n')) ch = getchar(); /* flush rest of input line */ - } } } - f = fopen( dstFileName, "wb" ); - if (f==NULL) DISPLAYLEVEL(1, "%s: %s\n", dstFileName, strerror(errno)); - } - - /* sparse file */ - if (f && g_sparseFileSupport) { SET_SPARSE_FILE_MODE(f); } - - return f; -} - - -/* unoptimized version; solves endianess & alignment issues */ -static void LizardIO_writeLE32 (void* p, unsigned value32) -{ - unsigned char* dstPtr = (unsigned char*)p; - dstPtr[0] = (unsigned char)value32; - dstPtr[1] = (unsigned char)(value32 >> 8); - dstPtr[2] = (unsigned char)(value32 >> 16); - dstPtr[3] = (unsigned char)(value32 >> 24); -} - - - -/********************************************* -* Compression using Frame format -*********************************************/ - -typedef struct { - void* srcBuffer; - size_t srcBufferSize; - void* dstBuffer; - size_t dstBufferSize; - LizardF_compressionContext_t ctx; -} cRess_t; - -static cRess_t LizardIO_createCResources(void) -{ - const size_t blockSize = (size_t)LizardIO_GetBlockSize_FromBlockId (g_blockSizeId); - cRess_t ress; - - LizardF_errorCode_t const errorCode = LizardF_createCompressionContext(&(ress.ctx), LIZARDF_VERSION); - if (LizardF_isError(errorCode)) EXM_THROW(30, "Allocation error : can't create LizardF context : %s", LizardF_getErrorName(errorCode)); - - /* Allocate Memory */ - ress.srcBuffer = malloc(blockSize); - ress.srcBufferSize = blockSize; - ress.dstBufferSize = LizardF_compressFrameBound(blockSize, NULL); /* cover worst case */ - ress.dstBuffer = malloc(ress.dstBufferSize); - if (!ress.srcBuffer || !ress.dstBuffer) EXM_THROW(31, "Allocation error : not enough memory"); - - return ress; -} - -static void LizardIO_freeCResources(cRess_t ress) -{ - free(ress.srcBuffer); - free(ress.dstBuffer); - { LizardF_errorCode_t const errorCode = LizardF_freeCompressionContext(ress.ctx); - if (LizardF_isError(errorCode)) EXM_THROW(38, "Error : can't free LizardF context resource : %s", LizardF_getErrorName(errorCode)); } -} - -/* - * LizardIO_compressFilename_extRess() - * result : 0 : compression completed correctly - * 1 : missing or pb opening srcFileName - */ -static int LizardIO_compressFilename_extRess(cRess_t ress, const char* srcFileName, const char* dstFileName, int compressionLevel) -{ - unsigned long long filesize = 0; - unsigned long long compressedfilesize = 0; - FILE* srcFile; - FILE* dstFile; - void* const srcBuffer = ress.srcBuffer; - void* const dstBuffer = ress.dstBuffer; - const size_t dstBufferSize = ress.dstBufferSize; - const size_t blockSize = (size_t)LizardIO_GetBlockSize_FromBlockId (g_blockSizeId); - size_t readSize; - LizardF_compressionContext_t ctx = ress.ctx; /* just a pointer */ - LizardF_preferences_t prefs; - - /* Init */ - srcFile = LizardIO_openSrcFile(srcFileName); - if (srcFile == NULL) return 1; - dstFile = LizardIO_openDstFile(dstFileName); - if (dstFile == NULL) { fclose(srcFile); return 1; } - memset(&prefs, 0, sizeof(prefs)); - - - /* Set compression parameters */ - prefs.autoFlush = 1; - prefs.compressionLevel = compressionLevel; - prefs.frameInfo.blockMode = (LizardF_blockMode_t)g_blockIndependence; - prefs.frameInfo.blockSizeID = (LizardF_blockSizeID_t)g_blockSizeId; - prefs.frameInfo.contentChecksumFlag = (LizardF_contentChecksum_t)g_streamChecksum; - if (g_contentSizeFlag) { - U64 const fileSize = UTIL_getFileSize(srcFileName); - prefs.frameInfo.contentSize = fileSize; /* == 0 if input == stdin */ - if (fileSize==0) - DISPLAYLEVEL(3, "Warning : cannot determine input content size \n"); - } - - /* read first block */ - readSize = fread(srcBuffer, (size_t)1, blockSize, srcFile); - if (ferror(srcFile)) EXM_THROW(30, "Error reading %s ", srcFileName); - filesize += readSize; - - /* single-block file */ - if (readSize < blockSize) { - /* Compress in single pass */ - size_t const cSize = LizardF_compressFrame(dstBuffer, dstBufferSize, srcBuffer, readSize, &prefs); - if (LizardF_isError(cSize)) EXM_THROW(31, "Compression failed : %s", LizardF_getErrorName(cSize)); - compressedfilesize = cSize; - DISPLAYUPDATE(2, "\rRead : %u MB ==> %.2f%% ", - (unsigned)(filesize>>20), (double)compressedfilesize/(filesize+!filesize)*100); /* avoid division by zero */ - - /* Write Block */ - { size_t const sizeCheck = fwrite(dstBuffer, 1, cSize, dstFile); - if (sizeCheck!=cSize) EXM_THROW(32, "Write error : cannot write compressed block"); - } } - - else - - /* multiple-blocks file */ - { - /* Write Archive Header */ - size_t headerSize = LizardF_compressBegin(ctx, dstBuffer, dstBufferSize, &prefs); - if (LizardF_isError(headerSize)) EXM_THROW(33, "File header generation failed : %s", LizardF_getErrorName(headerSize)); - { size_t const sizeCheck = fwrite(dstBuffer, 1, headerSize, dstFile); - if (sizeCheck!=headerSize) EXM_THROW(34, "Write error : cannot write header"); } - compressedfilesize += headerSize; - - /* Main Loop */ - while (readSize>0) { - size_t outSize; - - /* Compress Block */ - outSize = LizardF_compressUpdate(ctx, dstBuffer, dstBufferSize, srcBuffer, readSize, NULL); - if (LizardF_isError(outSize)) EXM_THROW(35, "Compression failed : %s", LizardF_getErrorName(outSize)); - compressedfilesize += outSize; - DISPLAYUPDATE(2, "\rRead : %u MB ==> %.2f%% ", (unsigned)(filesize>>20), (double)compressedfilesize/filesize*100); - - /* Write Block */ - { size_t const sizeCheck = fwrite(dstBuffer, 1, outSize, dstFile); - if (sizeCheck!=outSize) EXM_THROW(36, "Write error : cannot write compressed block"); } - - /* Read next block */ - readSize = fread(srcBuffer, (size_t)1, (size_t)blockSize, srcFile); - filesize += readSize; - } - if (ferror(srcFile)) EXM_THROW(37, "Error reading %s ", srcFileName); - - /* End of Stream mark */ - headerSize = LizardF_compressEnd(ctx, dstBuffer, dstBufferSize, NULL); - if (LizardF_isError(headerSize)) EXM_THROW(38, "End of file generation failed : %s", LizardF_getErrorName(headerSize)); - - { size_t const sizeCheck = fwrite(dstBuffer, 1, headerSize, dstFile); - if (sizeCheck!=headerSize) EXM_THROW(39, "Write error : cannot write end of stream"); } - compressedfilesize += headerSize; - } - - /* Release files */ - fclose (srcFile); - fclose (dstFile); - - /* Copy owner, file permissions and modification time */ - { stat_t statbuf; - if (strcmp (srcFileName, stdinmark) && strcmp (dstFileName, stdoutmark) && UTIL_getFileStat(srcFileName, &statbuf)) - UTIL_setFileStat(dstFileName, &statbuf); - } - - if (g_removeSrcFile) { if (remove(srcFileName)) EXM_THROW(40, "Remove error : %s: %s", srcFileName, strerror(errno)); } /* remove source file : --rm */ - - /* Final Status */ - DISPLAYLEVEL(2, "\r%79s\r", ""); - DISPLAYLEVEL(2, "Compressed %llu bytes into %llu bytes ==> %.2f%%\n", - filesize, compressedfilesize, (double)compressedfilesize/(filesize + !filesize)*100); /* avoid division by zero */ - - return 0; -} - - -int LizardIO_compressFilename(const char* srcFileName, const char* dstFileName, int compressionLevel) -{ - clock_t const start = clock(); - cRess_t const ress = LizardIO_createCResources(); - - int const issueWithSrcFile = LizardIO_compressFilename_extRess(ress, srcFileName, dstFileName, compressionLevel); - - /* Free resources */ - LizardIO_freeCResources(ress); - - /* Final Status */ - { clock_t const end = clock(); - double const seconds = (double)(end - start) / CLOCKS_PER_SEC; - DISPLAYLEVEL(4, "Completed in %.2f sec \n", seconds); - } - - return issueWithSrcFile; -} - - -#define FNSPACE 30 -int LizardIO_compressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix, int compressionLevel) -{ - int i; - int missed_files = 0; - char* dstFileName = (char*)malloc(FNSPACE); - size_t ofnSize = FNSPACE; - const size_t suffixSize = strlen(suffix); - cRess_t const ress = LizardIO_createCResources(); - - if (dstFileName == NULL) return ifntSize; /* not enough memory */ - - /* loop on each file */ - for (i=0; i 1 GB) { - int const seekResult = UTIL_fseek(file, 1 GB, SEEK_CUR); - if (seekResult != 0) EXM_THROW(71, "1 GB skip error (sparse file support)"); - storedSkips -= 1 GB; - } - - while (ptrT < bufferTEnd) { - size_t seg0SizeT = segmentSizeT; - size_t nb0T; - - /* count leading zeros */ - if (seg0SizeT > bufferSizeT) seg0SizeT = bufferSizeT; - bufferSizeT -= seg0SizeT; - for (nb0T=0; (nb0T < seg0SizeT) && (ptrT[nb0T] == 0); nb0T++) ; - storedSkips += (unsigned)(nb0T * sizeT); - - if (nb0T != seg0SizeT) { /* not all 0s */ - int const seekResult = UTIL_fseek(file, storedSkips, SEEK_CUR); - if (seekResult) EXM_THROW(72, "Sparse skip error ; try --no-sparse"); - storedSkips = 0; - seg0SizeT -= nb0T; - ptrT += nb0T; - { size_t const sizeCheck = fwrite(ptrT, sizeT, seg0SizeT, file); - if (sizeCheck != seg0SizeT) EXM_THROW(73, "Write error : cannot write decoded block"); - } } - ptrT += seg0SizeT; - } - - if (bufferSize & maskT) { /* size not multiple of sizeT : implies end of block */ - const char* const restStart = (const char*)bufferTEnd; - const char* restPtr = restStart; - size_t const restSize = bufferSize & maskT; - const char* const restEnd = restStart + restSize; - for (; (restPtr < restEnd) && (*restPtr == 0); restPtr++) ; - storedSkips += (unsigned) (restPtr - restStart); - if (restPtr != restEnd) { - int const seekResult = UTIL_fseek(file, storedSkips, SEEK_CUR); - if (seekResult) EXM_THROW(74, "Sparse skip error ; try --no-sparse"); - storedSkips = 0; - { size_t const sizeCheck = fwrite(restPtr, 1, restEnd - restPtr, file); - if (sizeCheck != (size_t)(restEnd - restPtr)) EXM_THROW(75, "Write error : cannot write decoded end of block"); - } } - } - - return storedSkips; -} - -static void LizardIO_fwriteSparseEnd(FILE* file, unsigned storedSkips) -{ - if (storedSkips>0) { /* implies g_sparseFileSupport>0 */ - int const seekResult = UTIL_fseek(file, storedSkips-1, SEEK_CUR); - if (seekResult != 0) EXM_THROW(69, "Final skip error (sparse file)\n"); - { const char lastZeroByte[1] = { 0 }; - size_t const sizeCheck = fwrite(lastZeroByte, 1, 1, file); - if (sizeCheck != 1) EXM_THROW(69, "Write error : cannot write last zero\n"); - } } -} - - - -typedef struct { - void* srcBuffer; - size_t srcBufferSize; - void* dstBuffer; - size_t dstBufferSize; - FILE* dstFile; - LizardF_decompressionContext_t dCtx; -} dRess_t; - -static const size_t LizardIO_dBufferSize = 64 KB; -static unsigned g_magicRead = 0; -static dRess_t LizardIO_createDResources(void) -{ - dRess_t ress; - - /* init */ - LizardF_errorCode_t const errorCode = LizardF_createDecompressionContext(&ress.dCtx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) EXM_THROW(60, "Can't create LizardF context : %s", LizardF_getErrorName(errorCode)); - - /* Allocate Memory */ - ress.srcBufferSize = LizardIO_dBufferSize; - ress.srcBuffer = malloc(ress.srcBufferSize); - ress.dstBufferSize = LizardIO_dBufferSize; - ress.dstBuffer = malloc(ress.dstBufferSize); - if (!ress.srcBuffer || !ress.dstBuffer) EXM_THROW(61, "Allocation error : not enough memory"); - - ress.dstFile = NULL; - return ress; -} - -static void LizardIO_freeDResources(dRess_t ress) -{ - LizardF_errorCode_t errorCode = LizardF_freeDecompressionContext(ress.dCtx); - if (LizardF_isError(errorCode)) EXM_THROW(69, "Error : can't free LizardF context resource : %s", LizardF_getErrorName(errorCode)); - free(ress.srcBuffer); - free(ress.dstBuffer); -} - - -static unsigned long long LizardIO_decompressLizardF(dRess_t ress, FILE* srcFile, FILE* dstFile) -{ - unsigned long long filesize = 0; - LizardF_errorCode_t nextToLoad; - unsigned storedSkips = 0; - - /* Init feed with magic number (already consumed from FILE* sFile) */ - { size_t inSize = MAGICNUMBER_SIZE; - size_t outSize= 0; - LizardIO_writeLE32(ress.srcBuffer, LIZARDIO_MAGICNUMBER); - nextToLoad = LizardF_decompress(ress.dCtx, ress.dstBuffer, &outSize, ress.srcBuffer, &inSize, NULL); - if (LizardF_isError(nextToLoad)) EXM_THROW(62, "Header error : %s", LizardF_getErrorName(nextToLoad)); - } - - /* Main Loop */ - for (;nextToLoad;) { - size_t readSize; - size_t pos = 0; - size_t decodedBytes = ress.dstBufferSize; - - /* Read input */ - if (nextToLoad > ress.srcBufferSize) nextToLoad = ress.srcBufferSize; - readSize = fread(ress.srcBuffer, 1, nextToLoad, srcFile); - if (!readSize) break; /* reached end of file or stream */ - - while ((pos < readSize) || (decodedBytes == ress.dstBufferSize)) { /* still to read, or still to flush */ - /* Decode Input (at least partially) */ - size_t remaining = readSize - pos; - decodedBytes = ress.dstBufferSize; - nextToLoad = LizardF_decompress(ress.dCtx, ress.dstBuffer, &decodedBytes, (char*)(ress.srcBuffer)+pos, &remaining, NULL); - if (LizardF_isError(nextToLoad)) EXM_THROW(66, "Decompression error : %s", LizardF_getErrorName(nextToLoad)); - pos += remaining; - - /* Write Block */ - if (decodedBytes) { - if (!g_testMode) - storedSkips = LizardIO_fwriteSparse(dstFile, ress.dstBuffer, decodedBytes, storedSkips); - filesize += decodedBytes; - DISPLAYUPDATE(2, "\rDecompressed : %u MB ", (unsigned)(filesize>>20)); - } - - if (!nextToLoad) break; - } - } - /* can be out because readSize == 0, which could be an fread() error */ - if (ferror(srcFile)) EXM_THROW(67, "Read error"); - - if (!g_testMode) LizardIO_fwriteSparseEnd(dstFile, storedSkips); - if (nextToLoad!=0) EXM_THROW(68, "Unfinished stream"); - - return filesize; -} - - -#define PTSIZE (64 KB) -#define PTSIZET (PTSIZE / sizeof(size_t)) -static unsigned long long LizardIO_passThrough(FILE* finput, FILE* foutput, unsigned char MNstore[MAGICNUMBER_SIZE]) -{ - size_t buffer[PTSIZET]; - size_t readBytes = 1; - unsigned long long total = MAGICNUMBER_SIZE; - unsigned storedSkips = 0; - - size_t const sizeCheck = fwrite(MNstore, 1, MAGICNUMBER_SIZE, foutput); - if (sizeCheck != MAGICNUMBER_SIZE) EXM_THROW(50, "Pass-through write error"); - - while (readBytes) { - readBytes = fread(buffer, 1, PTSIZE, finput); - total += readBytes; - storedSkips = LizardIO_fwriteSparse(foutput, buffer, readBytes, storedSkips); - } - if (ferror(finput)) EXM_THROW(51, "Read Error") - - LizardIO_fwriteSparseEnd(foutput, storedSkips); - return total; -} - - -/** Safely handle cases when (unsigned)offset > LONG_MAX */ -static int fseek_u32(FILE *fp, unsigned offset, int where) -{ - const unsigned stepMax = 1U << 30; - int errorNb = 0; - - if (where != SEEK_CUR) return -1; /* Only allows SEEK_CUR */ - while (offset > 0) { - unsigned s = offset; - if (s > stepMax) s = stepMax; - errorNb = UTIL_fseek(fp, (long) s, SEEK_CUR); - if (errorNb != 0) break; - offset -= s; - } - return errorNb; -} - -#define ENDOFSTREAM ((unsigned long long)-1) -static unsigned long long selectDecoder(dRess_t ress, FILE* finput, FILE* foutput) -{ - unsigned char MNstore[MAGICNUMBER_SIZE]; - unsigned magicNumber; - static unsigned nbCalls = 0; - - /* init */ - nbCalls++; - - /* Check Archive Header */ - if (g_magicRead) { /* magic number already read from finput (see legacy frame)*/ - magicNumber = g_magicRead; - g_magicRead = 0; - } else { - size_t const nbReadBytes = fread(MNstore, 1, MAGICNUMBER_SIZE, finput); - if (nbReadBytes==0) { nbCalls = 0; return ENDOFSTREAM; } /* EOF */ - if (nbReadBytes != MAGICNUMBER_SIZE) EXM_THROW(40, "Unrecognized header : Magic Number unreadable"); - magicNumber = LizardIO_readLE32(MNstore); /* Little Endian format */ - } - if (LizardIO_isSkippableMagicNumber(magicNumber)) magicNumber = LIZARDIO_SKIPPABLE0; /* fold skippable magic numbers */ - - switch(magicNumber) - { - case LIZARDIO_MAGICNUMBER: - return LizardIO_decompressLizardF(ress, finput, foutput); - case LIZARDIO_SKIPPABLE0: - DISPLAYLEVEL(4, "Skipping detected skippable area \n"); - { size_t const nbReadBytes = fread(MNstore, 1, 4, finput); - if (nbReadBytes != 4) EXM_THROW(42, "Stream error : skippable size unreadable"); } - { unsigned const size = LizardIO_readLE32(MNstore); /* Little Endian format */ - int const errorNb = fseek_u32(finput, size, SEEK_CUR); - if (errorNb != 0) EXM_THROW(43, "Stream error : cannot skip skippable area"); } - return 0; - EXTENDED_FORMAT; /* macro extension for custom formats */ - default: - if (nbCalls == 1) { /* just started */ - if (!g_testMode && g_overwrite) { - nbCalls = 0; - return LizardIO_passThrough(finput, foutput, MNstore); - } - EXM_THROW(44,"Unrecognized header : file cannot be decoded"); /* Wrong magic number at the beginning of 1st stream */ - } - DISPLAYLEVEL(2, "Stream followed by undecodable data\n"); - return ENDOFSTREAM; - } -} - - -static int LizardIO_decompressSrcFile(dRess_t ress, const char* input_filename, const char* output_filename) -{ - FILE* const foutput = ress.dstFile; - unsigned long long filesize = 0, decodedSize=0; - FILE* finput; - - /* Init */ - finput = LizardIO_openSrcFile(input_filename); - if (finput==NULL) return 1; - - /* Loop over multiple streams */ - do { - decodedSize = selectDecoder(ress, finput, foutput); - if (decodedSize != ENDOFSTREAM) - filesize += decodedSize; - } while (decodedSize != ENDOFSTREAM); - - /* Close */ - fclose(finput); - - if (g_removeSrcFile) { if (remove(input_filename)) EXM_THROW(45, "Remove error : %s: %s", input_filename, strerror(errno)); } /* remove source file : --rm */ - - /* Final Status */ - DISPLAYLEVEL(2, "\r%79s\r", ""); - DISPLAYLEVEL(2, "%-20.20s : decoded %llu bytes \n", input_filename, filesize); - (void)output_filename; - - return 0; -} - - -static int LizardIO_decompressDstFile(dRess_t ress, const char* input_filename, const char* output_filename) -{ - FILE* foutput; - - /* Init */ - foutput = LizardIO_openDstFile(output_filename); - if (foutput==NULL) return 1; /* failure */ - - ress.dstFile = foutput; - LizardIO_decompressSrcFile(ress, input_filename, output_filename); - - fclose(foutput); - - /* Copy owner, file permissions and modification time */ - { stat_t statbuf; - if (strcmp (input_filename, stdinmark) && strcmp (output_filename, stdoutmark) && UTIL_getFileStat(input_filename, &statbuf)) - UTIL_setFileStat(output_filename, &statbuf); - } - - return 0; -} - - -int LizardIO_decompressFilename(const char* input_filename, const char* output_filename) -{ - dRess_t const ress = LizardIO_createDResources(); - clock_t const start = clock(); - - int const missingFiles = LizardIO_decompressDstFile(ress, input_filename, output_filename); - - { clock_t const end = clock(); - double const seconds = (double)(end - start) / CLOCKS_PER_SEC; - DISPLAYLEVEL(4, "Done in %.2f sec \n", seconds); - } - - LizardIO_freeDResources(ress); - return missingFiles; -} - - -int LizardIO_decompressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix) -{ - int i; - int skippedFiles = 0; - int missingFiles = 0; - char* outFileName = (char*)malloc(FNSPACE); - size_t ofnSize = FNSPACE; - size_t const suffixSize = strlen(suffix); - dRess_t ress = LizardIO_createDResources(); - - if (outFileName==NULL) return ifntSize; /* not enough memory */ - ress.dstFile = LizardIO_openDstFile(stdoutmark); - - for (i=0; i /* size_t */ - - -/* ************************************************** */ -/* Special input/output values */ -/* ************************************************** */ -#define NULL_OUTPUT "null" -static const char stdinmark[] = "stdin"; -static const char stdoutmark[] = "stdout"; -#ifdef _WIN32 -static const char nulmark[] = "nul"; -#else -static const char nulmark[] = "/dev/null"; -#endif - - -/* ************************************************** */ -/* ****************** Functions ********************* */ -/* ************************************************** */ - -int LizardIO_compressFilename (const char* input_filename, const char* output_filename, int compressionlevel); -int LizardIO_decompressFilename(const char* input_filename, const char* output_filename); - -int LizardIO_compressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix, int compressionlevel); -int LizardIO_decompressMultipleFilenames(const char** inFileNamesTable, int ifntSize, const char* suffix); - - -/* ************************************************** */ -/* ****************** Parameters ******************** */ -/* ************************************************** */ - -/* Default setting : overwrite = 1; - return : overwrite mode (0/1) */ -int LizardIO_setOverwrite(int yes); - -/* Default setting : testMode = 0; - return : testMode (0/1) */ -int LizardIO_setTestMode(int yes); - -/* blockSizeID : valid values : 4-5-6-7 - return : 0 if error, blockSize if OK */ -size_t LizardIO_setBlockSizeID(unsigned blockSizeID); - -/* Default setting : independent blocks */ -typedef enum { LizardIO_blockLinked=0, LizardIO_blockIndependent} LizardIO_blockMode_t; -int LizardIO_setBlockMode(LizardIO_blockMode_t blockMode); - -/* Default setting : no block checksum */ -int LizardIO_setBlockChecksumMode(int xxhash); - -/* Default setting : stream checksum enabled */ -int LizardIO_setStreamChecksumMode(int xxhash); - -/* Default setting : 0 (no notification) */ -int LizardIO_setNotificationLevel(int level); - -/* Default setting : 0 (disabled) */ -int LizardIO_setSparseFile(int enable); - -/* Default setting : 0 (disabled) */ -int LizardIO_setContentSize(int enable); - -void LizardIO_setRemoveSrcFile(unsigned flag); - - -#endif /* LIZARDIO_H_237902873 */ diff --git a/contrib/lizard/programs/platform.h b/contrib/lizard/programs/platform.h deleted file mode 100644 index 51ce1ac2d14..00000000000 --- a/contrib/lizard/programs/platform.h +++ /dev/null @@ -1,145 +0,0 @@ -/* - platform.h - compiler and OS detection - Copyright (C) 2016-present, Przemyslaw Skibinski, Yann Collet - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. -*/ - -#ifndef PLATFORM_H_MODULE -#define PLATFORM_H_MODULE - -#if defined (__cplusplus) -extern "C" { -#endif - - - -/* ************************************** -* Compiler Options -****************************************/ -#if defined(_MSC_VER) -# define _CRT_SECURE_NO_WARNINGS /* Disable Visual Studio warning messages for fopen, strncpy, strerror */ -# define _CRT_SECURE_NO_DEPRECATE /* VS2005 - must be declared before and */ -# if (_MSC_VER <= 1800) /* (1800 = Visual Studio 2013) */ -# define snprintf sprintf_s /* snprintf unsupported by Visual <= 2013 */ -# endif -#endif - - -/* ************************************** -* Detect 64-bit OS -* http://nadeausoftware.com/articles/2012/02/c_c_tip_how_detect_processor_type_using_compiler_predefined_macros -****************************************/ -#if defined __ia64 || defined _M_IA64 /* Intel Itanium */ \ - || defined __powerpc64__ || defined __ppc64__ || defined __PPC64__ /* POWER 64-bit */ \ - || (defined __sparc && (defined __sparcv9 || defined __sparc_v9__ || defined __arch64__)) || defined __sparc64__ /* SPARC 64-bit */ \ - || defined __x86_64__s || defined _M_X64 /* x86 64-bit */ \ - || defined __arm64__ || defined __aarch64__ || defined __ARM64_ARCH_8__ /* ARM 64-bit */ \ - || (defined __mips && (__mips == 64 || __mips == 4 || __mips == 3)) /* MIPS 64-bit */ \ - || defined _LP64 || defined __LP64__ /* NetBSD, OpenBSD */ || defined __64BIT__ /* AIX */ || defined _ADDR64 /* Cray */ \ - || (defined __SIZEOF_POINTER__ && __SIZEOF_POINTER__ == 8) /* gcc */ -# if !defined(__64BIT__) -# define __64BIT__ 1 -# endif -#endif - - -/* ********************************************************* -* Turn on Large Files support (>4GB) for 32-bit Linux/Unix -***********************************************************/ -#if !defined(__64BIT__) || defined(__MINGW32__) /* No point defining Large file for 64 bit but MinGW-w64 requires it */ -# if !defined(_FILE_OFFSET_BITS) -# define _FILE_OFFSET_BITS 64 /* turn off_t into a 64-bit type for ftello, fseeko */ -# endif -# if !defined(_LARGEFILE_SOURCE) /* obsolete macro, replaced with _FILE_OFFSET_BITS */ -# define _LARGEFILE_SOURCE 1 /* Large File Support extension (LFS) - fseeko, ftello */ -# endif -# if defined(_AIX) || defined(__hpux) -# define _LARGE_FILES /* Large file support on 32-bits AIX and HP-UX */ -# endif -#endif - - -/* ************************************************************ -* Detect POSIX version -* PLATFORM_POSIX_VERSION = -1 for non-Unix e.g. Windows -* PLATFORM_POSIX_VERSION = 0 for Unix-like non-POSIX -* PLATFORM_POSIX_VERSION >= 1 is equal to found _POSIX_VERSION -***************************************************************/ -#if !defined(_WIN32) && (defined(__unix__) || defined(__unix) || (defined(__APPLE__) && defined(__MACH__)) /* UNIX-like OS */ \ - || defined(__midipix__) || defined(__VMS)) -# if (defined(__APPLE__) && defined(__MACH__)) || defined(__SVR4) || defined(_AIX) || defined(__hpux) /* POSIX.1–2001 (SUSv3) conformant */ \ - || defined(__DragonFly__) || defined(__FreeBSD__) || defined(__NetBSD__) || defined(__OpenBSD__) /* BSD distros */ -# define PLATFORM_POSIX_VERSION 200112L -# else -# if defined(__linux__) || defined(__linux) -# ifndef _POSIX_C_SOURCE -# define _POSIX_C_SOURCE 200112L /* use feature test macro */ -# endif -# endif -# include /* declares _POSIX_VERSION */ -# if defined(_POSIX_VERSION) /* POSIX compliant */ -# define PLATFORM_POSIX_VERSION _POSIX_VERSION -# else -# define PLATFORM_POSIX_VERSION 0 -# endif -# endif -#endif -#if !defined(PLATFORM_POSIX_VERSION) -# define PLATFORM_POSIX_VERSION -1 -#endif - - -/*-********************************************* -* Detect if isatty() and fileno() are available -************************************************/ -#if (defined(__linux__) && (PLATFORM_POSIX_VERSION >= 1)) || (PLATFORM_POSIX_VERSION >= 200112L) || defined(__DJGPP__) -# include /* isatty */ -# define IS_CONSOLE(stdStream) isatty(fileno(stdStream)) -#elif defined(MSDOS) || defined(OS2) || defined(WIN32) || defined(_WIN32) || defined(__CYGWIN__) -# include /* _isatty */ -# define IS_CONSOLE(stdStream) _isatty(_fileno(stdStream)) -#else -# define IS_CONSOLE(stdStream) 0 -#endif - - -/****************************** -* OS-specific Includes -******************************/ -#if defined(MSDOS) || defined(OS2) || defined(WIN32) || defined(_WIN32) -# include /* _O_BINARY */ -# include /* _setmode, _fileno, _get_osfhandle */ -# if !defined(__DJGPP__) -# include /* DeviceIoControl, HANDLE, FSCTL_SET_SPARSE */ -# include /* FSCTL_SET_SPARSE */ -# define SET_BINARY_MODE(file) { int unused=_setmode(_fileno(file), _O_BINARY); (void)unused; } -# define SET_SPARSE_FILE_MODE(file) { DWORD dw; DeviceIoControl((HANDLE) _get_osfhandle(_fileno(file)), FSCTL_SET_SPARSE, 0, 0, 0, 0, &dw, 0); } -# else -# define SET_BINARY_MODE(file) setmode(fileno(file), O_BINARY) -# define SET_SPARSE_FILE_MODE(file) -# endif -#else -# define SET_BINARY_MODE(file) -# define SET_SPARSE_FILE_MODE(file) -#endif - - - -#if defined (__cplusplus) -} -#endif - -#endif /* PLATFORM_H_MODULE */ diff --git a/contrib/lizard/programs/util.h b/contrib/lizard/programs/util.h deleted file mode 100644 index 4c2313eef00..00000000000 --- a/contrib/lizard/programs/util.h +++ /dev/null @@ -1,497 +0,0 @@ -/* - util.h - utility functions - Copyright (C) 2016-present, Przemyslaw Skibinski, Yann Collet - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. -*/ - -#ifndef UTIL_H_MODULE -#define UTIL_H_MODULE - -#if defined (__cplusplus) -extern "C" { -#endif - - - -/*-**************************************** -* Dependencies -******************************************/ -#include "platform.h" /* PLATFORM_POSIX_VERSION */ -#include /* malloc */ -#include /* size_t, ptrdiff_t */ -#include /* fprintf */ -#include /* strlen, strncpy */ -#include /* stat, utime */ -#include /* stat */ -#if defined(_MSC_VER) -# include /* utime */ -# include /* _chmod */ -#else -# include /* chown, stat */ -# include /* utime */ -#endif -#include /* time */ -#include - - - -#ifndef UTIL_WITHOUT_BASIC_TYPES -/*-************************************************************** -* Basic Types -*****************************************************************/ -#if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) ) -# include - typedef uint8_t BYTE; - typedef uint16_t U16; - typedef int16_t S16; - typedef uint32_t U32; - typedef int32_t S32; - typedef uint64_t U64; - typedef int64_t S64; -#else - typedef unsigned char BYTE; - typedef unsigned short U16; - typedef signed short S16; - typedef unsigned int U32; - typedef signed int S32; - typedef unsigned long long U64; - typedef signed long long S64; -#endif -#endif - - -/* ************************************************************ -* Avoid fseek()'s 2GiB barrier with MSVC, MacOS, *BSD, MinGW -***************************************************************/ -#if defined(_MSC_VER) && (_MSC_VER >= 1400) -# define UTIL_fseek _fseeki64 -#elif !defined(__64BIT__) && (PLATFORM_POSIX_VERSION >= 200112L) /* No point defining Large file for 64 bit */ -# define UTIL_fseek fseeko -#elif defined(__MINGW32__) && defined(__MSVCRT__) && !defined(__STRICT_ANSI__) && !defined(__NO_MINGW_LFS) -# define UTIL_fseek fseeko64 -#else -# define UTIL_fseek fseek -#endif - - -/*-**************************************** -* Sleep functions: Windows - Posix - others -******************************************/ -#if defined(_WIN32) -# include -# define SET_REALTIME_PRIORITY SetPriorityClass(GetCurrentProcess(), REALTIME_PRIORITY_CLASS) -# define UTIL_sleep(s) Sleep(1000*s) -# define UTIL_sleepMilli(milli) Sleep(milli) -#elif PLATFORM_POSIX_VERSION >= 0 /* Unix-like operating system */ -# include -# include /* setpriority */ -# include /* clock_t, nanosleep, clock, CLOCKS_PER_SEC */ -# if defined(PRIO_PROCESS) -# define SET_REALTIME_PRIORITY setpriority(PRIO_PROCESS, 0, -20) -# else -# define SET_REALTIME_PRIORITY /* disabled */ -# endif -# define UTIL_sleep(s) sleep(s) -# if (defined(__linux__) && (PLATFORM_POSIX_VERSION >= 199309L)) || (PLATFORM_POSIX_VERSION >= 200112L) /* nanosleep requires POSIX.1-2001 */ -# define UTIL_sleepMilli(milli) { struct timespec t; t.tv_sec=0; t.tv_nsec=milli*1000000ULL; nanosleep(&t, NULL); } -# else -# define UTIL_sleepMilli(milli) /* disabled */ -# endif -#else -# define SET_REALTIME_PRIORITY /* disabled */ -# define UTIL_sleep(s) /* disabled */ -# define UTIL_sleepMilli(milli) /* disabled */ -#endif - - -/* ************************************* -* Constants -***************************************/ -#define LIST_SIZE_INCREASE (8*1024) - - -/*-**************************************** -* Compiler specifics -******************************************/ -#if defined(__INTEL_COMPILER) -# pragma warning(disable : 177) /* disable: message #177: function was declared but never referenced, useful with UTIL_STATIC */ -#endif -#if defined(__GNUC__) -# define UTIL_STATIC static __attribute__((unused)) -#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) -# define UTIL_STATIC static inline -#elif defined(_MSC_VER) -# define UTIL_STATIC static __inline -#else -# define UTIL_STATIC static /* this version may generate warnings for unused static functions; disable the relevant warning */ -#endif - - -/*-**************************************** -* Time functions -******************************************/ -#if (PLATFORM_POSIX_VERSION >= 1) -#include -#include /* times */ - typedef U64 UTIL_time_t; - UTIL_STATIC void UTIL_initTimer(UTIL_time_t* ticksPerSecond) { *ticksPerSecond=sysconf(_SC_CLK_TCK); } - UTIL_STATIC void UTIL_getTime(UTIL_time_t* x) { struct tms junk; clock_t newTicks = (clock_t) times(&junk); (void)junk; *x = (UTIL_time_t)newTicks; } - UTIL_STATIC U64 UTIL_getSpanTimeMicro(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000ULL * (clockEnd - clockStart) / ticksPerSecond; } - UTIL_STATIC U64 UTIL_getSpanTimeNano(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000000ULL * (clockEnd - clockStart) / ticksPerSecond; } -#elif defined(_WIN32) /* Windows */ - typedef LARGE_INTEGER UTIL_time_t; - UTIL_STATIC void UTIL_initTimer(UTIL_time_t* ticksPerSecond) { if (!QueryPerformanceFrequency(ticksPerSecond)) fprintf(stderr, "ERROR: QueryPerformance not present\n"); } - UTIL_STATIC void UTIL_getTime(UTIL_time_t* x) { QueryPerformanceCounter(x); } - UTIL_STATIC U64 UTIL_getSpanTimeMicro(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000ULL*(clockEnd.QuadPart - clockStart.QuadPart)/ticksPerSecond.QuadPart; } - UTIL_STATIC U64 UTIL_getSpanTimeNano(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { return 1000000000ULL*(clockEnd.QuadPart - clockStart.QuadPart)/ticksPerSecond.QuadPart; } -#else /* relies on standard C (note : clock_t measurements can be wrong when using multi-threading) */ - typedef clock_t UTIL_time_t; - UTIL_STATIC void UTIL_initTimer(UTIL_time_t* ticksPerSecond) { *ticksPerSecond=0; } - UTIL_STATIC void UTIL_getTime(UTIL_time_t* x) { *x = clock(); } - UTIL_STATIC U64 UTIL_getSpanTimeMicro(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { (void)ticksPerSecond; return 1000000ULL * (clockEnd - clockStart) / CLOCKS_PER_SEC; } - UTIL_STATIC U64 UTIL_getSpanTimeNano(UTIL_time_t ticksPerSecond, UTIL_time_t clockStart, UTIL_time_t clockEnd) { (void)ticksPerSecond; return 1000000000ULL * (clockEnd - clockStart) / CLOCKS_PER_SEC; } -#endif - - -/* returns time span in microseconds */ -UTIL_STATIC U64 UTIL_clockSpanMicro( UTIL_time_t clockStart, UTIL_time_t ticksPerSecond ) -{ - UTIL_time_t clockEnd; - UTIL_getTime(&clockEnd); - return UTIL_getSpanTimeMicro(ticksPerSecond, clockStart, clockEnd); -} - - -UTIL_STATIC void UTIL_waitForNextTick(UTIL_time_t ticksPerSecond) -{ - UTIL_time_t clockStart, clockEnd; - UTIL_getTime(&clockStart); - do { - UTIL_getTime(&clockEnd); - } while (UTIL_getSpanTimeNano(ticksPerSecond, clockStart, clockEnd) == 0); -} - - - -/*-**************************************** -* File functions -******************************************/ -#if defined(_MSC_VER) - #define chmod _chmod - typedef struct __stat64 stat_t; -#else - typedef struct stat stat_t; -#endif - - -UTIL_STATIC int UTIL_setFileStat(const char *filename, stat_t *statbuf) -{ - int res = 0; - struct utimbuf timebuf; - - timebuf.actime = time(NULL); - timebuf.modtime = statbuf->st_mtime; - res += utime(filename, &timebuf); /* set access and modification times */ - -#if !defined(_WIN32) - res += chown(filename, statbuf->st_uid, statbuf->st_gid); /* Copy ownership */ -#endif - - res += chmod(filename, statbuf->st_mode & 07777); /* Copy file permissions */ - - errno = 0; - return -res; /* number of errors is returned */ -} - - -UTIL_STATIC int UTIL_getFileStat(const char* infilename, stat_t *statbuf) -{ - int r; -#if defined(_MSC_VER) - r = _stat64(infilename, statbuf); - if (r || !(statbuf->st_mode & S_IFREG)) return 0; /* No good... */ -#else - r = stat(infilename, statbuf); - if (r || !S_ISREG(statbuf->st_mode)) return 0; /* No good... */ -#endif - return 1; -} - - -UTIL_STATIC int UTIL_isRegFile(const char* infilename) -{ - stat_t statbuf; - return UTIL_getFileStat(infilename, &statbuf); /* Only need to know whether it is a regular file */ -} - - -UTIL_STATIC U32 UTIL_isDirectory(const char* infilename) -{ - int r; - stat_t statbuf; -#if defined(_MSC_VER) - r = _stat64(infilename, &statbuf); - if (!r && (statbuf.st_mode & _S_IFDIR)) return 1; -#else - r = stat(infilename, &statbuf); - if (!r && S_ISDIR(statbuf.st_mode)) return 1; -#endif - return 0; -} - - -UTIL_STATIC U64 UTIL_getFileSize(const char* infilename) -{ - int r; -#if defined(_MSC_VER) - struct __stat64 statbuf; - r = _stat64(infilename, &statbuf); - if (r || !(statbuf.st_mode & S_IFREG)) return 0; /* No good... */ -#elif defined(__MINGW32__) && defined (__MSVCRT__) - struct _stati64 statbuf; - r = _stati64(infilename, &statbuf); - if (r || !(statbuf.st_mode & S_IFREG)) return 0; /* No good... */ -#else - struct stat statbuf; - r = stat(infilename, &statbuf); - if (r || !S_ISREG(statbuf.st_mode)) return 0; /* No good... */ -#endif - return (U64)statbuf.st_size; -} - - -UTIL_STATIC U64 UTIL_getTotalFileSize(const char** fileNamesTable, unsigned nbFiles) -{ - U64 total = 0; - unsigned n; - for (n=0; n= *bufEnd) { - ptrdiff_t newListSize = (*bufEnd - *bufStart) + LIST_SIZE_INCREASE; - *bufStart = (char*)UTIL_realloc(*bufStart, newListSize); - *bufEnd = *bufStart + newListSize; - if (*bufStart == NULL) { free(path); FindClose(hFile); return 0; } - } - if (*bufStart + *pos + pathLength < *bufEnd) { - strncpy(*bufStart + *pos, path, *bufEnd - (*bufStart + *pos)); - *pos += pathLength + 1; - nbFiles++; - } - } - free(path); - } while (FindNextFileA(hFile, &cFile)); - - FindClose(hFile); - return nbFiles; -} - -#elif defined(__linux__) || (PLATFORM_POSIX_VERSION >= 200112L) /* opendir, readdir require POSIX.1-2001 */ -# define UTIL_HAS_CREATEFILELIST -# include /* opendir, readdir */ -# include /* strerror, memcpy */ - -UTIL_STATIC int UTIL_prepareFileList(const char *dirName, char** bufStart, size_t* pos, char** bufEnd) -{ - DIR *dir; - struct dirent *entry; - char* path; - int dirLength, fnameLength, pathLength, nbFiles = 0; - - if (!(dir = opendir(dirName))) { - fprintf(stderr, "Cannot open directory '%s': %s\n", dirName, strerror(errno)); - return 0; - } - - dirLength = (int)strlen(dirName); - errno = 0; - while ((entry = readdir(dir)) != NULL) { - if (strcmp (entry->d_name, "..") == 0 || - strcmp (entry->d_name, ".") == 0) continue; - fnameLength = (int)strlen(entry->d_name); - path = (char*) malloc(dirLength + fnameLength + 2); - if (!path) { closedir(dir); return 0; } - memcpy(path, dirName, dirLength); - path[dirLength] = '/'; - memcpy(path+dirLength+1, entry->d_name, fnameLength); - pathLength = dirLength+1+fnameLength; - path[pathLength] = 0; - - if (UTIL_isDirectory(path)) { - nbFiles += UTIL_prepareFileList(path, bufStart, pos, bufEnd); /* Recursively call "UTIL_prepareFileList" with the new path. */ - if (*bufStart == NULL) { free(path); closedir(dir); return 0; } - } else { - if (*bufStart + *pos + pathLength >= *bufEnd) { - ptrdiff_t newListSize = (*bufEnd - *bufStart) + LIST_SIZE_INCREASE; - *bufStart = (char*)UTIL_realloc(*bufStart, newListSize); - *bufEnd = *bufStart + newListSize; - if (*bufStart == NULL) { free(path); closedir(dir); return 0; } - } - if (*bufStart + *pos + pathLength < *bufEnd) { - strncpy(*bufStart + *pos, path, *bufEnd - (*bufStart + *pos)); - *pos += pathLength + 1; - nbFiles++; - } - } - free(path); - errno = 0; /* clear errno after UTIL_isDirectory, UTIL_prepareFileList */ - } - - if (errno != 0) { - fprintf(stderr, "readdir(%s) error: %s\n", dirName, strerror(errno)); - free(*bufStart); - *bufStart = NULL; - } - closedir(dir); - return nbFiles; -} - -#else - -UTIL_STATIC int UTIL_prepareFileList(const char *dirName, char** bufStart, size_t* pos, char** bufEnd) -{ - (void)bufStart; (void)bufEnd; (void)pos; - fprintf(stderr, "Directory %s ignored (compiled without _WIN32 or _POSIX_C_SOURCE)\n", dirName); - return 0; -} - -#endif /* #ifdef _WIN32 */ - -/* - * UTIL_createFileList - takes a list of files and directories (params: inputNames, inputNamesNb), scans directories, - * and returns a new list of files (params: return value, allocatedBuffer, allocatedNamesNb). - * After finishing usage of the list the structures should be freed with UTIL_freeFileList(params: return value, allocatedBuffer) - * In case of error UTIL_createFileList returns NULL and UTIL_freeFileList should not be called. - */ -UTIL_STATIC const char** UTIL_createFileList(const char **inputNames, unsigned inputNamesNb, char** allocatedBuffer, unsigned* allocatedNamesNb) -{ - size_t pos; - unsigned i, nbFiles; - char* buf = (char*)malloc(LIST_SIZE_INCREASE); - char* bufend = buf + LIST_SIZE_INCREASE; - const char** fileTable; - - if (!buf) return NULL; - - for (i=0, pos=0, nbFiles=0; i= bufend) { - ptrdiff_t newListSize = (bufend - buf) + LIST_SIZE_INCREASE; - buf = (char*)UTIL_realloc(buf, newListSize); - bufend = buf + newListSize; - if (!buf) return NULL; - } - if (buf + pos + len < bufend) { - strncpy(buf + pos, inputNames[i], bufend - (buf + pos)); - pos += len + 1; - nbFiles++; - } - } else { - nbFiles += UTIL_prepareFileList(inputNames[i], &buf, &pos, &bufend); - if (buf == NULL) return NULL; - } } - - if (nbFiles == 0) { free(buf); return NULL; } - - fileTable = (const char**)malloc((nbFiles+1) * sizeof(const char*)); - if (!fileTable) { free(buf); return NULL; } - - for (i=0, pos=0; i bufend) { free(buf); free((void*)fileTable); return NULL; } - - *allocatedBuffer = buf; - *allocatedNamesNb = nbFiles; - - return fileTable; -} - - -UTIL_STATIC void UTIL_freeFileList(const char** filenameTable, char* allocatedBuffer) -{ - if (allocatedBuffer) free(allocatedBuffer); - if (filenameTable) free((void*)filenameTable); -} - - -#if defined (__cplusplus) -} -#endif - -#endif /* UTIL_H_MODULE */ diff --git a/contrib/lizard/tests/.gitignore b/contrib/lizard/tests/.gitignore deleted file mode 100644 index 06891acf794..00000000000 --- a/contrib/lizard/tests/.gitignore +++ /dev/null @@ -1,13 +0,0 @@ - -# test build artefacts -datagen -frametest -frametest32 -fullbench -fullbench32 -fuzzer -fuzzer32 -fasttest - -# test artefacts -tmp* diff --git a/contrib/lizard/tests/COPYING b/contrib/lizard/tests/COPYING deleted file mode 100644 index d159169d105..00000000000 --- a/contrib/lizard/tests/COPYING +++ /dev/null @@ -1,339 +0,0 @@ - GNU GENERAL PUBLIC LICENSE - Version 2, June 1991 - - Copyright (C) 1989, 1991 Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The licenses for most software are designed to take away your -freedom to share and change it. By contrast, the GNU General Public -License is intended to guarantee your freedom to share and change free -software--to make sure the software is free for all its users. This -General Public License applies to most of the Free Software -Foundation's software and to any other program whose authors commit to -using it. (Some other Free Software Foundation software is covered by -the GNU Lesser General Public License instead.) You can apply it to -your programs, too. - - When we speak of free software, we are referring to freedom, not -price. Our General Public Licenses are designed to make sure that you -have the freedom to distribute copies of free software (and charge for -this service if you wish), that you receive source code or can get it -if you want it, that you can change the software or use pieces of it -in new free programs; and that you know you can do these things. - - To protect your rights, we need to make restrictions that forbid -anyone to deny you these rights or to ask you to surrender the rights. -These restrictions translate to certain responsibilities for you if you -distribute copies of the software, or if you modify it. - - For example, if you distribute copies of such a program, whether -gratis or for a fee, you must give the recipients all the rights that -you have. You must make sure that they, too, receive or can get the -source code. And you must show them these terms so they know their -rights. - - We protect your rights with two steps: (1) copyright the software, and -(2) offer you this license which gives you legal permission to copy, -distribute and/or modify the software. - - Also, for each author's protection and ours, we want to make certain -that everyone understands that there is no warranty for this free -software. If the software is modified by someone else and passed on, we -want its recipients to know that what they have is not the original, so -that any problems introduced by others will not reflect on the original -authors' reputations. - - Finally, any free program is threatened constantly by software -patents. We wish to avoid the danger that redistributors of a free -program will individually obtain patent licenses, in effect making the -program proprietary. To prevent this, we have made it clear that any -patent must be licensed for everyone's free use or not licensed at all. - - The precise terms and conditions for copying, distribution and -modification follow. - - GNU GENERAL PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License applies to any program or other work which contains -a notice placed by the copyright holder saying it may be distributed -under the terms of this General Public License. The "Program", below, -refers to any such program or work, and a "work based on the Program" -means either the Program or any derivative work under copyright law: -that is to say, a work containing the Program or a portion of it, -either verbatim or with modifications and/or translated into another -language. (Hereinafter, translation is included without limitation in -the term "modification".) Each licensee is addressed as "you". - -Activities other than copying, distribution and modification are not -covered by this License; they are outside its scope. The act of -running the Program is not restricted, and the output from the Program -is covered only if its contents constitute a work based on the -Program (independent of having been made by running the Program). -Whether that is true depends on what the Program does. - - 1. You may copy and distribute verbatim copies of the Program's -source code as you receive it, in any medium, provided that you -conspicuously and appropriately publish on each copy an appropriate -copyright notice and disclaimer of warranty; keep intact all the -notices that refer to this License and to the absence of any warranty; -and give any other recipients of the Program a copy of this License -along with the Program. - -You may charge a fee for the physical act of transferring a copy, and -you may at your option offer warranty protection in exchange for a fee. - - 2. You may modify your copy or copies of the Program or any portion -of it, thus forming a work based on the Program, and copy and -distribute such modifications or work under the terms of Section 1 -above, provided that you also meet all of these conditions: - - a) You must cause the modified files to carry prominent notices - stating that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in - whole or in part contains or is derived from the Program or any - part thereof, to be licensed as a whole at no charge to all third - parties under the terms of this License. - - c) If the modified program normally reads commands interactively - when run, you must cause it, when started running for such - interactive use in the most ordinary way, to print or display an - announcement including an appropriate copyright notice and a - notice that there is no warranty (or else, saying that you provide - a warranty) and that users may redistribute the program under - these conditions, and telling the user how to view a copy of this - License. (Exception: if the Program itself is interactive but - does not normally print such an announcement, your work based on - the Program is not required to print an announcement.) - -These requirements apply to the modified work as a whole. If -identifiable sections of that work are not derived from the Program, -and can be reasonably considered independent and separate works in -themselves, then this License, and its terms, do not apply to those -sections when you distribute them as separate works. But when you -distribute the same sections as part of a whole which is a work based -on the Program, the distribution of the whole must be on the terms of -this License, whose permissions for other licensees extend to the -entire whole, and thus to each and every part regardless of who wrote it. - -Thus, it is not the intent of this section to claim rights or contest -your rights to work written entirely by you; rather, the intent is to -exercise the right to control the distribution of derivative or -collective works based on the Program. - -In addition, mere aggregation of another work not based on the Program -with the Program (or with a work based on the Program) on a volume of -a storage or distribution medium does not bring the other work under -the scope of this License. - - 3. You may copy and distribute the Program (or a work based on it, -under Section 2) in object code or executable form under the terms of -Sections 1 and 2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable - source code, which must be distributed under the terms of Sections - 1 and 2 above on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three - years, to give any third party, for a charge no more than your - cost of physically performing source distribution, a complete - machine-readable copy of the corresponding source code, to be - distributed under the terms of Sections 1 and 2 above on a medium - customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer - to distribute corresponding source code. (This alternative is - allowed only for noncommercial distribution and only if you - received the program in object code or executable form with such - an offer, in accord with Subsection b above.) - -The source code for a work means the preferred form of the work for -making modifications to it. For an executable work, complete source -code means all the source code for all modules it contains, plus any -associated interface definition files, plus the scripts used to -control compilation and installation of the executable. However, as a -special exception, the source code distributed need not include -anything that is normally distributed (in either source or binary -form) with the major components (compiler, kernel, and so on) of the -operating system on which the executable runs, unless that component -itself accompanies the executable. - -If distribution of executable or object code is made by offering -access to copy from a designated place, then offering equivalent -access to copy the source code from the same place counts as -distribution of the source code, even though third parties are not -compelled to copy the source along with the object code. - - 4. You may not copy, modify, sublicense, or distribute the Program -except as expressly provided under this License. Any attempt -otherwise to copy, modify, sublicense or distribute the Program is -void, and will automatically terminate your rights under this License. -However, parties who have received copies, or rights, from you under -this License will not have their licenses terminated so long as such -parties remain in full compliance. - - 5. You are not required to accept this License, since you have not -signed it. However, nothing else grants you permission to modify or -distribute the Program or its derivative works. These actions are -prohibited by law if you do not accept this License. Therefore, by -modifying or distributing the Program (or any work based on the -Program), you indicate your acceptance of this License to do so, and -all its terms and conditions for copying, distributing or modifying -the Program or works based on it. - - 6. Each time you redistribute the Program (or any work based on the -Program), the recipient automatically receives a license from the -original licensor to copy, distribute or modify the Program subject to -these terms and conditions. You may not impose any further -restrictions on the recipients' exercise of the rights granted herein. -You are not responsible for enforcing compliance by third parties to -this License. - - 7. If, as a consequence of a court judgment or allegation of patent -infringement or for any other reason (not limited to patent issues), -conditions are imposed on you (whether by court order, agreement or -otherwise) that contradict the conditions of this License, they do not -excuse you from the conditions of this License. If you cannot -distribute so as to satisfy simultaneously your obligations under this -License and any other pertinent obligations, then as a consequence you -may not distribute the Program at all. For example, if a patent -license would not permit royalty-free redistribution of the Program by -all those who receive copies directly or indirectly through you, then -the only way you could satisfy both it and this License would be to -refrain entirely from distribution of the Program. - -If any portion of this section is held invalid or unenforceable under -any particular circumstance, the balance of the section is intended to -apply and the section as a whole is intended to apply in other -circumstances. - -It is not the purpose of this section to induce you to infringe any -patents or other property right claims or to contest validity of any -such claims; this section has the sole purpose of protecting the -integrity of the free software distribution system, which is -implemented by public license practices. Many people have made -generous contributions to the wide range of software distributed -through that system in reliance on consistent application of that -system; it is up to the author/donor to decide if he or she is willing -to distribute software through any other system and a licensee cannot -impose that choice. - -This section is intended to make thoroughly clear what is believed to -be a consequence of the rest of this License. - - 8. If the distribution and/or use of the Program is restricted in -certain countries either by patents or by copyrighted interfaces, the -original copyright holder who places the Program under this License -may add an explicit geographical distribution limitation excluding -those countries, so that distribution is permitted only in or among -countries not thus excluded. In such case, this License incorporates -the limitation as if written in the body of this License. - - 9. The Free Software Foundation may publish revised and/or new versions -of the General Public License from time to time. Such new versions will -be similar in spirit to the present version, but may differ in detail to -address new problems or concerns. - -Each version is given a distinguishing version number. If the Program -specifies a version number of this License which applies to it and "any -later version", you have the option of following the terms and conditions -either of that version or of any later version published by the Free -Software Foundation. If the Program does not specify a version number of -this License, you may choose any version ever published by the Free Software -Foundation. - - 10. If you wish to incorporate parts of the Program into other free -programs whose distribution conditions are different, write to the author -to ask for permission. For software which is copyrighted by the Free -Software Foundation, write to the Free Software Foundation; we sometimes -make exceptions for this. Our decision will be guided by the two goals -of preserving the free status of all derivatives of our free software and -of promoting the sharing and reuse of software generally. - - NO WARRANTY - - 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY -FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN -OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES -PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED -OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF -MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS -TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE -PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, -REPAIR OR CORRECTION. - - 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING -WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR -REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, -INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING -OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED -TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY -YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER -PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE -POSSIBILITY OF SUCH DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest -possible use to the public, the best way to achieve this is to make it -free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest -to attach them to the start of each source file to most effectively -convey the exclusion of warranty; and each file should have at least -the "copyright" line and a pointer to where the full notice is found. - - - Copyright (C) - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - -Also add information on how to contact you by electronic and paper mail. - -If the program is interactive, make it output a short notice like this -when it starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author - Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. - This is free software, and you are welcome to redistribute it - under certain conditions; type `show c' for details. - -The hypothetical commands `show w' and `show c' should show the appropriate -parts of the General Public License. Of course, the commands you use may -be called something other than `show w' and `show c'; they could even be -mouse-clicks or menu items--whatever suits your program. - -You should also get your employer (if you work as a programmer) or your -school, if any, to sign a "copyright disclaimer" for the program, if -necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the program - `Gnomovision' (which makes passes at compilers) written by James Hacker. - - , 1 April 1989 - Ty Coon, President of Vice - -This General Public License does not permit incorporating your program into -proprietary programs. If your program is a subroutine library, you may -consider it more useful to permit linking proprietary applications with the -library. If this is what you want to do, use the GNU Lesser General -Public License instead of this License. diff --git a/contrib/lizard/tests/README.md b/contrib/lizard/tests/README.md deleted file mode 100644 index 4dc2edc4396..00000000000 --- a/contrib/lizard/tests/README.md +++ /dev/null @@ -1,15 +0,0 @@ -Programs and scripts for automated testing of Lizard -======================================================= - -This directory contains the following programs and scripts: -- `datagen` : Synthetic and parametrable data generator, for tests -- `frametest` : Test tool that checks lizard_frame integrity on target platform -- `fullbench` : Precisely measure speed for each lizard inner functions -- `fuzzer` : Test tool, to check lizard integrity on target platform - - -#### License - -All files in this directory are licensed under GPL-v2. -See [COPYING](COPYING) for details. -The text of the license is also included at the top of each source file. diff --git a/contrib/lizard/tests/datagencli.c b/contrib/lizard/tests/datagencli.c deleted file mode 100644 index 47c79d31522..00000000000 --- a/contrib/lizard/tests/datagencli.c +++ /dev/null @@ -1,192 +0,0 @@ -/* - datagencli.c - compressible data command line generator - Copyright (C) Yann Collet 2012-2015 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -/************************************** -* Includes -**************************************/ -#include /* fprintf, stderr */ -#include "datagen.h" /* RDG_generate */ - - -/************************************** -* Basic Types -**************************************/ -#if defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */ -# include - typedef uint8_t BYTE; - typedef uint16_t U16; - typedef uint32_t U32; - typedef int32_t S32; - typedef uint64_t U64; -#else - typedef unsigned char BYTE; - typedef unsigned short U16; - typedef unsigned int U32; - typedef signed int S32; - typedef unsigned long long U64; -#endif - - -/************************************** -* Constants -**************************************/ -#ifndef ZSTD_VERSION -# define ZSTD_VERSION "r1" -#endif - -#define KB *(1 <<10) -#define MB *(1 <<20) -#define GB *(1U<<30) - -#define SIZE_DEFAULT (64 KB) -#define SEED_DEFAULT 0 -#define COMPRESSIBILITY_DEFAULT 50 - - -/************************************** -* Macros -**************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define DISPLAYLEVEL(l, ...) if (displayLevel>=l) { DISPLAY(__VA_ARGS__); } -static unsigned displayLevel = 2; - - -/********************************************************* -* Command line -*********************************************************/ -static int usage(char* programName) -{ - DISPLAY( "Compressible data generator\n"); - DISPLAY( "Usage :\n"); - DISPLAY( " %s [size] [args]\n", programName); - DISPLAY( "\n"); - DISPLAY( "Arguments :\n"); - DISPLAY( " -g# : generate # data (default:%i)\n", SIZE_DEFAULT); - DISPLAY( " -s# : Select seed (default:%i)\n", SEED_DEFAULT); - DISPLAY( " -P# : Select compressibility in %% (default:%i%%)\n", COMPRESSIBILITY_DEFAULT); - DISPLAY( " -h : display help and exit\n"); - DISPLAY( "Special values :\n"); - DISPLAY( " -P0 : generate incompressible noise\n"); - DISPLAY( " -P100 : generate sparse files\n"); - return 0; -} - - -int main(int argc, char** argv) -{ - int argNb; - double proba = (double)COMPRESSIBILITY_DEFAULT / 100; - double litProba = 0.0; - U64 size = SIZE_DEFAULT; - U32 seed = SEED_DEFAULT; - char* programName; - - /* Check command line */ - programName = argv[0]; - for(argNb=1; argNb='0') && (*argument<='9')) - { - size *= 10; - size += *argument - '0'; - argument++; - } - if (*argument=='K') { size <<= 10; argument++; } - if (*argument=='M') { size <<= 20; argument++; } - if (*argument=='G') { size <<= 30; argument++; } - if (*argument=='B') { argument++; } - break; - case 's': - argument++; - seed=0; - while ((*argument>='0') && (*argument<='9')) - { - seed *= 10; - seed += *argument - '0'; - argument++; - } - break; - case 'P': - argument++; - proba=0.0; - while ((*argument>='0') && (*argument<='9')) - { - proba *= 10; - proba += *argument - '0'; - argument++; - } - if (proba>100.) proba=100.; - proba /= 100.; - break; - case 'L': /* hidden argument : Literal distribution probability */ - argument++; - litProba=0.; - while ((*argument>='0') && (*argument<='9')) - { - litProba *= 10; - litProba += *argument - '0'; - argument++; - } - if (litProba>100.) litProba=100.; - litProba /= 100.; - break; - case 'v': - displayLevel = 4; - argument++; - break; - default: - return usage(programName); - } - } - - } - } - - DISPLAYLEVEL(4, "Data Generator %s \n", ZSTD_VERSION); - DISPLAYLEVEL(3, "Seed = %u \n", seed); - if (proba!=COMPRESSIBILITY_DEFAULT) DISPLAYLEVEL(3, "Compressibility : %i%%\n", (U32)(proba*100)); - - RDG_genOut(size, proba, litProba, seed); - DISPLAYLEVEL(1, "\n"); - - return 0; -} diff --git a/contrib/lizard/tests/frametest.c b/contrib/lizard/tests/frametest.c deleted file mode 100644 index 8fe64869fa4..00000000000 --- a/contrib/lizard/tests/frametest.c +++ /dev/null @@ -1,866 +0,0 @@ -/* - frameTest - test tool for lizard_frame - Copyright (C) Yann Collet 2014-2016 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -/*-************************************ -* Compiler specific -**************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4146) /* disable: C4146: minus unsigned expression */ -#endif - - -/*-************************************ -* Includes -**************************************/ -#include /* malloc, free */ -#include /* fprintf */ -#include /* strcmp */ -#include /* clock_t, clock(), CLOCKS_PER_SEC */ -#include "lizard_frame_static.h" -#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ -#define XXH_STATIC_LINKING_ONLY -#include "xxhash/xxhash.h" /* XXH64 */ - - -/*-************************************ -* Basic Types -**************************************/ -#if defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */ -# include -typedef uint8_t BYTE; -typedef uint16_t U16; -typedef uint32_t U32; -typedef int32_t S32; -typedef uint64_t U64; -#else -typedef unsigned char BYTE; -typedef unsigned short U16; -typedef unsigned int U32; -typedef signed int S32; -typedef unsigned long long U64; -#endif - - -/* unoptimized version; solves endianess & alignment issues */ -static void FUZ_writeLE32 (void* dstVoidPtr, U32 value32) -{ - BYTE* dstPtr = (BYTE*)dstVoidPtr; - dstPtr[0] = (BYTE)value32; - dstPtr[1] = (BYTE)(value32 >> 8); - dstPtr[2] = (BYTE)(value32 >> 16); - dstPtr[3] = (BYTE)(value32 >> 24); -} - - -/*-************************************ -* Constants -**************************************/ -#define LIZARDF_MAGIC_SKIPPABLE_START 0x184D2A50U - -#define KB *(1U<<10) -#define MB *(1U<<20) -#define GB *(1U<<30) - -static const U32 nbTestsDefault = 256 KB; -#define COMPRESSIBLE_NOISE_LENGTH (2 MB) -#define FUZ_COMPRESSIBILITY_DEFAULT 50 -static const U32 prime1 = 2654435761U; -static const U32 prime2 = 2246822519U; - - - -/*-************************************ -* Macros -**************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define DISPLAYLEVEL(l, ...) if (displayLevel>=l) { DISPLAY(__VA_ARGS__); } -#define DISPLAYUPDATE(l, ...) if (displayLevel>=l) { \ - if ((FUZ_GetClockSpan(g_clockTime) > refreshRate) || (displayLevel>=4)) \ - { g_clockTime = clock(); DISPLAY(__VA_ARGS__); \ - if (displayLevel>=4) fflush(stdout); } } -static const clock_t refreshRate = CLOCKS_PER_SEC / 6; -static clock_t g_clockTime = 0; - - -/*-*************************************** -* Local Parameters -*****************************************/ -static U32 no_prompt = 0; -static char* programName; -static U32 displayLevel = 2; -static U32 pause = 0; - - -/*-******************************************************* -* Fuzzer functions -*********************************************************/ -static clock_t FUZ_GetClockSpan(clock_t clockStart) -{ - return clock() - clockStart; /* works even if overflow; max span ~ 30 mn */ -} - - -#define FUZ_rotl32(x,r) ((x << r) | (x >> (32 - r))) -unsigned int FUZ_rand(unsigned int* src) -{ - U32 rand32 = *src; - rand32 *= prime1; - rand32 += prime2; - rand32 = FUZ_rotl32(rand32, 13); - *src = rand32; - return rand32 >> 5; -} - - -#define FUZ_RAND15BITS (FUZ_rand(seed) & 0x7FFF) -#define FUZ_RANDLENGTH ( (FUZ_rand(seed) & 3) ? (FUZ_rand(seed) % 15) : (FUZ_rand(seed) % 510) + 15) -static void FUZ_fillCompressibleNoiseBuffer(void* buffer, unsigned bufferSize, double proba, U32* seed) -{ - BYTE* BBuffer = (BYTE*)buffer; - unsigned pos = 0; - U32 P32 = (U32)(32768 * proba); - - /* First Byte */ - BBuffer[pos++] = (BYTE)(FUZ_rand(seed)); - - while (pos < bufferSize) { - /* Select : Literal (noise) or copy (within 64K) */ - if (FUZ_RAND15BITS < P32) { - /* Copy (within 64K) */ - unsigned match, end; - unsigned length = FUZ_RANDLENGTH + 4; - unsigned offset = FUZ_RAND15BITS + 1; - if (offset > pos) offset = pos; - if (pos + length > bufferSize) length = bufferSize - pos; - match = pos - offset; - end = pos + length; - while (pos < end) BBuffer[pos++] = BBuffer[match++]; - } else { - /* Literal (noise) */ - unsigned end; - unsigned length = FUZ_RANDLENGTH; - if (pos + length > bufferSize) length = bufferSize - pos; - end = pos + length; - while (pos < end) BBuffer[pos++] = (BYTE)(FUZ_rand(seed) >> 5); - } - } -} - - -static unsigned FUZ_highbit(U32 v32) -{ - unsigned nbBits = 0; - if (v32==0) return 0; - while (v32) v32 >>= 1, nbBits ++; - return nbBits; -} - - -int basicTests(U32 seed, double compressibility) -{ - int testResult = 0; - void* CNBuffer; - void* compressedBuffer; - void* decodedBuffer; - U32 randState = seed; - size_t cSize, testSize; - LizardF_preferences_t prefs; - LizardF_decompressionContext_t dCtx = NULL; - LizardF_compressionContext_t cctx = NULL; - U64 crcOrig; - - /* Create compressible test buffer */ - memset(&prefs, 0, sizeof(prefs)); - CNBuffer = malloc(COMPRESSIBLE_NOISE_LENGTH); - compressedBuffer = malloc(LizardF_compressFrameBound(COMPRESSIBLE_NOISE_LENGTH, NULL)); - decodedBuffer = malloc(COMPRESSIBLE_NOISE_LENGTH); - FUZ_fillCompressibleNoiseBuffer(CNBuffer, COMPRESSIBLE_NOISE_LENGTH, compressibility, &randState); - crcOrig = XXH64(CNBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); - - /* Special case : null-content frame */ - testSize = 0; - DISPLAYLEVEL(3, "LizardF_compressFrame, compress null content : \n"); - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, NULL), CNBuffer, testSize, NULL); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed null content into a %i bytes frame \n", (int)cSize); - - DISPLAYLEVEL(3, "LizardF_createDecompressionContext \n"); - { LizardF_errorCode_t const errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) goto _output_error; } - - DISPLAYLEVEL(3, "LizardF_getFrameInfo on null-content frame (#157) \n"); - { size_t avail_in = cSize; - LizardF_frameInfo_t frame_info; - LizardF_errorCode_t const errorCode = LizardF_getFrameInfo(dCtx, &frame_info, compressedBuffer, &avail_in); - if (LizardF_isError(errorCode)) goto _output_error; - } - - DISPLAYLEVEL(3, "LizardF_freeDecompressionContext \n"); - { LizardF_errorCode_t const errorCode = LizardF_freeDecompressionContext(dCtx); - if (LizardF_isError(errorCode)) goto _output_error; } - - /* Trivial tests : one-step frame */ - testSize = COMPRESSIBLE_NOISE_LENGTH; - DISPLAYLEVEL(3, "LizardF_compressFrame, using default preferences : \n"); - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, NULL), CNBuffer, testSize, NULL); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "Decompression test : \n"); - { size_t decodedBufferSize = COMPRESSIBLE_NOISE_LENGTH; - size_t compressedBufferSize = cSize; - BYTE* op = (BYTE*)decodedBuffer; - BYTE* const oend = (BYTE*)decodedBuffer + COMPRESSIBLE_NOISE_LENGTH; - BYTE* ip = (BYTE*)compressedBuffer; - BYTE* const iend = (BYTE*)compressedBuffer + cSize; - U64 crcDest; - - LizardF_errorCode_t errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) goto _output_error; - - DISPLAYLEVEL(3, "Single Block : \n"); - errorCode = LizardF_decompress(dCtx, decodedBuffer, &decodedBufferSize, compressedBuffer, &compressedBufferSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); - if (crcDest != crcOrig) goto _output_error; - DISPLAYLEVEL(3, "Regenerated %i bytes \n", (int)decodedBufferSize); - - DISPLAYLEVEL(3, "Reusing decompression context \n"); - { size_t iSize = compressedBufferSize - 4; - const BYTE* cBuff = (const BYTE*) compressedBuffer; - size_t decResult; - DISPLAYLEVEL(3, "Missing last 4 bytes : "); - decResult = LizardF_decompress(dCtx, decodedBuffer, &decodedBufferSize, cBuff, &iSize, NULL); - if (LizardF_isError(decResult)) goto _output_error; - if (!decResult) goto _output_error; /* not finished */ - DISPLAYLEVEL(3, "indeed, request %u bytes \n", (unsigned)decResult); - cBuff += iSize; - iSize = decResult; - decResult = LizardF_decompress(dCtx, decodedBuffer, &decodedBufferSize, cBuff, &iSize, NULL); - if (decResult != 0) goto _output_error; /* should finish now */ - crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); - if (crcDest != crcOrig) goto _output_error; - } - - { size_t oSize = 0; - size_t iSize = 0; - LizardF_frameInfo_t fi; - - DISPLAYLEVEL(3, "Start by feeding 0 bytes, to get next input size : "); - errorCode = LizardF_decompress(dCtx, NULL, &oSize, ip, &iSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - DISPLAYLEVEL(3, " %u \n", (unsigned)errorCode); - - DISPLAYLEVEL(3, "get FrameInfo on null input : "); - errorCode = LizardF_getFrameInfo(dCtx, &fi, ip, &iSize); - if (errorCode != (size_t)-LizardF_ERROR_frameHeader_incomplete) goto _output_error; - DISPLAYLEVEL(3, " correctly failed : %s \n", LizardF_getErrorName(errorCode)); - - DISPLAYLEVEL(3, "get FrameInfo on not enough input : "); - iSize = 6; - errorCode = LizardF_getFrameInfo(dCtx, &fi, ip, &iSize); - if (errorCode != (size_t)-LizardF_ERROR_frameHeader_incomplete) goto _output_error; - DISPLAYLEVEL(3, " correctly failed : %s \n", LizardF_getErrorName(errorCode)); - ip += iSize; - - DISPLAYLEVEL(3, "get FrameInfo on enough input : "); - iSize = 15 - iSize; - errorCode = LizardF_getFrameInfo(dCtx, &fi, ip, &iSize); - if (LizardF_isError(errorCode)) goto _output_error; - DISPLAYLEVEL(3, " correctly decoded \n"); - ip += iSize; - } - - DISPLAYLEVEL(3, "Byte after byte : \n"); - while (ip < iend) { - size_t oSize = oend-op; - size_t iSize = 1; - errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += oSize; - ip += iSize; - } - crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); - if (crcDest != crcOrig) goto _output_error; - DISPLAYLEVEL(3, "Regenerated %u/%u bytes \n", (unsigned)(op-(BYTE*)decodedBuffer), COMPRESSIBLE_NOISE_LENGTH); - - errorCode = LizardF_freeDecompressionContext(dCtx); - if (LizardF_isError(errorCode)) goto _output_error; - } - - DISPLAYLEVEL(3, "Using 128 KB block : \n"); - prefs.frameInfo.blockSizeID = LizardF_max128KB; - prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "without checksum : \n"); - prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "Using 256 KB block : \n"); - prefs.frameInfo.blockSizeID = LizardF_max256KB; - prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "Decompression test : \n"); - { size_t decodedBufferSize = COMPRESSIBLE_NOISE_LENGTH; - unsigned maxBits = FUZ_highbit((U32)decodedBufferSize); - BYTE* op = (BYTE*)decodedBuffer; - BYTE* const oend = (BYTE*)decodedBuffer + COMPRESSIBLE_NOISE_LENGTH; - BYTE* ip = (BYTE*)compressedBuffer; - BYTE* const iend = (BYTE*)compressedBuffer + cSize; - U64 crcDest; - - LizardF_errorCode_t errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) goto _output_error; - - DISPLAYLEVEL(3, "random segment sizes : \n"); - while (ip < iend) { - unsigned nbBits = FUZ_rand(&randState) % maxBits; - size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; - //DISPLAY("%7i : + %6i\n", (int)(ip-(BYTE*)compressedBuffer), (int)iSize); - errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += oSize; - ip += iSize; - } - crcDest = XXH64(decodedBuffer, COMPRESSIBLE_NOISE_LENGTH, 1); - if (crcDest != crcOrig) goto _output_error; - DISPLAYLEVEL(3, "Regenerated %i bytes \n", (int)decodedBufferSize); - - errorCode = LizardF_freeDecompressionContext(dCtx); - if (LizardF_isError(errorCode)) goto _output_error; - } - - DISPLAYLEVEL(3, "without checksum : \n"); - prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "Using 1 MB block : \n"); - prefs.frameInfo.blockSizeID = LizardF_max1MB; - prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "without checksum : \n"); - prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "Using 4 MB block : \n"); - prefs.frameInfo.blockSizeID = LizardF_max4MB; - prefs.frameInfo.contentChecksumFlag = LizardF_contentChecksumEnabled; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - DISPLAYLEVEL(3, "without checksum : \n"); - prefs.frameInfo.contentChecksumFlag = LizardF_noContentChecksum; - cSize = LizardF_compressFrame(compressedBuffer, LizardF_compressFrameBound(testSize, &prefs), CNBuffer, testSize, &prefs); - if (LizardF_isError(cSize)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)cSize); - - { size_t errorCode; - BYTE* const ostart = (BYTE*)compressedBuffer; - BYTE* op = ostart; - errorCode = LizardF_createCompressionContext(&cctx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) goto _output_error; - - DISPLAYLEVEL(3, "compress without frameSize : \n"); - memset(&(prefs.frameInfo), 0, sizeof(prefs.frameInfo)); - errorCode = LizardF_compressBegin(cctx, compressedBuffer, testSize, &prefs); - if (LizardF_isError(errorCode)) goto _output_error; - op += errorCode; - errorCode = LizardF_compressUpdate(cctx, op, LizardF_compressBound(testSize, &prefs), CNBuffer, testSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += errorCode; - errorCode = LizardF_compressEnd(cctx, compressedBuffer, testSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)(op-ostart)); - - DISPLAYLEVEL(3, "compress with frameSize : \n"); - prefs.frameInfo.contentSize = testSize; - op = ostart; - errorCode = LizardF_compressBegin(cctx, compressedBuffer, testSize, &prefs); - if (LizardF_isError(errorCode)) goto _output_error; - op += errorCode; - errorCode = LizardF_compressUpdate(cctx, op, LizardF_compressBound(testSize, &prefs), CNBuffer, testSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += errorCode; - errorCode = LizardF_compressEnd(cctx, compressedBuffer, testSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - DISPLAYLEVEL(3, "Compressed %i bytes into a %i bytes frame \n", (int)testSize, (int)(op-ostart)); - - DISPLAYLEVEL(3, "compress with wrong frameSize : \n"); - prefs.frameInfo.contentSize = testSize+1; - op = ostart; - errorCode = LizardF_compressBegin(cctx, compressedBuffer, testSize, &prefs); - if (LizardF_isError(errorCode)) goto _output_error; - op += errorCode; - errorCode = LizardF_compressUpdate(cctx, op, LizardF_compressBound(testSize, &prefs), CNBuffer, testSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += errorCode; - errorCode = LizardF_compressEnd(cctx, op, testSize, NULL); - if (LizardF_isError(errorCode)) { DISPLAYLEVEL(3, "Error correctly detected : %s \n", LizardF_getErrorName(errorCode)); } - else - goto _output_error; - - errorCode = LizardF_freeCompressionContext(cctx); - if (LizardF_isError(errorCode)) goto _output_error; - cctx = NULL; - } - - DISPLAYLEVEL(3, "Skippable frame test : \n"); - { size_t decodedBufferSize = COMPRESSIBLE_NOISE_LENGTH; - unsigned maxBits = FUZ_highbit((U32)decodedBufferSize); - BYTE* op = (BYTE*)decodedBuffer; - BYTE* const oend = (BYTE*)decodedBuffer + COMPRESSIBLE_NOISE_LENGTH; - BYTE* ip = (BYTE*)compressedBuffer; - BYTE* iend = (BYTE*)compressedBuffer + cSize + 8; - - LizardF_errorCode_t errorCode = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) goto _output_error; - - /* generate skippable frame */ - FUZ_writeLE32(ip, LIZARDF_MAGIC_SKIPPABLE_START); - FUZ_writeLE32(ip+4, (U32)cSize); - - DISPLAYLEVEL(3, "random segment sizes : \n"); - while (ip < iend) { - unsigned nbBits = FUZ_rand(&randState) % maxBits; - size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; - errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += oSize; - ip += iSize; - } - DISPLAYLEVEL(3, "Skipped %i bytes \n", (int)decodedBufferSize); - - /* generate zero-size skippable frame */ - DISPLAYLEVEL(3, "zero-size skippable frame\n"); - ip = (BYTE*)compressedBuffer; - op = (BYTE*)decodedBuffer; - FUZ_writeLE32(ip, LIZARDF_MAGIC_SKIPPABLE_START+1); - FUZ_writeLE32(ip+4, 0); - iend = ip+8; - - while (ip < iend) { - unsigned nbBits = FUZ_rand(&randState) % maxBits; - size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; - errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += oSize; - ip += iSize; - } - DISPLAYLEVEL(3, "Skipped %i bytes \n", (int)(ip - (BYTE*)compressedBuffer - 8)); - - DISPLAYLEVEL(3, "Skippable frame header complete in first call \n"); - ip = (BYTE*)compressedBuffer; - op = (BYTE*)decodedBuffer; - FUZ_writeLE32(ip, LIZARDF_MAGIC_SKIPPABLE_START+2); - FUZ_writeLE32(ip+4, 10); - iend = ip+18; - while (ip < iend) { - size_t iSize = 10; - size_t oSize = 10; - if (iSize > (size_t)(iend-ip)) iSize = iend-ip; - errorCode = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, NULL); - if (LizardF_isError(errorCode)) goto _output_error; - op += oSize; - ip += iSize; - } - DISPLAYLEVEL(3, "Skipped %i bytes \n", (int)(ip - (BYTE*)compressedBuffer - 8)); - } - - DISPLAY("Basic tests completed \n"); -_end: - free(CNBuffer); - free(compressedBuffer); - free(decodedBuffer); - LizardF_freeDecompressionContext(dCtx); dCtx = NULL; - LizardF_freeCompressionContext(cctx); cctx = NULL; - return testResult; - -_output_error: - testResult = 1; - DISPLAY("Error detected ! \n"); - goto _end; -} - - -static void locateBuffDiff(const void* buff1, const void* buff2, size_t size, unsigned nonContiguous) -{ - int p=0; - const BYTE* b1=(const BYTE*)buff1; - const BYTE* b2=(const BYTE*)buff2; - if (nonContiguous) { - DISPLAY("Non-contiguous output test (%i bytes)\n", (int)size); - return; - } - while (b1[p]==b2[p]) p++; - DISPLAY("Error at pos %i/%i : %02X != %02X \n", p, (int)size, b1[p], b2[p]); -} - - -static const U32 srcDataLength = 9 MB; /* needs to be > 2x4MB to test large blocks */ - -int fuzzerTests(U32 seed, unsigned nbTests, unsigned startTest, double compressibility, U32 duration_s) -{ - unsigned testResult = 0; - unsigned testNb = 0; - void* srcBuffer = NULL; - void* compressedBuffer = NULL; - void* decodedBuffer = NULL; - U32 coreRand = seed; - LizardF_decompressionContext_t dCtx = NULL; - LizardF_compressionContext_t cCtx = NULL; - size_t result; - clock_t const startClock = clock(); - clock_t const clockDuration = duration_s * CLOCKS_PER_SEC; - XXH64_state_t xxh64; -# define CHECK(cond, ...) if (cond) { DISPLAY("Error => "); DISPLAY(__VA_ARGS__); \ - DISPLAY(" (seed %u, test nb %u) \n", seed, testNb); goto _output_error; } - - /* Create buffers */ - result = LizardF_createDecompressionContext(&dCtx, LIZARDF_VERSION); - CHECK(LizardF_isError(result), "Allocation failed (error %i)", (int)result); - result = LizardF_createCompressionContext(&cCtx, LIZARDF_VERSION); - CHECK(LizardF_isError(result), "Allocation failed (error %i)", (int)result); - srcBuffer = malloc(srcDataLength); - CHECK(srcBuffer==NULL, "srcBuffer Allocation failed"); - compressedBuffer = malloc(LizardF_compressFrameBound(srcDataLength, NULL)); - CHECK(compressedBuffer==NULL, "compressedBuffer Allocation failed"); - decodedBuffer = calloc(1, srcDataLength); /* calloc avoids decodedBuffer being considered "garbage" by scan-build */ - CHECK(decodedBuffer==NULL, "decodedBuffer Allocation failed"); - FUZ_fillCompressibleNoiseBuffer(srcBuffer, srcDataLength, compressibility, &coreRand); - - /* jump to requested testNb */ - for (testNb =0; (testNb < startTest); testNb++) (void)FUZ_rand(&coreRand); // sync randomizer - - /* main fuzzer test loop */ - for ( ; (testNb < nbTests) || (clockDuration > FUZ_GetClockSpan(startClock)) ; testNb++) { - U32 randState = coreRand ^ prime1; - unsigned BSId = 4 + (FUZ_rand(&randState) & 3); - unsigned BMId = FUZ_rand(&randState) & 1; - unsigned CCflag = FUZ_rand(&randState) & 1; - unsigned autoflush = (FUZ_rand(&randState) & 7) == 2; - LizardF_preferences_t prefs; - LizardF_compressOptions_t cOptions; - LizardF_decompressOptions_t dOptions; - unsigned nbBits = (FUZ_rand(&randState) % (FUZ_highbit(srcDataLength-1) - 1)) + 1; - size_t srcSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; - cOptions.stableSrc = ((FUZ_rand(&randState) & 3) == 1); - - result = LizardF_compressUpdate(cCtx, op, oSize, ip, iSize, &cOptions); - CHECK(LizardF_isError(result), "Compression failed (error %i) iSize=%d oSize=%d", (int)result, (int)iSize, (int)oSize); - op += result; - ip += iSize; - - if (forceFlush) { - result = LizardF_flush(cCtx, op, oend-op, &cOptions); - CHECK(LizardF_isError(result), "Compression flush failed (error %i)", (int)result); - op += result; - } - } - result = LizardF_compressEnd(cCtx, op, oend-op, &cOptions); - CHECK(LizardF_isError(result), "Compression completion failed (error %i)", (int)result); - op += result; - cSize = op-(BYTE*)compressedBuffer; - } - - { const BYTE* ip = (const BYTE*)compressedBuffer; - const BYTE* const iend = ip + cSize; - BYTE* op = (BYTE*)decodedBuffer; - BYTE* const oend = op + srcDataLength; - size_t totalOut = 0; - unsigned maxBits = FUZ_highbit((U32)cSize); - unsigned nonContiguousDst = (FUZ_rand(&randState) & 3) == 1; - nonContiguousDst += FUZ_rand(&randState) & nonContiguousDst; /* 0=>0; 1=>1,2 */ - XXH64_reset(&xxh64, 1); - if (maxBits < 3) maxBits = 3; - while (ip < iend) { - unsigned nbBitsI = (FUZ_rand(&randState) % (maxBits-1)) + 1; - unsigned nbBitsO = (FUZ_rand(&randState) % (maxBits)) + 1; - size_t iSize = (FUZ_rand(&randState) & ((1< (size_t)(iend-ip)) iSize = iend-ip; - if (oSize > (size_t)(oend-op)) oSize = oend-op; - dOptions.stableDst = FUZ_rand(&randState) & 1; - if (nonContiguousDst==2) dOptions.stableDst = 0; - result = LizardF_decompress(dCtx, op, &oSize, ip, &iSize, &dOptions); - if (result == (size_t)-LizardF_ERROR_contentChecksum_invalid) - locateBuffDiff((BYTE*)srcBuffer+srcStart, decodedBuffer, srcSize, nonContiguousDst); - CHECK(LizardF_isError(result), "Decompression failed (error %i:%s)", (int)result, LizardF_getErrorName((LizardF_errorCode_t)result)); - XXH64_update(&xxh64, op, (U32)oSize); - totalOut += oSize; - op += oSize; - ip += iSize; - op += nonContiguousDst; - if (nonContiguousDst==2) op = (BYTE*)decodedBuffer; /* overwritten destination */ - } - CHECK(result != 0, "Frame decompression failed (error %i)", (int)result); - if (totalOut) { /* otherwise, it's a skippable frame */ - crcDecoded = XXH64_digest(&xxh64); - if (crcDecoded != crcOrig) locateBuffDiff((BYTE*)srcBuffer+srcStart, decodedBuffer, srcSize, nonContiguousDst); - CHECK(crcDecoded != crcOrig, "Decompression corruption"); - } - } - } - - DISPLAYLEVEL(2, "\rAll tests completed \n"); - -_end: - LizardF_freeDecompressionContext(dCtx); - LizardF_freeCompressionContext(cCtx); - free(srcBuffer); - free(compressedBuffer); - free(decodedBuffer); - - if (pause) { - DISPLAY("press enter to finish \n"); - (void)getchar(); - } - return testResult; - -_output_error: - testResult = 1; - goto _end; -} - - -int FUZ_usage(void) -{ - DISPLAY( "Usage :\n"); - DISPLAY( " %s [args]\n", programName); - DISPLAY( "\n"); - DISPLAY( "Arguments :\n"); - DISPLAY( " -i# : Nb of tests (default:%u) \n", nbTestsDefault); - DISPLAY( " -T# : Duration of tests, in seconds (default: use Nb of tests) \n"); - DISPLAY( " -s# : Select seed (default:prompt user)\n"); - DISPLAY( " -t# : Select starting test number (default:0)\n"); - DISPLAY( " -p# : Select compressibility in %% (default:%i%%)\n", FUZ_COMPRESSIBILITY_DEFAULT); - DISPLAY( " -v : verbose\n"); - DISPLAY( " -h : display help and exit\n"); - return 0; -} - - -int main(int argc, char** argv) -{ - U32 seed=0; - int seedset=0; - int argNb; - int nbTests = nbTestsDefault; - int testNb = 0; - int proba = FUZ_COMPRESSIBILITY_DEFAULT; - int result=0; - U32 duration=0; - - /* Check command line */ - programName = argv[0]; - for(argNb=1; argNb='0') && (*argument<='9')) { - nbTests *= 10; - nbTests += *argument - '0'; - argument++; - } - break; - - case 'T': - argument++; - nbTests = 0; duration = 0; - for (;;) { - switch(*argument) - { - case 'm': duration *= 60; argument++; continue; - case 's': - case 'n': argument++; continue; - case '0': - case '1': - case '2': - case '3': - case '4': - case '5': - case '6': - case '7': - case '8': - case '9': duration *= 10; duration += *argument++ - '0'; continue; - } - break; - } - break; - - case 's': - argument++; - seed=0; - seedset=1; - while ((*argument>='0') && (*argument<='9')) { - seed *= 10; - seed += *argument - '0'; - argument++; - } - break; - case 't': - argument++; - testNb=0; - while ((*argument>='0') && (*argument<='9')) { - testNb *= 10; - testNb += *argument - '0'; - argument++; - } - break; - case 'P': /* compressibility % */ - argument++; - proba=0; - while ((*argument>='0') && (*argument<='9')) { - proba *= 10; - proba += *argument - '0'; - argument++; - } - if (proba<0) proba=0; - if (proba>100) proba=100; - break; - default: - ; - return FUZ_usage(); - } - } - } - } - - /* Get Seed */ - printf("Starting lizard_frame tester (%i-bits, %s)\n", (int)(sizeof(size_t)*8), LIZARD_VERSION_STRING); - - if (!seedset) { - time_t const t = time(NULL); - U32 const h = XXH32(&t, sizeof(t), 1); - seed = h % 10000; - } - printf("Seed = %u\n", seed); - if (proba!=FUZ_COMPRESSIBILITY_DEFAULT) printf("Compressibility : %i%%\n", proba); - - if (nbTests<=0) nbTests=1; - - if (testNb==0) result = basicTests(seed, ((double)proba) / 100); - if (result) return 1; - return fuzzerTests(seed, nbTests, testNb, ((double)proba) / 100, duration); -} diff --git a/contrib/lizard/tests/fullbench.c b/contrib/lizard/tests/fullbench.c deleted file mode 100644 index d9070d13345..00000000000 --- a/contrib/lizard/tests/fullbench.c +++ /dev/null @@ -1,732 +0,0 @@ -/* - bench.c - Demo program to benchmark open-source compression algorithm - Copyright (C) Yann Collet 2012-2015 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repository : https://github.com/inikep/lizard -*/ - -/************************************** -* Compiler Options -**************************************/ -/* Disable some Visual warning messages */ -#define _CRT_SECURE_NO_WARNINGS -#define _CRT_SECURE_NO_DEPRECATE /* VS2005 */ - -/* Unix Large Files support (>4GB) */ -#if (defined(__sun__) && (!defined(__LP64__))) // Sun Solaris 32-bits requires specific definitions -# define _LARGEFILE_SOURCE -# define _FILE_OFFSET_BITS 64 -#elif ! defined(__LP64__) // No point defining Large file for 64 bit -# define _LARGEFILE64_SOURCE -#endif - -// S_ISREG & gettimeofday() are not supported by MSVC -#if defined(_MSC_VER) || defined(_WIN32) -# define BMK_LEGACY_TIMER 1 -#endif - - -/************************************** -* Includes -**************************************/ -#include /* malloc, free */ -#include /* fprintf, fopen, ftello64 */ -#include /* stat64 */ -#include /* stat64 */ -#include /* strcmp */ -#include /* clock_t, clock(), CLOCKS_PER_SEC */ - -#include "lizard_compress.h" -#include "lizard_decompress.h" -#include "lizard_common.h" /* Lizard_compress_MinLevel, Lizard_createStream_MinLevel */ -#include "lizard_frame.h" - -#include "xxhash/xxhash.h" - - -/************************************** -* Compiler Options -**************************************/ -/* S_ISREG & gettimeofday() are not supported by MSVC */ -#if !defined(S_ISREG) -# define S_ISREG(x) (((x) & S_IFMT) == S_IFREG) -#endif - - - -/************************************** -* Constants -**************************************/ -#define PROGRAM_DESCRIPTION "Lizard speed analyzer" -#define AUTHOR "Yann Collet" -#define WELCOME_MESSAGE "*** %s v%s %i-bits, by %s ***\n", PROGRAM_DESCRIPTION, LIZARD_VERSION_STRING, (int)(sizeof(void*)*8), AUTHOR - -#define NBLOOPS 6 -#define TIMELOOP (CLOCKS_PER_SEC * 25 / 10) - -#define KNUTH 2654435761U -#define MAX_MEM (1920 MB) -#define DEFAULT_CHUNKSIZE (4 MB) - -#define ALL_COMPRESSORS 0 -#define ALL_DECOMPRESSORS 0 - - -/************************************** -* Local structures -**************************************/ -struct chunkParameters -{ - U32 id; - char* origBuffer; - char* compressedBuffer; - int origSize; - int compressedSize; -}; - - -/************************************** -* Macros -**************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define PROGRESS(...) g_noPrompt ? 0 : DISPLAY(__VA_ARGS__) - - -/************************************** -* Benchmark Parameters -**************************************/ -static int g_chunkSize = DEFAULT_CHUNKSIZE; -static int g_nbIterations = NBLOOPS; -static int g_pause = 0; -static int g_compressionTest = 1; -static int g_compressionAlgo = ALL_COMPRESSORS; -static int g_decompressionTest = 1; -static int g_decompressionAlgo = ALL_DECOMPRESSORS; -static int g_noPrompt = 0; - -static void BMK_setBlocksize(int bsize) -{ - g_chunkSize = bsize; - DISPLAY("-Using Block Size of %i KB-\n", g_chunkSize>>10); -} - -static void BMK_setNbIterations(int nbLoops) -{ - g_nbIterations = nbLoops; - DISPLAY("- %i iterations -\n", g_nbIterations); -} - -static void BMK_setPause(void) -{ - g_pause = 1; -} - - -/********************************************************* -* Private functions -*********************************************************/ -static clock_t BMK_GetClockSpan( clock_t clockStart ) -{ - return clock() - clockStart; /* works even if overflow; max span ~30 mn */ -} - - -static size_t BMK_findMaxMem(U64 requiredMem) -{ - size_t step = 64 MB; - BYTE* testmem=NULL; - - requiredMem = (((requiredMem >> 26) + 1) << 26); - requiredMem += 2*step; - if (requiredMem > MAX_MEM) requiredMem = MAX_MEM; - - while (!testmem) { - if (requiredMem > step) requiredMem -= step; - else requiredMem >>= 1; - testmem = (BYTE*) malloc ((size_t)requiredMem); - } - free (testmem); - - /* keep some space available */ - if (requiredMem > step) requiredMem -= step; - else requiredMem >>= 1; - - return (size_t)requiredMem; -} - - -static U64 BMK_GetFileSize(const char* infilename) -{ - int r; -#if defined(_MSC_VER) - struct _stat64 statbuf; - r = _stat64(infilename, &statbuf); -#else - struct stat statbuf; - r = stat(infilename, &statbuf); -#endif - if (r || !S_ISREG(statbuf.st_mode)) return 0; /* No good... */ - return (U64)statbuf.st_size; -} - - -/********************************************************* -* Benchmark function -*********************************************************/ -Lizard_stream_t *Lizard_stream; -static void local_Lizard_createStream(void) -{ - Lizard_stream = Lizard_resetStream_MinLevel(Lizard_stream); -} - -static int local_Lizard_saveDict(const char* in, char* out, int inSize) -{ - (void)in; - return Lizard_saveDict(Lizard_stream, out, inSize); -} - -static int local_Lizard_compress_default_large(const char* in, char* out, int inSize) -{ - return Lizard_compress_MinLevel(in, out, inSize, Lizard_compressBound(inSize)); -} - -static int local_Lizard_compress_default_small(const char* in, char* out, int inSize) -{ - return Lizard_compress_MinLevel(in, out, inSize, Lizard_compressBound(inSize)-1); -} - -static int local_Lizard_compress_withState(const char* in, char* out, int inSize) -{ - return Lizard_compress_extState_MinLevel(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)); -} - -static int local_Lizard_compress_limitedOutput_withState(const char* in, char* out, int inSize) -{ - return Lizard_compress_extState_MinLevel(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)-1); -} - -static int local_Lizard_compress_continue(const char* in, char* out, int inSize) -{ - return Lizard_compress_continue(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)); -} - -static int local_Lizard_compress_limitedOutput_continue(const char* in, char* out, int inSize) -{ - return Lizard_compress_continue(Lizard_stream, in, out, inSize, Lizard_compressBound(inSize)-1); -} - - -/* HC compression functions */ -Lizard_stream_t* Lizard_streamPtr; -static void local_Lizard_resetStream(void) -{ - Lizard_streamPtr = Lizard_resetStream(Lizard_streamPtr, 0); -} - -static int local_Lizard_saveDictHC(const char* in, char* out, int inSize) -{ - (void)in; - return Lizard_saveDict(Lizard_streamPtr, out, inSize); -} - -static int local_Lizard_compress_extState(const char* in, char* out, int inSize) -{ - return Lizard_compress_extState(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize), 0); -} - -static int local_Lizard_compress_extState_limitedOutput(const char* in, char* out, int inSize) -{ - return Lizard_compress_extState(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize)-1, 0); -} - -static int local_Lizard_compress(const char* in, char* out, int inSize) -{ - return Lizard_compress(in, out, inSize, Lizard_compressBound(inSize), 0); -} - -static int local_Lizard_compress_limitedOutput(const char* in, char* out, int inSize) -{ - return Lizard_compress(in, out, inSize, Lizard_compressBound(inSize)-1, 0); -} - -static int local_Lizard_compressHC_continue(const char* in, char* out, int inSize) -{ - return Lizard_compress_continue(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize)); -} - -static int local_Lizard_compress_continue_limitedOutput(const char* in, char* out, int inSize) -{ - return Lizard_compress_continue(Lizard_streamPtr, in, out, inSize, Lizard_compressBound(inSize)-1); -} - - -/* decompression functions */ -static int local_Lizard_decompress_safe_usingDict(const char* in, char* out, int inSize, int outSize) -{ - (void)inSize; - Lizard_decompress_safe_usingDict(in, out, inSize, outSize, out - 65536, 65536); - return outSize; -} - -extern int Lizard_decompress_safe_forceExtDict(const char* in, char* out, int inSize, int outSize, const char* dict, int dictSize); - -static int local_Lizard_decompress_safe_forceExtDict(const char* in, char* out, int inSize, int outSize) -{ - (void)inSize; - Lizard_decompress_safe_forceExtDict(in, out, inSize, outSize, out - 65536, 65536); - return outSize; -} - -static int local_Lizard_decompress_safe_partial(const char* in, char* out, int inSize, int outSize) -{ - return Lizard_decompress_safe_partial(in, out, inSize, outSize - 5, outSize); -} - - -/* frame functions */ -static int local_LizardF_compressFrame(const char* in, char* out, int inSize) -{ - return (int)LizardF_compressFrame(out, 2*inSize + 16, in, inSize, NULL); -} - -static LizardF_decompressionContext_t g_dCtx; - -static int local_LizardF_decompress(const char* in, char* out, int inSize, int outSize) -{ - size_t srcSize = inSize; - size_t dstSize = outSize; - size_t result; - result = LizardF_decompress(g_dCtx, out, &dstSize, in, &srcSize, NULL); - if (result!=0) { DISPLAY("Error decompressing frame : unfinished frame (%d)\n", (int)result); exit(8); } - if (srcSize != (size_t)inSize) { DISPLAY("Error decompressing frame : read size incorrect\n"); exit(9); } - return (int)dstSize; -} - - -#define NB_COMPRESSION_ALGORITHMS 100 -#define NB_DECOMPRESSION_ALGORITHMS 100 -int fullSpeedBench(const char** fileNamesTable, int nbFiles) -{ - int fileIdx=0; - - /* Init */ - { size_t const errorCode = LizardF_createDecompressionContext(&g_dCtx, LIZARDF_VERSION); - if (LizardF_isError(errorCode)) { DISPLAY("dctx allocation issue \n"); return 10; } } - - Lizard_streamPtr = Lizard_createStream(0); - if (!Lizard_streamPtr) { DISPLAY("Lizard_streamPtr allocation issue \n"); return 10; } - - Lizard_stream = Lizard_createStream_MinLevel(); - if (!Lizard_stream) { DISPLAY("Lizard_stream allocation issue \n"); return 10; } - - /* Loop for each fileName */ - while (fileIdx inFileSize) benchedSize = (size_t)inFileSize; - if (benchedSize < inFileSize) - DISPLAY("Not enough memory for '%s' full size; testing %i MB only...\n", inFileName, (int)(benchedSize>>20)); - - /* Allocation */ - chunkP = (struct chunkParameters*) malloc(((benchedSize / (size_t)g_chunkSize)+1) * sizeof(struct chunkParameters)); - orig_buff = (char*) malloc(benchedSize); - nbChunks = (int) ((benchedSize + (g_chunkSize-1)) / g_chunkSize); - maxCompressedChunkSize = Lizard_compressBound(g_chunkSize); - compressedBuffSize = nbChunks * maxCompressedChunkSize; - compressed_buff = (char*)malloc((size_t)compressedBuffSize); - if(!chunkP || !orig_buff || !compressed_buff) { - DISPLAY("\nError: not enough memory!\n"); - fclose(inFile); - free(orig_buff); - free(compressed_buff); - free(chunkP); - return(12); - } - - /* Fill in src buffer */ - DISPLAY("Loading %s... \r", inFileName); - readSize = fread(orig_buff, 1, benchedSize, inFile); - fclose(inFile); - - if (readSize != benchedSize) { - DISPLAY("\nError: problem reading file '%s' !! \n", inFileName); - free(orig_buff); - free(compressed_buff); - free(chunkP); - return 13; - } - - /* Calculating input Checksum */ - crcOriginal = XXH32(orig_buff, benchedSize,0); - - - /* Bench */ - { int loopNb, nb_loops, chunkNb, cAlgNb, dAlgNb; - size_t cSize=0; - double ratio=0.; - - DISPLAY("\r%79s\r", ""); - DISPLAY(" %s : \n", inFileName); - - /* Bench Compression Algorithms */ - for (cAlgNb=0; (cAlgNb <= NB_COMPRESSION_ALGORITHMS) && (g_compressionTest); cAlgNb++) { - const char* compressorName; - int (*compressionFunction)(const char*, char*, int); - void (*initFunction)(void) = NULL; - double bestTime = 100000000.; - - /* filter compressionAlgo only */ - if ((g_compressionAlgo != ALL_COMPRESSORS) && (g_compressionAlgo != cAlgNb)) continue; - - /* Init data chunks */ - { int i; - size_t remaining = benchedSize; - char* in = orig_buff; - char* out = compressed_buff; - nbChunks = (int) (((int)benchedSize + (g_chunkSize-1))/ g_chunkSize); - for (i=0; i g_chunkSize) { chunkP[i].origSize = g_chunkSize; remaining -= g_chunkSize; } else { chunkP[i].origSize = (int)remaining; remaining = 0; } - chunkP[i].compressedBuffer = out; out += maxCompressedChunkSize; - chunkP[i].compressedSize = 0; - } - } - - switch(cAlgNb) - { - case 0 : DISPLAY("Compression functions : \n"); continue; - case 1 : compressionFunction = local_Lizard_compress_default_large; compressorName = "Lizard_compress_MinLevel"; break; - case 2 : compressionFunction = local_Lizard_compress_default_small; compressorName = "Lizard_compress_MinLevel(small dst)"; break; - - case 10: compressionFunction = local_Lizard_compress; compressorName = "Lizard_compress"; break; - case 11: compressionFunction = local_Lizard_compress_limitedOutput; compressorName = "Lizard_compress limitedOutput"; break; - case 12 : compressionFunction = local_Lizard_compress_extState; compressorName = "Lizard_compress_extState"; break; - case 13: compressionFunction = local_Lizard_compress_extState_limitedOutput; compressorName = "Lizard_compress_extState limitedOutput"; break; - case 14: compressionFunction = local_Lizard_compressHC_continue; initFunction = local_Lizard_resetStream; compressorName = "Lizard_compress_continue"; break; - case 15: compressionFunction = local_Lizard_compress_continue_limitedOutput; initFunction = local_Lizard_resetStream; compressorName = "Lizard_compress_continue limitedOutput"; break; - case 30: compressionFunction = local_LizardF_compressFrame; compressorName = "LizardF_compressFrame"; - chunkP[0].origSize = (int)benchedSize; nbChunks=1; - break; - case 40: compressionFunction = local_Lizard_saveDict; compressorName = "Lizard_saveDict"; - Lizard_loadDict(Lizard_stream, chunkP[0].origBuffer, chunkP[0].origSize); - break; - case 41: compressionFunction = local_Lizard_saveDictHC; compressorName = "Lizard_saveDict"; - Lizard_loadDict(Lizard_streamPtr, chunkP[0].origBuffer, chunkP[0].origSize); - break; - case 16: compressionFunction = local_Lizard_compress_withState; compressorName = "Lizard_compress_extState_MinLevel(1)"; break; - case 17: compressionFunction = local_Lizard_compress_limitedOutput_withState; compressorName = "Lizard_compress_extState_MinLevel(1) limitedOutput"; break; - case 18: compressionFunction = local_Lizard_compress_continue; initFunction = local_Lizard_createStream; compressorName = "Lizard_compress_continue(1)"; break; - case 19: compressionFunction = local_Lizard_compress_limitedOutput_continue; initFunction = local_Lizard_createStream; compressorName = "Lizard_compress_continue(1) limitedOutput"; break; - case 60: DISPLAY("Obsolete compression functions : \n"); continue; - default : - continue; /* unknown ID : just skip */ - } - - for (loopNb = 1; loopNb <= g_nbIterations; loopNb++) { - double averageTime; - clock_t clockTime; - - PROGRESS("%1i- %-28.28s :%9i ->\r", loopNb, compressorName, (int)benchedSize); - { size_t i; for (i=0; i%9i (%5.2f%%),%7.1f MB/s\r", loopNb, compressorName, (int)benchedSize, (int)cSize, ratio, (double)benchedSize / bestTime / 1000000); - } - - if (ratio<100.) - DISPLAY("%2i-%-28.28s :%9i ->%9i (%5.2f%%),%7.1f MB/s\n", cAlgNb, compressorName, (int)benchedSize, (int)cSize, ratio, (double)benchedSize / bestTime / 1000000); - else - DISPLAY("%2i-%-28.28s :%9i ->%9i (%5.1f%%),%7.1f MB/s\n", cAlgNb, compressorName, (int)benchedSize, (int)cSize, ratio, (double)benchedSize / bestTime / 100000); - } - - /* Prepare layout for decompression */ - /* Init data chunks */ - { int i; - size_t remaining = benchedSize; - char* in = orig_buff; - char* out = compressed_buff; - - nbChunks = (int) (((int)benchedSize + (g_chunkSize-1))/ g_chunkSize); - for (i=0; i g_chunkSize) { chunkP[i].origSize = g_chunkSize; remaining -= g_chunkSize; } else { chunkP[i].origSize = (int)remaining; remaining = 0; } - chunkP[i].compressedBuffer = out; out += maxCompressedChunkSize; - chunkP[i].compressedSize = 0; - } - } - for (chunkNb=0; chunkNb\r", loopNb, dName, (int)benchedSize); - - nb_loops = 0; - clockTime = clock(); - while(clock() == clockTime); - clockTime = clock(); - while(BMK_GetClockSpan(clockTime) < TIMELOOP) { - for (chunkNb=0; chunkNb %7.1f MB/s\r", loopNb, dName, (int)benchedSize, (double)benchedSize / bestTime / 1000000); - - /* CRC Checking */ - crcDecoded = XXH32(orig_buff, (int)benchedSize, 0); - if (crcOriginal!=crcDecoded) { DISPLAY("\n!!! WARNING !!! %14s : Invalid Checksum : %x != %x\n", inFileName, (unsigned)crcOriginal, (unsigned)crcDecoded); exit(1); } - } - - DISPLAY("%2i-%-29.29s :%10i -> %7.1f MB/s\n", dAlgNb, dName, (int)benchedSize, (double)benchedSize / bestTime / 1000000); - } - } - free(orig_buff); - free(compressed_buff); - free(chunkP); - } - - Lizard_freeStream(Lizard_stream); - Lizard_freeStream(Lizard_streamPtr); - LizardF_freeDecompressionContext(g_dCtx); - if (g_pause) { printf("press enter...\n"); (void)getchar(); } - - return 0; -} - - -static int usage(const char* exename) -{ - DISPLAY( "Usage :\n"); - DISPLAY( " %s [arg] file1 file2 ... fileX\n", exename); - DISPLAY( "Arguments :\n"); - DISPLAY( " -c : compression tests only\n"); - DISPLAY( " -d : decompression tests only\n"); - DISPLAY( " -H/-h : Help (this text + advanced options)\n"); - return 0; -} - -static int usage_advanced(void) -{ - DISPLAY( "\nAdvanced options :\n"); - DISPLAY( " -c# : test only compression function # [1-%i]\n", NB_COMPRESSION_ALGORITHMS); - DISPLAY( " -d# : test only decompression function # [1-%i]\n", NB_DECOMPRESSION_ALGORITHMS); - DISPLAY( " -i# : iteration loops [1-9](default : %i)\n", NBLOOPS); - DISPLAY( " -B# : Block size [4-7](default : 7)\n"); - return 0; -} - -static int badusage(const char* exename) -{ - DISPLAY("Wrong parameters\n"); - usage(exename); - return 0; -} - -int main(int argc, const char** argv) -{ - int i, - filenamesStart=2; - const char* exename = argv[0]; - const char* input_filename=0; - - // Welcome message - DISPLAY(WELCOME_MESSAGE); - - if (argc<2) { badusage(exename); return 1; } - - for(i=1; i= '0') && (argument[1]<= '9')) { - g_compressionAlgo *= 10; - g_compressionAlgo += argument[1] - '0'; - argument++; - } - break; - - // Select decompression algorithm only - case 'd': - g_compressionTest = 0; - while ((argument[1]>= '0') && (argument[1]<= '9')) { - g_decompressionAlgo *= 10; - g_decompressionAlgo += argument[1] - '0'; - argument++; - } - break; - - // Display help on usage - case 'h' : - case 'H': usage(exename); usage_advanced(); return 0; - - // Modify Block Properties - case 'B': - while (argument[1]!=0) - switch(argument[1]) - { - case '4': - case '5': - case '6': - case '7': - { int B = argument[1] - '0'; - int S = 1 << (8 + 2*B); - BMK_setBlocksize(S); - argument++; - break; - } - case 'D': argument++; break; - default : goto _exit_blockProperties; - } -_exit_blockProperties: - break; - - // Modify Nb Iterations - case 'i': - if ((argument[1] >='0') && (argument[1] <='9')) { - int iters = argument[1] - '0'; - BMK_setNbIterations(iters); - argument++; - } - break; - - // Pause at the end (hidden option) - case 'p': BMK_setPause(); break; - - // Unknown command - default : badusage(exename); return 1; - } - } - continue; - } - - // first provided filename is input - if (!input_filename) { input_filename=argument; filenamesStart=i; continue; } - - } - - // No input filename ==> Error - if(!input_filename) { badusage(exename); return 1; } - - return fullSpeedBench(argv+filenamesStart, argc-filenamesStart); - -} diff --git a/contrib/lizard/tests/fuzzer.c b/contrib/lizard/tests/fuzzer.c deleted file mode 100644 index 90864dc6d02..00000000000 --- a/contrib/lizard/tests/fuzzer.c +++ /dev/null @@ -1,1086 +0,0 @@ -/* - fuzzer.c - Fuzzer test tool for Lizard - Copyright (C) Yann Collet 2012-2016 - Copyright (C) Przemyslaw Skibinski 2016-2017 - - GPL v2 License - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License along - with this program; if not, write to the Free Software Foundation, Inc., - 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. - - You can contact the author at : - - Lizard source repo : https://github.com/inikep/lizard -*/ - -/*-************************************ -* Compiler options -**************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define _CRT_SECURE_NO_WARNINGS /* fgets */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4146) /* disable: C4146: minus unsigned expression */ -# pragma warning(disable : 4310) /* disable: C4310: constant char value > 127 */ -#endif - -/* S_ISREG & gettimeofday() are not supported by MSVC */ -#if defined(_MSC_VER) || defined(_WIN32) -# define FUZ_LEGACY_TIMER 1 -#endif - - -/*-************************************ -* Includes -**************************************/ -#include -#include /* fgets, sscanf */ -#include /* strcmp */ -#include /* clock_t, clock, CLOCKS_PER_SEC */ -#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ -#include "lizard_decompress.h" -#include "lizard_common.h" -#define XXH_STATIC_LINKING_ONLY -#include "xxhash/xxhash.h" - - - -/*-************************************ -* Constants -**************************************/ -#define NB_ATTEMPTS (1<<16) -#define COMPRESSIBLE_NOISE_LENGTH (1 << 21) -#define FUZ_MAX_BLOCK_SIZE (1 << 17) -#define FUZ_MAX_DICT_SIZE (1 << 15) -#define FUZ_COMPRESSIBILITY_DEFAULT 60 -#define PRIME1 2654435761U -#define PRIME2 2246822519U -#define PRIME3 3266489917U - - -/*-*************************************** -* Macros -*****************************************/ -#define DISPLAY(...) fprintf(stderr, __VA_ARGS__) -#define DISPLAYLEVEL(l, ...) if (g_displayLevel>=l) { DISPLAY(__VA_ARGS__); } -static int g_displayLevel = 2; -static const clock_t g_refreshRate = CLOCKS_PER_SEC * 25 / 100; -static clock_t g_time = 0; - - -/*-******************************************************* -* Fuzzer functions -*********************************************************/ -static clock_t FUZ_GetClockSpan(clock_t clockStart) -{ - return clock() - clockStart; /* works even if overflow; max span ~ 30mn */ -} - -static U32 FUZ_rotl32(U32 u32, U32 nbBits) -{ - return ((u32 << nbBits) | (u32 >> (32 - nbBits))); -} - -static U32 FUZ_rand(U32* src) -{ - U32 rand32 = *src; - rand32 *= PRIME1; - rand32 ^= PRIME2; - rand32 = FUZ_rotl32(rand32, 13); - *src = rand32; - return rand32; -} - - -#define FUZ_RAND15BITS ((FUZ_rand(seed) >> 3) & 32767) -#define FUZ_RANDLENGTH ( ((FUZ_rand(seed) >> 7) & 3) ? (FUZ_rand(seed) % 15) : (FUZ_rand(seed) % 510) + 15) -static void FUZ_fillCompressibleNoiseBuffer(void* buffer, size_t bufferSize, double proba, U32* seed) -{ - BYTE* BBuffer = (BYTE*)buffer; - size_t pos = 0; - U32 P32 = (U32)(32768 * proba); - - /* First Bytes */ - while (pos < 20) - BBuffer[pos++] = (BYTE)(FUZ_rand(seed)); - - while (pos < bufferSize) - { - /* Select : Literal (noise) or copy (within 64K) */ - if (FUZ_RAND15BITS < P32) - { - /* Copy (within 64K) */ - size_t match, d; - size_t length = FUZ_RANDLENGTH + 4; - size_t offset = FUZ_RAND15BITS + 1; - while (offset > pos) offset >>= 1; - d = pos + length; - while (d > bufferSize) d = bufferSize; - match = pos - offset; - while (pos < d) BBuffer[pos++] = BBuffer[match++]; - } - else - { - /* Literal (noise) */ - size_t d; - size_t length = FUZ_RANDLENGTH; - d = pos + length; - if (d > bufferSize) d = bufferSize; - while (pos < d) BBuffer[pos++] = (BYTE)(FUZ_rand(seed) >> 5); - } - } -} - - -#define MAX_NB_BUFF_I134 150 -#define BLOCKSIZE_I134 (32 MB) -/*! FUZ_AddressOverflow() : -* Aggressively pushes memory allocation limits, -* and generates patterns which create address space overflow. -* only possible in 32-bits mode */ -static int FUZ_AddressOverflow(U32* seed) -{ - char* buffers[MAX_NB_BUFF_I134]; - int i, nbBuff=0; - int highAddress = 0; - - DISPLAY("Overflow tests : "); - - /* Only possible in 32-bits */ - /* if (sizeof(void*)==8) - { - DISPLAY("64 bits mode : no overflow \n"); - fflush(stdout); - return 0; - }*/ - - buffers[0] = (char*)malloc(BLOCKSIZE_I134); - buffers[1] = (char*)malloc(BLOCKSIZE_I134); - if ((!buffers[0]) || (!buffers[1])) { - DISPLAY("not enough memory for tests \n"); - return 0; - } - - for (nbBuff=2; nbBuff < MAX_NB_BUFF_I134; nbBuff++) { - DISPLAY("%3i \b\b\b\b", nbBuff); - buffers[nbBuff] = (char*)malloc(BLOCKSIZE_I134); - if (buffers[nbBuff]==NULL) goto _endOfTests; - - if (((size_t)buffers[nbBuff] > (size_t)0x80000000) && (!highAddress)) { - DISPLAY("high address detected : "); - fflush(stdout); - highAddress=1; - } - - { int const nbOf255 = 1 + (FUZ_rand(seed) % (BLOCKSIZE_I134-1)); - char* const input = buffers[nbBuff-1]; - char* output = buffers[nbBuff]; - int r; - BYTE cLevel = LIZARD_MIN_CLEVEL + (FUZ_rand(seed) % (1+LIZARD_MAX_CLEVEL-LIZARD_MIN_CLEVEL)); - for(i = 5; i < nbOf255; i++) input[i] = (char)0xff; - for(i = 5; i < nbOf255; i+=(FUZ_rand(seed) % 128)) input[i] = (BYTE)(FUZ_rand(seed)%256); - - input[0] = (char)cLevel; /* Compression Level */ - input[1] = (char)0xF0; /* Literal length overflow */ - input[2] = (char)0xFF; - input[3] = (char)0xFF; - input[4] = (char)0xFF; - r = Lizard_decompress_safe(input, output, nbOf255, BLOCKSIZE_I134); - if (r>0 && r0 && r0 && r0 && r g_refreshRate) | (g_displayLevel>=3)) { - g_time = clock(); - DISPLAY("\r%5u ", testNb); - if (g_displayLevel>=3) fflush(stdout); - } -} - - -/*! FUZ_findDiff() : -* find the first different byte between buff1 and buff2. -* presumes buff1 != buff2. -* presumes a difference exists before end of either buffer. -* Typically invoked after a checksum mismatch. -*/ -static void FUZ_findDiff(const void* buff1, const void* buff2) -{ - const BYTE* const b1 = (const BYTE*)buff1; - const BYTE* const b2 = (const BYTE*)buff2; - size_t i=0; - while (b1[i]==b2[i]) i++; - DISPLAY("Wrong Byte at position %u\n", (unsigned)i); -} - - -static int FUZ_test(U32 seed, U32 nbCycles, const U32 startCycle, const double compressibility, U32 duration_s) -{ - unsigned long long bytes = 0; - unsigned long long cbytes = 0; - unsigned long long hcbytes = 0; - unsigned long long ccbytes = 0; - void* CNBuffer; - char* compressedBuffer; - char* decodedBuffer; -# define FUZ_max LIZARD_COMPRESSBOUND(LEN) - int ret; - unsigned cycleNb; -# define FUZ_CHECKTEST(cond, ...) if (cond) { printf("Test %u : ", testNb); printf(__VA_ARGS__); \ - printf(" (seed %u, cycle %u) \n", seed, cycleNb); goto _output_error; } -# define FUZ_DISPLAYTEST { testNb++; g_displayLevel<3 ? 0 : printf("%2u\b\b", testNb); if (g_displayLevel==4) fflush(stdout); } - void* stateLizard = malloc(Lizard_sizeofState_MinLevel()); - void* stateLizardHC = malloc(Lizard_sizeofState(0)); - Lizard_stream_t* Lizarddict; - Lizard_stream_t* Lizard_streamHCPtr; - U32 crcOrig, crcCheck; - U32 coreRandState = seed; - U32 randState = coreRandState ^ PRIME3; - int result = 0; - clock_t const clockStart = clock(); - clock_t const clockDuration = (clock_t)duration_s * CLOCKS_PER_SEC; - - - /* init */ - Lizard_streamHCPtr = Lizard_createStream(0); - Lizarddict = Lizard_createStream_MinLevel(); - - /* Create compressible test buffer */ - CNBuffer = malloc(COMPRESSIBLE_NOISE_LENGTH); - compressedBuffer = (char*)malloc(Lizard_compressBound(FUZ_MAX_BLOCK_SIZE)); - decodedBuffer = (char*)malloc(FUZ_MAX_DICT_SIZE + FUZ_MAX_BLOCK_SIZE); - - if (!stateLizard || !stateLizardHC || !Lizarddict || !Lizard_streamHCPtr || !CNBuffer || !compressedBuffer || !decodedBuffer) goto _output_error; - - FUZ_fillCompressibleNoiseBuffer(CNBuffer, COMPRESSIBLE_NOISE_LENGTH, compressibility, &randState); - - /* move to startCycle */ - for (cycleNb = 0; cycleNb < startCycle; cycleNb++) { - (void)FUZ_rand(&coreRandState); - - if (0) { /* some problems can be related to dictionary re-use; in this case, enable this loop */ - int dictSize, blockSize, blockStart; - char* dict; - char* block; - FUZ_displayUpdate(cycleNb); - randState = coreRandState ^ PRIME3; - blockSize = FUZ_rand(&randState) % FUZ_MAX_BLOCK_SIZE; - blockStart = FUZ_rand(&randState) % (COMPRESSIBLE_NOISE_LENGTH - blockSize); - dictSize = FUZ_rand(&randState) % FUZ_MAX_DICT_SIZE; - if (dictSize > blockStart) dictSize = blockStart; - block = ((char*)CNBuffer) + blockStart; - dict = block - dictSize; - Lizard_loadDict(Lizarddict, dict, dictSize); - Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - Lizard_loadDict(Lizarddict, dict, dictSize); - Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - Lizard_loadDict(Lizarddict, dict, dictSize); - Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - } } - - /* Main test loop */ - for (cycleNb = startCycle; (cycleNb < nbCycles) || (FUZ_GetClockSpan(clockStart) < clockDuration) ; cycleNb++) { - U32 testNb = 0; - char* dict; - char* block; - int dictSize, blockSize, blockStart, compressedSize, HCcompressedSize; - int blockContinueCompressedSize; - - FUZ_displayUpdate(cycleNb); - (void)FUZ_rand(&coreRandState); - randState = coreRandState ^ PRIME3; - - /* Select block to test */ - blockSize = (FUZ_rand(&randState) % (FUZ_MAX_BLOCK_SIZE-1)) + 1; - blockStart = FUZ_rand(&randState) % (COMPRESSIBLE_NOISE_LENGTH - blockSize); - dictSize = FUZ_rand(&randState) % FUZ_MAX_DICT_SIZE; - if (dictSize > blockStart) dictSize = blockStart; - block = ((char*)CNBuffer) + blockStart; - dict = block - dictSize; - - /* Compression tests */ - - /* Test compression HC */ - FUZ_DISPLAYTEST; - ret = Lizard_compress(block, compressedBuffer, blockSize, Lizard_compressBound(blockSize), 0); - FUZ_CHECKTEST(ret==0, "Lizard_compress() failed"); - HCcompressedSize = ret; - - /* Test compression HC using external state */ - FUZ_DISPLAYTEST; - ret = Lizard_compress_extState(stateLizardHC, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize), 0); - FUZ_CHECKTEST(ret==0, "Lizard_compress_extState() failed"); - - /* Test compression using external state */ - FUZ_DISPLAYTEST; - ret = Lizard_compress_extState_MinLevel(stateLizard, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - FUZ_CHECKTEST(ret==0, "Lizard_compress_extState_MinLevel(1) failed"); - - /* Test compression */ - FUZ_DISPLAYTEST; - ret = Lizard_compress_MinLevel(block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - FUZ_CHECKTEST(ret==0, "Lizard_compress_MinLevel() failed"); - compressedSize = ret; - - /* Decompression tests */ - - crcOrig = XXH32(block, blockSize, 0); - - /* Test decoding with output size exactly what's necessary => must work */ - FUZ_DISPLAYTEST; - decodedBuffer[blockSize] = 0; - ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize); - FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe failed despite sufficient space"); - FUZ_CHECKTEST(ret!=blockSize, "Lizard_decompress_safe did not regenerate original data"); - FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe overrun specified output buffer size"); - crcCheck = XXH32(decodedBuffer, blockSize, 0); - FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe corrupted decoded data"); - - // Test decoding with more than enough output size => must work - FUZ_DISPLAYTEST; - decodedBuffer[blockSize] = 0; - decodedBuffer[blockSize+1] = 0; - ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize+1); - FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe failed despite amply sufficient space"); - FUZ_CHECKTEST(ret!=blockSize, "Lizard_decompress_safe did not regenerate original data"); - //FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe wrote more than (unknown) target size"); // well, is that an issue ? - FUZ_CHECKTEST(decodedBuffer[blockSize+1], "Lizard_decompress_safe overrun specified output buffer size"); - crcCheck = XXH32(decodedBuffer, blockSize, 0); - FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe corrupted decoded data"); - - // Test decoding with output size being one byte too short => must fail - FUZ_DISPLAYTEST; - decodedBuffer[blockSize-1] = 0; - ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize-1); - FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to Output Size being one byte too short"); - FUZ_CHECKTEST(decodedBuffer[blockSize-1], "Lizard_decompress_safe overrun specified output buffer size"); - - // Test decoding with output size being 10 bytes too short => must fail - FUZ_DISPLAYTEST; - if (blockSize>10) - { - decodedBuffer[blockSize-10] = 0; - ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize, blockSize-10); - FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to Output Size being 10 bytes too short"); - FUZ_CHECKTEST(decodedBuffer[blockSize-10], "Lizard_decompress_safe overrun specified output buffer size"); - } - - // Test decoding with input size being one byte too short => must fail - FUZ_DISPLAYTEST; - ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize-1, blockSize); - FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to input size being one byte too short (blockSize=%i, ret=%i, compressedSize=%i)", blockSize, ret, compressedSize); - - // Test decoding with input size being one byte too large => must fail - FUZ_DISPLAYTEST; - decodedBuffer[blockSize] = 0; - compressedBuffer[compressedSize] = 0; /* valgrind */ - ret = Lizard_decompress_safe(compressedBuffer, decodedBuffer, compressedSize+1, blockSize); - FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe should have failed, due to input size being too large"); - FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe overrun specified output buffer size"); - - // Test partial decoding with target output size being max/2 => must work - FUZ_DISPLAYTEST; - ret = Lizard_decompress_safe_partial(compressedBuffer, decodedBuffer, compressedSize, blockSize/2, blockSize); - FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe_partial failed despite sufficient space"); - - // Test partial decoding with target output size being just below max => must work - FUZ_DISPLAYTEST; - ret = Lizard_decompress_safe_partial(compressedBuffer, decodedBuffer, compressedSize, blockSize-3, blockSize); - FUZ_CHECKTEST(ret<0, "Lizard_decompress_safe_partial failed despite sufficient space"); - - /* Test Compression with limited output size */ - - /* Test compression with output size being exactly what's necessary (should work) */ - FUZ_DISPLAYTEST; - ret = Lizard_compress_MinLevel(block, compressedBuffer, blockSize, compressedSize); - FUZ_CHECKTEST(ret==0, "Lizard_compress_MinLevel() failed despite sufficient space"); - - /* Test compression with output size being exactly what's necessary and external state (should work) */ - FUZ_DISPLAYTEST; - ret = Lizard_compress_extState_MinLevel(stateLizard, block, compressedBuffer, blockSize, compressedSize); - FUZ_CHECKTEST(ret==0, "Lizard_compress_extState_MinLevel() failed despite sufficient space"); - - /* Test HC compression with output size being exactly what's necessary (should work) */ - FUZ_DISPLAYTEST; - ret = Lizard_compress(block, compressedBuffer, blockSize, HCcompressedSize, 0); - FUZ_CHECKTEST(ret==0, "Lizard_compress() (limitedOutput) failed despite sufficient space"); - - /* Test HC compression with output size being exactly what's necessary (should work) */ - FUZ_DISPLAYTEST; - ret = Lizard_compress_extState(stateLizardHC, block, compressedBuffer, blockSize, HCcompressedSize, 0); - FUZ_CHECKTEST(ret==0, "Lizard_compress_extState() failed despite sufficient space"); - - /* Test compression with missing bytes into output buffer => must fail */ - FUZ_DISPLAYTEST; - { int missingBytes = (FUZ_rand(&randState) % 0x3F) + 1; - if (missingBytes >= compressedSize) missingBytes = compressedSize-1; - missingBytes += !missingBytes; /* avoid special case missingBytes==0 */ - compressedBuffer[compressedSize-missingBytes] = 0; - ret = Lizard_compress_MinLevel(block, compressedBuffer, blockSize, compressedSize-missingBytes); - FUZ_CHECKTEST(ret, "Lizard_compress_MinLevel should have failed (output buffer too small by %i byte)", missingBytes); - FUZ_CHECKTEST(compressedBuffer[compressedSize-missingBytes], "Lizard_compress_MinLevel overran output buffer ! (%i missingBytes)", missingBytes) - } - - /* Test HC compression with missing bytes into output buffer => must fail */ - FUZ_DISPLAYTEST; - { int missingBytes = (FUZ_rand(&randState) % 0x3F) + 1; - if (missingBytes >= HCcompressedSize) missingBytes = HCcompressedSize-1; - missingBytes += !missingBytes; /* avoid special case missingBytes==0 */ - compressedBuffer[HCcompressedSize-missingBytes] = 0; - ret = Lizard_compress(block, compressedBuffer, blockSize, HCcompressedSize-missingBytes, 0); - FUZ_CHECKTEST(ret, "Lizard_compress(limitedOutput) should have failed (output buffer too small by %i byte)", missingBytes); - FUZ_CHECKTEST(compressedBuffer[HCcompressedSize-missingBytes], "Lizard_compress overran output buffer ! (%i missingBytes)", missingBytes) - } - - - /*-******************/ - /* Dictionary tests */ - /*-******************/ - - /* Compress using dictionary */ - FUZ_DISPLAYTEST; - { Lizard_stream_t* Lizard_stream = Lizard_createStream_MinLevel(); - FUZ_CHECKTEST(Lizard_stream==NULL, "Lizard_createStream_MinLevel() allocation failed"); - Lizard_stream = Lizard_resetStream_MinLevel(Lizard_stream); - FUZ_CHECKTEST(Lizard_stream==NULL, "Lizard_resetStream_MinLevel() failed"); - Lizard_compress_continue (Lizard_stream, dict, compressedBuffer, dictSize, Lizard_compressBound(dictSize)); /* Just to fill hash tables */ - blockContinueCompressedSize = Lizard_compress_continue (Lizard_stream, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - FUZ_CHECKTEST(blockContinueCompressedSize==0, "Lizard_compress_continue failed"); - Lizard_freeStream(Lizard_stream); - } - - /* Compress using External dictionary */ - FUZ_DISPLAYTEST; - dict -= (FUZ_rand(&randState) & 0xF) + 1; /* Separation, so it is an ExtDict */ - if (dict < (char*)CNBuffer) dict = (char*)CNBuffer; - Lizard_loadDict(Lizarddict, dict, dictSize); - blockContinueCompressedSize = Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - FUZ_CHECKTEST(blockContinueCompressedSize==0, "Lizard_compress_continue failed"); - - FUZ_DISPLAYTEST; - Lizard_loadDict(Lizarddict, dict, dictSize); - ret = Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, blockContinueCompressedSize-1); - FUZ_CHECKTEST(ret>0, "Lizard_compress_continue using ExtDict should fail : one missing byte for output buffer : %i written, %i buffer", ret, blockContinueCompressedSize); - - FUZ_DISPLAYTEST; - Lizard_loadDict(Lizarddict, dict, dictSize); - ret = Lizard_compress_continue(Lizarddict, block, compressedBuffer, blockSize, blockContinueCompressedSize); - FUZ_CHECKTEST(ret!=blockContinueCompressedSize, "Lizard_compress_limitedOutput_compressed size is different (%i != %i)", ret, blockContinueCompressedSize); - FUZ_CHECKTEST(ret<=0, "Lizard_compress_continue should work : enough size available within output buffer"); - - FUZ_DISPLAYTEST; - decodedBuffer[blockSize] = 0; - ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize, dict, dictSize); - FUZ_CHECKTEST(ret!=blockSize, "2Lizard_decompress_safe_usingDict did not regenerate original data ret[%d]!=blockSize[%d]", (int)ret, (int)blockSize); - FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe_usingDict overrun specified output buffer size") - crcCheck = XXH32(decodedBuffer, blockSize, 0); - FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe_usingDict corrupted decoded data"); - - FUZ_DISPLAYTEST; - decodedBuffer[blockSize-1] = 0; - ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize-1, dict, dictSize); - FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe_usingDict should have failed : not enough output size (-1 byte)"); - FUZ_CHECKTEST(decodedBuffer[blockSize-1], "Lizard_decompress_safe_usingDict overrun specified output buffer size"); - - FUZ_DISPLAYTEST; - { U32 const missingBytes = (FUZ_rand(&randState) & 0xF) + 2; - if ((U32)blockSize > missingBytes) { - decodedBuffer[blockSize-missingBytes] = 0; - ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize-missingBytes, dict, dictSize); - FUZ_CHECKTEST(ret>=0, "Lizard_decompress_safe_usingDict should have failed : output buffer too small (-%u byte)", missingBytes); - FUZ_CHECKTEST(decodedBuffer[blockSize-missingBytes], "Lizard_decompress_safe_usingDict overrun specified output buffer size (-%u byte) (blockSize=%i)", missingBytes, blockSize); - } } - - /* Compress HC using External dictionary */ - FUZ_DISPLAYTEST; - dict -= (FUZ_rand(&randState) & 7); /* even bigger separation */ - if (dict < (char*)CNBuffer) dict = (char*)CNBuffer; - Lizard_streamHCPtr = Lizard_resetStream (Lizard_streamHCPtr, FUZ_rand(&randState) & 0x7); - FUZ_CHECKTEST(Lizard_streamHCPtr==NULL, "Lizard_resetStream failed"); - Lizard_loadDict(Lizard_streamHCPtr, dict, dictSize); - blockContinueCompressedSize = Lizard_compress_continue(Lizard_streamHCPtr, block, compressedBuffer, blockSize, Lizard_compressBound(blockSize)); - FUZ_CHECKTEST(blockContinueCompressedSize==0, "Lizard_compress_continue failed"); - - FUZ_DISPLAYTEST; - Lizard_loadDict(Lizard_streamHCPtr, dict, dictSize); - ret = Lizard_compress_continue(Lizard_streamHCPtr, block, compressedBuffer, blockSize, blockContinueCompressedSize-1); - FUZ_CHECKTEST(ret>0, "Lizard_compress_continue using ExtDict should fail : one missing byte for output buffer"); - - FUZ_DISPLAYTEST; - Lizard_loadDict(Lizard_streamHCPtr, dict, dictSize); - ret = Lizard_compress_continue(Lizard_streamHCPtr, block, compressedBuffer, blockSize, blockContinueCompressedSize); - FUZ_CHECKTEST(ret!=blockContinueCompressedSize, "Lizard_compress_limitedOutput_compressed size is different (%i != %i)", ret, blockContinueCompressedSize); - FUZ_CHECKTEST(ret<=0, "Lizard_compress_continue should work : enough size available within output buffer"); - - FUZ_DISPLAYTEST; - decodedBuffer[blockSize] = 0; - ret = Lizard_decompress_safe_usingDict(compressedBuffer, decodedBuffer, blockContinueCompressedSize, blockSize, dict, dictSize); - FUZ_CHECKTEST(ret!=blockSize, "3Lizard_decompress_safe_usingDict did not regenerate original data ret[%d]!=blockSize[%d]", (int)ret, (int)blockSize); - FUZ_CHECKTEST(decodedBuffer[blockSize], "Lizard_decompress_safe_usingDict overrun specified output buffer size") - crcCheck = XXH32(decodedBuffer, blockSize, 0); - if (crcCheck!=crcOrig) - FUZ_findDiff(block, decodedBuffer); - FUZ_CHECKTEST(crcCheck!=crcOrig, "Lizard_decompress_safe_usingDict corrupted decoded data"); - - /* ***** End of tests *** */ - /* Fill stats */ - bytes += blockSize; - cbytes += compressedSize; - hcbytes += HCcompressedSize; - ccbytes += blockContinueCompressedSize; - } - - if (nbCycles<=1) nbCycles = cycleNb; /* end by time */ - bytes += !bytes; /* avoid division by 0 */ - printf("\r%7u /%7u - ", cycleNb, nbCycles); - printf("all tests completed successfully \n"); - printf("compression ratio: %0.3f%%\n", (double)cbytes/bytes*100); - printf("HC compression ratio: %0.3f%%\n", (double)hcbytes/bytes*100); - printf("ratio with dict: %0.3f%%\n", (double)ccbytes/bytes*100); - - /* release memory */ - { -_exit: - free(CNBuffer); - free(compressedBuffer); - free(decodedBuffer); - free(stateLizard); - free(stateLizardHC); - Lizard_freeStream(Lizard_streamHCPtr); - Lizard_freeStream(Lizarddict); - return result; - -_output_error: - result = 1; - goto _exit; - } -} - - -#define testInputSize (192 KB) -#define testCompressedSize (128 KB) -#define ringBufferSize (8 KB) - -static void FUZ_unitTests(U32 seed) -{ - const unsigned testNb = 0; - const unsigned cycleNb= 0; - char testInput[testInputSize]; - char testCompressed[testCompressedSize]; - char testVerify[testInputSize]; - char ringBuffer[ringBufferSize]; - U32 randState = seed ^ PRIME3; - - /* Init */ - FUZ_fillCompressibleNoiseBuffer(testInput, testInputSize, 0.50, &randState); - - /* 32-bits address space overflow test */ - FUZ_AddressOverflow(&randState); - - /* Lizard streaming tests */ - { Lizard_stream_t* statePtr; - Lizard_stream_t* streamingState; - U64 crcOrig; - U64 crcNew; - int result; - - /* Allocation test */ - statePtr = Lizard_createStream_MinLevel(); - FUZ_CHECKTEST(statePtr==NULL, "Lizard_createStream_MinLevel() allocation failed"); - Lizard_freeStream(statePtr); - - streamingState = Lizard_createStream_MinLevel(); - FUZ_CHECKTEST(streamingState==NULL, "Lizard_createStream_MinLevel() allocation failed"); - - /* simple compression test */ - crcOrig = XXH64(testInput, testCompressedSize, 0); - streamingState = Lizard_resetStream_MinLevel(streamingState); - FUZ_CHECKTEST(streamingState==NULL, "Lizard_resetStream_MinLevel() failed"); - result = Lizard_compress_continue(streamingState, testInput, testCompressed, testCompressedSize, testCompressedSize-1); - FUZ_CHECKTEST(result==0, "Lizard_compress_continue() compression failed"); - - result = Lizard_decompress_safe(testCompressed, testVerify, result, testCompressedSize); - FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe() decompression failed Level 1 (result=%d testCompressedSize=%d)", (int)result, (int)testCompressedSize); - crcNew = XXH64(testVerify, testCompressedSize, 0); - FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() decompression corruption"); - - /* ring buffer test */ - { XXH64_state_t xxhOrig; - XXH64_state_t xxhNew; - Lizard_streamDecode_t decodeState; - const U32 maxMessageSizeLog = 10; - const U32 maxMessageSizeMask = (1< ringBufferSize) rNext = 0; - if (dNext + messageSize > dBufferSize) dNext = 0; - } - } - Lizard_freeStream(streamingState); - } - - /* Lizard streaming tests */ - { Lizard_stream_t* streamPtr; - U64 crcOrig; - U64 crcNew; - int result; - - /* Allocation test */ - streamPtr = Lizard_createStream(0); - FUZ_CHECKTEST(streamPtr==NULL, "Lizard_createStream() allocation failed"); - - /* simple HC compression test */ - crcOrig = XXH64(testInput, testCompressedSize, 0); - streamPtr = Lizard_resetStream(streamPtr, 0); - FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); - result = Lizard_compress_continue(streamPtr, testInput, testCompressed, testCompressedSize, testCompressedSize-1); - FUZ_CHECKTEST(result==0, "Lizard_compress_continue() compression failed"); - - result = Lizard_decompress_safe(testCompressed, testVerify, result, testCompressedSize); - FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe() decompression failed Level 0 (result=%d testCompressedSize=%d)", (int)result, (int)testCompressedSize); - crcNew = XXH64(testVerify, testCompressedSize, 0); - FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() decompression corruption"); - - /* simple dictionary HC compression test */ - crcOrig = XXH64(testInput + testInputSize - testCompressedSize, testCompressedSize, 0); - streamPtr = Lizard_resetStream(streamPtr, 0); - FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); - Lizard_loadDict(streamPtr, testInput, testInputSize - testCompressedSize); - result = Lizard_compress_continue(streamPtr, testInput + testInputSize - testCompressedSize, testCompressed, testCompressedSize, testCompressedSize-1); - FUZ_CHECKTEST(result==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result); - - result = Lizard_decompress_safe_usingDict(testCompressed, testVerify, result, testCompressedSize, testInput, testInputSize - testCompressedSize); - FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe() simple dictionary decompression test failed"); - crcNew = XXH64(testVerify, testCompressedSize, 0); - FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() simple dictionary decompression test : corruption"); - - /* multiple HC compression test with dictionary */ - { int result1, result2; - int segSize = testCompressedSize / 2; - crcOrig = XXH64(testInput + segSize, testCompressedSize, 0); - streamPtr = Lizard_resetStream(streamPtr, 0); - FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); - Lizard_loadDict(streamPtr, testInput, segSize); - result1 = Lizard_compress_continue(streamPtr, testInput + segSize, testCompressed, segSize, segSize -1); - FUZ_CHECKTEST(result1==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result1); - result2 = Lizard_compress_continue(streamPtr, testInput + 2*segSize, testCompressed+result1, segSize, segSize-1); - FUZ_CHECKTEST(result2==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result2); - - result = Lizard_decompress_safe_usingDict(testCompressed, testVerify, result1, segSize, testInput, segSize); - FUZ_CHECKTEST(result!=segSize, "Lizard_decompress_safe() dictionary decompression part 1 failed"); - result = Lizard_decompress_safe_usingDict(testCompressed+result1, testVerify+segSize, result2, segSize, testInput, 2*segSize); - FUZ_CHECKTEST(result!=segSize, "Lizard_decompress_safe() dictionary decompression part 2 failed"); - crcNew = XXH64(testVerify, testCompressedSize, 0); - FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe() dictionary decompression corruption"); - } - - /* remote dictionary HC compression test */ - crcOrig = XXH64(testInput + testInputSize - testCompressedSize, testCompressedSize, 0); - streamPtr = Lizard_resetStream(streamPtr, 0); - FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); - Lizard_loadDict(streamPtr, testInput, 32 KB); - result = Lizard_compress_continue(streamPtr, testInput + testInputSize - testCompressedSize, testCompressed, testCompressedSize, testCompressedSize-1); - FUZ_CHECKTEST(result==0, "Lizard_compress_continue() remote dictionary failed : result = %i", result); - - result = Lizard_decompress_safe_usingDict(testCompressed, testVerify, result, testCompressedSize, testInput, 32 KB); - FUZ_CHECKTEST(result!=(int)testCompressedSize, "Lizard_decompress_safe_usingDict() decompression failed following remote dictionary HC compression test"); - crcNew = XXH64(testVerify, testCompressedSize, 0); - FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe_usingDict() decompression corruption"); - - /* multiple HC compression with ext. dictionary */ - { XXH64_state_t crcOrigState; - XXH64_state_t crcNewState; - const char* dict = testInput + 3; - int dictSize = (FUZ_rand(&randState) & 8191); - char* dst = testVerify; - - size_t segStart = dictSize + 7; - int segSize = (FUZ_rand(&randState) & 8191); - int segNb = 1; - - streamPtr = Lizard_resetStream(streamPtr, 0); - FUZ_CHECKTEST(streamPtr==NULL, "Lizard_resetStream failed"); - Lizard_loadDict(streamPtr, dict, dictSize); - - XXH64_reset(&crcOrigState, 0); - XXH64_reset(&crcNewState, 0); - - while (segStart + segSize < testInputSize) { - XXH64_update(&crcOrigState, testInput + segStart, segSize); - crcOrig = XXH64_digest(&crcOrigState); - result = Lizard_compress_continue(streamPtr, testInput + segStart, testCompressed, segSize, Lizard_compressBound(segSize)); - FUZ_CHECKTEST(result==0, "Lizard_compress_continue() dictionary compression failed : result = %i", result); - - result = Lizard_decompress_safe_usingDict(testCompressed, dst, result, segSize, dict, dictSize); - FUZ_CHECKTEST(result!=segSize, "Lizard_decompress_safe_usingDict() dictionary decompression part %i failed", segNb); - XXH64_update(&crcNewState, dst, segSize); - crcNew = XXH64_digest(&crcNewState); - if (crcOrig!=crcNew) { - size_t c=0; - while (dst[c] == testInput[segStart+c]) c++; - DISPLAY("Bad decompression at %u / %u \n", (U32)c, (U32)segSize); - } - FUZ_CHECKTEST(crcOrig!=crcNew, "Lizard_decompress_safe_usingDict() part %i corruption", segNb); - - dict = dst; - //dict = testInput + segStart; - dictSize = segSize; - - dst += segSize + 1; - segNb ++; - - segStart += segSize + (FUZ_rand(&randState) & 0xF) + 1; - segSize = (FUZ_rand(&randState) & 8191); - } - } - - /* ring buffer test */ - { XXH64_state_t xxhOrig; - XXH64_state_t xxhNew; - Lizard_streamDecode_t decodeState; - const U32 maxMessageSizeLog = 10; - const U32 maxMessageSizeMask = (1< ringBufferSize) rNext = 0; - if (dNext + messageSize > dBufferSize) dNext = 0; - } - } - - /* small decoder-side ring buffer test */ - { XXH64_state_t xxhOrig; - XXH64_state_t xxhNew; - Lizard_streamDecode_t decodeState; - const U32 maxMessageSizeLog = 12; - const U32 maxMessageSizeMask = (1< dBufferSize) dNext = 0; - } - } - - Lizard_freeStream(streamPtr); - } - - printf("All unit tests completed successfully \n"); - return; -_output_error: - exit(1); -} - - -static int FUZ_usage(char* programName) -{ - DISPLAY( "Usage :\n"); - DISPLAY( " %s [args]\n", programName); - DISPLAY( "\n"); - DISPLAY( "Arguments :\n"); - DISPLAY( " -i# : Nb of tests (default:%i) \n", NB_ATTEMPTS); - DISPLAY( " -T# : Duration of tests, in seconds (default: use Nb of tests) \n"); - DISPLAY( " -s# : Select seed (default:prompt user)\n"); - DISPLAY( " -t# : Select starting test number (default:0)\n"); - DISPLAY( " -P# : Select compressibility in %% (default:%i%%)\n", FUZ_COMPRESSIBILITY_DEFAULT); - DISPLAY( " -v : verbose\n"); - DISPLAY( " -p : pause at the end\n"); - DISPLAY( " -h : display help and exit\n"); - return 0; -} - - -int main(int argc, char** argv) -{ - U32 seed=0; - int seedset=0; - int argNb; - int nbTests = NB_ATTEMPTS; - int testNb = 0; - int proba = FUZ_COMPRESSIBILITY_DEFAULT; - int pause = 0; - char* programName = argv[0]; - U32 duration = 0; - - /* Check command line */ - for(argNb=1; argNb='0') && (*argument<='9')) { - nbTests *= 10; - nbTests += *argument - '0'; - argument++; - } - break; - - case 'T': - argument++; - nbTests = 0; duration = 0; - for (;;) { - switch(*argument) - { - case 'm': duration *= 60; argument++; continue; - case 's': - case 'n': argument++; continue; - case '0': - case '1': - case '2': - case '3': - case '4': - case '5': - case '6': - case '7': - case '8': - case '9': duration *= 10; duration += *argument++ - '0'; continue; - } - break; - } - break; - - case 's': - argument++; - seed=0; seedset=1; - while ((*argument>='0') && (*argument<='9')) { - seed *= 10; - seed += *argument - '0'; - argument++; - } - break; - - case 't': /* select starting test nb */ - argument++; - testNb=0; - while ((*argument>='0') && (*argument<='9')) { - testNb *= 10; - testNb += *argument - '0'; - argument++; - } - break; - - case 'P': /* change probability */ - argument++; - proba=0; - while ((*argument>='0') && (*argument<='9')) { - proba *= 10; - proba += *argument - '0'; - argument++; - } - if (proba<0) proba=0; - if (proba>100) proba=100; - break; - default: ; - } - } - } - } - - printf("Starting Lizard fuzzer (%i-bits, v%s)\n", (int)(sizeof(size_t)*8), LIZARD_VERSION_STRING); - - if (!seedset) { - time_t const t = time(NULL); - U32 const h = XXH32(&t, sizeof(t), 1); - seed = h % 10000; - } - printf("Seed = %u\n", seed); - - if (proba!=FUZ_COMPRESSIBILITY_DEFAULT) printf("Compressibility : %i%%\n", proba); - - if (testNb==0) FUZ_unitTests(seed); - - if (nbTests<=0) nbTests=1; - - { int const result = FUZ_test(seed, nbTests, testNb, ((double)proba) / 100, duration); - if (pause) { - DISPLAY("press enter ... \n"); - (void)getchar(); - } - return result; - } -} diff --git a/contrib/lizard/visual/.gitignore b/contrib/lizard/visual/.gitignore deleted file mode 100644 index 538cd1a21c8..00000000000 --- a/contrib/lizard/visual/.gitignore +++ /dev/null @@ -1,11 +0,0 @@ -# Visual C++ -.vs/ -*Copy -*.db -*.opensdf -*.sdf -*.suo -*.user -*.opendb - -VS2010/bin/ diff --git a/contrib/lizard/visual/README.md b/contrib/lizard/visual/README.md deleted file mode 100644 index 527e4be1874..00000000000 --- a/contrib/lizard/visual/README.md +++ /dev/null @@ -1,53 +0,0 @@ -Projects for various integrated development environments (IDE) -============================================================== - -#### Included projects - -The following projects are included with the lizard distribution: -- `VS2010` - Visual Studio 2010 project (which also works well with Visual Studio 2012, 2013, 2015) - - -#### How to compile lizard with Visual Studio - -1. Install Visual Studio e.g. VS 2015 Community Edition (it's free). -2. Download the latest version of lizard from https://github.com/inikep/lizard/releases -3. Decompress ZIP archive. -4. Go to decompressed directory then to `visual` then `VS2010` and open `lizard.sln` -5. Visual Studio will ask about converting VS2010 project to VS2015 and you should agree. -6. Change `Debug` to `Release` and if you have 64-bit Windows change also `Win32` to `x64`. -7. Press F7 on keyboard or select `BUILD` from the menu bar and choose `Build Solution`. -8. If compilation will be fine a compiled executable will be in `visual\VS2010\bin\x64_Release\lizard.exe` - - -#### Projects available within lizard.sln - -The Visual Studio solution file `lizard.sln` contains many projects that will be compiled to the -`visual\VS2010\bin\$(Platform)_$(Configuration)` directory. For example `lizard` set to `x64` and -`Release` will be compiled to `visual\VS2010\bin\x64_Release\lizard.exe`. The solution file contains the -following projects: - -- `lizard` : Command Line Utility, supporting gzip-like arguments -- `datagen` : Synthetic and parametrable data generator, for tests -- `frametest` : Test tool that checks lizard_frame integrity on target platform -- `fullbench` : Precisely measure speed for each lizard inner functions -- `fuzzer` : Test tool, to check lizard integrity on target platform -- `liblizard` : A static Lizard library compiled to `liblizard_static.lib` -- `liblizard-dll` : A dynamic Lizard library (DLL) compiled to `liblizard.dll` with the import library `liblizard.lib` -- `fullbench-dll` : The fullbench program compiled with the import library; the executable requires Lizard DLL - - -#### Using Lizard DLL with Microsoft Visual C++ project - -The header files `lib\lizard.h`, `lib\lizardhc.h`, `lib\lizard_frame.h` and the import library -`visual\VS2010\bin\$(Platform)_$(Configuration)\liblizard.lib` are required to compile a -project using Visual C++. - -1. The path to header files should be added to `Additional Include Directories` that can - be found in Project Properties of Visual Studio IDE in the `C/C++` Property Pages on the `General` page. -2. The import library has to be added to `Additional Dependencies` that can - be found in Project Properties in the `Linker` Property Pages on the `Input` page. - If one will provide only the name `liblizard.lib` without a full path to the library - then the directory has to be added to `Linker\General\Additional Library Directories`. - -The compiled executable will require Lizard DLL which is available at -`visual\VS2010\bin\$(Platform)_$(Configuration)\liblizard.dll`. diff --git a/contrib/lizard/visual/VS2010/datagen/datagen.vcxproj b/contrib/lizard/visual/VS2010/datagen/datagen.vcxproj deleted file mode 100644 index 7f7581d01b8..00000000000 --- a/contrib/lizard/visual/VS2010/datagen/datagen.vcxproj +++ /dev/null @@ -1,165 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {D745AE2F-596A-403A-9B91-81A8C6779243} - Win32Proj - datagen - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - Application - true - Unicode - - - Application - true - Unicode - - - Application - false - true - Unicode - - - Application - false - true - Unicode - - - - - - - - - - - - - - - - - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\programs;$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - - - Console - true - - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - false - - - Console - true - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - true - true - - - - - - - - - - - - - \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/frametest/frametest.vcxproj b/contrib/lizard/visual/VS2010/frametest/frametest.vcxproj deleted file mode 100644 index b753d651be1..00000000000 --- a/contrib/lizard/visual/VS2010/frametest/frametest.vcxproj +++ /dev/null @@ -1,177 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7} - Win32Proj - frametest - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - Application - true - Unicode - - - Application - true - Unicode - - - Application - false - true - Unicode - - - Application - false - true - Unicode - - - - - - - - - - - - - - - - - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - - - Console - true - - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - false - - - Console - true - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - true - true - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj b/contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj deleted file mode 100644 index f05bb298055..00000000000 --- a/contrib/lizard/visual/VS2010/fullbench/fullbench.vcxproj +++ /dev/null @@ -1,175 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E} - Win32Proj - fullbench - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - Application - true - Unicode - - - Application - true - Unicode - - - Application - false - true - Unicode - - - Application - false - true - Unicode - - - - - - - - - - - - - - - - - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - - - Console - true - - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - - - Console - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - false - - - Console - true - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - false - - - Console - true - true - true - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj b/contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj deleted file mode 100644 index 7fa0fec022f..00000000000 --- a/contrib/lizard/visual/VS2010/fuzzer/fuzzer.vcxproj +++ /dev/null @@ -1,174 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {18B9F1A7-9C66-4352-898B-30804DADE0FD} - Win32Proj - fuzzer - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - Application - true - Unicode - - - Application - true - Unicode - - - Application - false - true - Unicode - - - Application - false - true - Unicode - - - - - - - - - - - - - - - - - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - - - Console - true - - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - false - - - Console - true - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - true - true - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc b/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc deleted file mode 100644 index 79110101f3b..00000000000 --- a/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.rc +++ /dev/null @@ -1,51 +0,0 @@ -// Microsoft Visual C++ generated resource script. -// - -#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ -#define APSTUDIO_READONLY_SYMBOLS -#include "verrsrc.h" -#undef APSTUDIO_READONLY_SYMBOLS - - -#if !defined(AFX_RESOURCE_DLL) || defined(AFX_TARG_ENU) -LANGUAGE 9, 1 - -///////////////////////////////////////////////////////////////////////////// -// -// Version -// - -VS_VERSION_INFO VERSIONINFO - FILEVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 - PRODUCTVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 - FILEFLAGSMASK VS_FFI_FILEFLAGSMASK -#ifdef _DEBUG - FILEFLAGS VS_FF_DEBUG -#else - FILEFLAGS 0x0L -#endif - FILEOS VOS_NT_WINDOWS32 - FILETYPE VFT_DLL - FILESUBTYPE VFT2_UNKNOWN -BEGIN - BLOCK "StringFileInfo" - BEGIN - BLOCK "040904B0" - BEGIN - VALUE "CompanyName", "Yann Collet, Przemyslaw Skibinski" - VALUE "FileDescription", "Extremely fast compression" - VALUE "FileVersion", LIZARD_VERSION_STRING - VALUE "InternalName", "liblizard.dll" - VALUE "LegalCopyright", "Copyright (C) 2013-2017, Yann Collet, Przemyslaw Skibinski" - VALUE "OriginalFilename", "liblizard.dll" - VALUE "ProductName", "Lizard" - VALUE "ProductVersion", LIZARD_VERSION_STRING - END - END - BLOCK "VarFileInfo" - BEGIN - VALUE "Translation", 0x0409, 1200 - END -END - -#endif diff --git a/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj b/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj deleted file mode 100644 index 09cdaa16180..00000000000 --- a/contrib/lizard/visual/VS2010/liblizard-dll/liblizard-dll.vcxproj +++ /dev/null @@ -1,179 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {00000000-3E71-41B3-BF68-4A7BDD8A5476} - Win32Proj - liblizard-dll - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - DynamicLibrary - true - Unicode - - - DynamicLibrary - true - Unicode - - - DynamicLibrary - false - true - Unicode - - - DynamicLibrary - false - true - Unicode - - - - - - - - - - - - - - - - - - - true - liblizard - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - liblizard - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - liblizard - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - liblizard - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - true - false - - - true - - - - - - - Level4 - Disabled - WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - true - true - /analyze:stacksize295252 %(AdditionalOptions) - - - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - false - false - - - true - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - false - true - /analyze:stacksize295252 %(AdditionalOptions) - - - true - true - true - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj b/contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj deleted file mode 100644 index 15b74523b96..00000000000 --- a/contrib/lizard/visual/VS2010/liblizard/liblizard.vcxproj +++ /dev/null @@ -1,176 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476} - Win32Proj - liblizard - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - StaticLibrary - true - Unicode - - - StaticLibrary - true - Unicode - - - StaticLibrary - false - true - Unicode - - - StaticLibrary - false - true - Unicode - - - - - - - - - - - - - - - - - - - true - liblizard_static - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - liblizard_static - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - liblizard_static - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - liblizard_static - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - true - false - - - true - - - - - - - Level4 - Disabled - WIN32;_DEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - true - true - /analyze:stacksize295252 %(AdditionalOptions) - - - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - false - false - - - true - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;LIZARD_DLL_EXPORT=1;%(PreprocessorDefinitions) - false - true - /analyze:stacksize295252 %(AdditionalOptions) - - - true - true - true - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/contrib/lizard/visual/VS2010/lizard.sln b/contrib/lizard/visual/VS2010/lizard.sln deleted file mode 100644 index 72d68335c1e..00000000000 --- a/contrib/lizard/visual/VS2010/lizard.sln +++ /dev/null @@ -1,86 +0,0 @@ - -Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio Express 2012 for Windows Desktop -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "lizard", "lizard\lizard.vcxproj", "{E30329AC-0057-4FE0-8FDA-7F650D398C4C}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "liblizard", "liblizard\liblizard.vcxproj", "{9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "liblizard-dll", "liblizard-dll\liblizard-dll.vcxproj", "{00000000-3E71-41B3-BF68-4A7BDD8A5476}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "fuzzer", "fuzzer\fuzzer.vcxproj", "{18B9F1A7-9C66-4352-898B-30804DADE0FD}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "fullbench", "fullbench\fullbench.vcxproj", "{6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "frametest", "frametest\frametest.vcxproj", "{39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}" -EndProject -Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "datagen", "datagen\datagen.vcxproj", "{D745AE2F-596A-403A-9B91-81A8C6779243}" -EndProject -Global - GlobalSection(SolutionConfigurationPlatforms) = preSolution - Debug|Win32 = Debug|Win32 - Debug|x64 = Debug|x64 - Release|Win32 = Release|Win32 - Release|x64 = Release|x64 - EndGlobalSection - GlobalSection(ProjectConfigurationPlatforms) = postSolution - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|Win32.ActiveCfg = Debug|Win32 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|Win32.Build.0 = Debug|Win32 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|x64.ActiveCfg = Debug|x64 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Debug|x64.Build.0 = Debug|x64 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|Win32.ActiveCfg = Release|Win32 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|Win32.Build.0 = Release|Win32 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|x64.ActiveCfg = Release|x64 - {E30329AC-0057-4FE0-8FDA-7F650D398C4C}.Release|x64.Build.0 = Release|x64 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.ActiveCfg = Debug|Win32 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.Build.0 = Debug|Win32 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.ActiveCfg = Debug|x64 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.Build.0 = Debug|x64 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.ActiveCfg = Release|Win32 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.Build.0 = Release|Win32 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.ActiveCfg = Release|x64 - {9092C5CC-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.Build.0 = Release|x64 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.ActiveCfg = Debug|Win32 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|Win32.Build.0 = Debug|Win32 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.ActiveCfg = Debug|x64 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Debug|x64.Build.0 = Debug|x64 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.ActiveCfg = Release|Win32 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|Win32.Build.0 = Release|Win32 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.ActiveCfg = Release|x64 - {00000000-3E71-41B3-BF68-4A7BDD8A5476}.Release|x64.Build.0 = Release|x64 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|Win32.ActiveCfg = Debug|Win32 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|Win32.Build.0 = Debug|Win32 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|x64.ActiveCfg = Debug|x64 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Debug|x64.Build.0 = Debug|x64 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|Win32.ActiveCfg = Release|Win32 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|Win32.Build.0 = Release|Win32 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|x64.ActiveCfg = Release|x64 - {18B9F1A7-9C66-4352-898B-30804DADE0FD}.Release|x64.Build.0 = Release|x64 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|Win32.ActiveCfg = Debug|Win32 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|Win32.Build.0 = Debug|Win32 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|x64.ActiveCfg = Debug|x64 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Debug|x64.Build.0 = Debug|x64 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|Win32.ActiveCfg = Release|Win32 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|Win32.Build.0 = Release|Win32 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|x64.ActiveCfg = Release|x64 - {6A4DF4EF-C77F-43C6-8901-DDCD20879E4E}.Release|x64.Build.0 = Release|x64 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|Win32.ActiveCfg = Debug|Win32 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|Win32.Build.0 = Debug|Win32 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|x64.ActiveCfg = Debug|x64 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Debug|x64.Build.0 = Debug|x64 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|Win32.ActiveCfg = Release|Win32 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|Win32.Build.0 = Release|Win32 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|x64.ActiveCfg = Release|x64 - {39AD6ECC-8BAD-4368-95E4-A1AA2F077BB7}.Release|x64.Build.0 = Release|x64 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|Win32.ActiveCfg = Debug|Win32 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|Win32.Build.0 = Debug|Win32 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|x64.ActiveCfg = Debug|x64 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Debug|x64.Build.0 = Debug|x64 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|Win32.ActiveCfg = Release|Win32 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|Win32.Build.0 = Release|Win32 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|x64.ActiveCfg = Release|x64 - {D745AE2F-596A-403A-9B91-81A8C6779243}.Release|x64.Build.0 = Release|x64 - EndGlobalSection - GlobalSection(SolutionProperties) = preSolution - HideSolutionNode = FALSE - EndGlobalSection -EndGlobal diff --git a/contrib/lizard/visual/VS2010/lizard/lizard.rc b/contrib/lizard/visual/VS2010/lizard/lizard.rc deleted file mode 100644 index 76c3c7f947e..00000000000 --- a/contrib/lizard/visual/VS2010/lizard/lizard.rc +++ /dev/null @@ -1,51 +0,0 @@ -// Microsoft Visual C++ generated resource script. -// - -#include "lizard_compress.h" /* LIZARD_VERSION_STRING */ -#define APSTUDIO_READONLY_SYMBOLS -#include "verrsrc.h" -#undef APSTUDIO_READONLY_SYMBOLS - - -#if !defined(AFX_RESOURCE_DLL) || defined(AFX_TARG_ENU) -LANGUAGE 9, 1 - -///////////////////////////////////////////////////////////////////////////// -// -// Version -// - -VS_VERSION_INFO VERSIONINFO - FILEVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 - PRODUCTVERSION LIZARD_VERSION_MAJOR,LIZARD_VERSION_MINOR,LIZARD_VERSION_RELEASE,0 - FILEFLAGSMASK VS_FFI_FILEFLAGSMASK -#ifdef _DEBUG - FILEFLAGS VS_FF_DEBUG -#else - FILEFLAGS 0x0L -#endif - FILEOS VOS_NT_WINDOWS32 - FILETYPE VFT_DLL - FILESUBTYPE VFT2_UNKNOWN -BEGIN - BLOCK "StringFileInfo" - BEGIN - BLOCK "040904B0" - BEGIN - VALUE "CompanyName", "Yann Collet, Przemyslaw Skibinski" - VALUE "FileDescription", "Extremely fast compression" - VALUE "FileVersion", LIZARD_VERSION_STRING - VALUE "InternalName", "lizard.exe" - VALUE "LegalCopyright", "Copyright (C) 2013-2017, Yann Collet, Przemyslaw Skibinski" - VALUE "OriginalFilename", "lizard.exe" - VALUE "ProductName", "Lizard" - VALUE "ProductVersion", LIZARD_VERSION_STRING - END - END - BLOCK "VarFileInfo" - BEGIN - VALUE "Translation", 0x0409, 1200 - END -END - -#endif diff --git a/contrib/lizard/visual/VS2010/lizard/lizard.vcxproj b/contrib/lizard/visual/VS2010/lizard/lizard.vcxproj deleted file mode 100644 index 98be4d599db..00000000000 --- a/contrib/lizard/visual/VS2010/lizard/lizard.vcxproj +++ /dev/null @@ -1,190 +0,0 @@ - - - - - Debug - Win32 - - - Debug - x64 - - - Release - Win32 - - - Release - x64 - - - - {E30329AC-0057-4FE0-8FDA-7F650D398C4C} - Win32Proj - lizard - $(SolutionDir)bin\$(Platform)_$(Configuration)\ - $(SolutionDir)bin\obj\$(RootNamespace)_$(Platform)_$(Configuration)\ - - - - Application - true - MultiByte - - - Application - true - MultiByte - - - Application - false - true - MultiByte - - - Application - false - true - MultiByte - - - - - - - - - - - - - - - - - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - true - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - - - false - $(IncludePath);$(UniversalCRT_IncludePath);$(SolutionDir)..\..\lib;$(VCInstallDir)include;$(VCInstallDir)atlmfc\include;$(WindowsSDK_IncludePath); - true - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - false - - - Console - true - setargv.obj;%(AdditionalDependencies) - - - - - - - Level4 - Disabled - WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions) - true - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - setargv.obj;%(AdditionalDependencies) - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - false - - - Console - true - setargv.obj;%(AdditionalDependencies) - true - true - - - - - Level4 - - - MaxSpeed - true - true - WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions) - false - true - /analyze:stacksize295252 %(AdditionalOptions) - - - Console - true - setargv.obj;%(AdditionalDependencies) - true - true - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/contrib/lzsse/.gitignore b/contrib/lzsse/.gitignore deleted file mode 100644 index 379a8f7f570..00000000000 --- a/contrib/lzsse/.gitignore +++ /dev/null @@ -1,4 +0,0 @@ -bin/* -gmake/* -vs2013/* -vs2015/* \ No newline at end of file diff --git a/contrib/lzsse/.travis.yml b/contrib/lzsse/.travis.yml deleted file mode 100644 index 68bb59ef0b9..00000000000 --- a/contrib/lzsse/.travis.yml +++ /dev/null @@ -1,185 +0,0 @@ -language: c -sudo: false -branches: - except: - - /^(wip\/)?(appveyor|msvc|mingw|windows)(\-.+)?$/ -matrix: - include: - ### - ## Linux builds using various versions of GCC. - ### - - env: C_COMPILER=gcc-7 - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - premake4 - packages: - - gcc-7 - - g++-7 - - premake4 - # - env: C_COMPILER=gcc-6 - # addons: - # apt: - # sources: - # - ubuntu-toolchain-r-test - # - premake4 - # packages: - # - gcc-6 - # - g++-6 - # - premake4 - - env: C_COMPILER=gcc-5 - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - premake4 - packages: - - gcc-5 - - g++-5 - - premake4 - # - env: C_COMPILER=gcc-4.9 - # addons: - # apt: - # sources: - # - ubuntu-toolchain-r-test - # - premake4 - # packages: - # - gcc-4.9 - # - g++-4.9 - # - premake4 - - env: C_COMPILER=gcc-4.8 - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - premake4 - packages: - - gcc-4.8 - - g++-4.8 - - premake4 - # - env: C_COMPILER=gcc-4.7 - # addons: - # apt: - # sources: - # - ubuntu-toolchain-r-test - # - premake4 - # packages: - # - gcc-4.7 - # - g++-4.7 - # - premake4 - - env: C_COMPILER=gcc-4.6 - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - premake4 - packages: - - gcc-4.6 - - g++-4.6 - - premake4 - # - os: linux - # env: C_COMPILER=gcc-4.5 - # addons: - # apt: - # sources: - # - ubuntu-toolchain-r-test - # - premake4 - # packages: - # - gcc-4.5 - # - g++-4.5 - # - premake4 - - env: C_COMPILER=gcc-4.4 - addons: - apt: - sources: - - ubuntu-toolchain-r-test - - premake4 - packages: - - gcc-4.4 - - g++-4.4 - - premake4 - - ### - ## clang on Linux - ### - - env: C_COMPILER=clang-3.9 - addons: - apt: - sources: - - llvm-toolchain-precise-3.9 - - ubuntu-toolchain-r-test - - premake4 - packages: - - clang-3.9 - - premake4 - # - env: C_COMPILER=clang-3.8 - # addons: - # apt: - # sources: - # - llvm-toolchain-precise-3.8 - # - ubuntu-toolchain-r-test - # - premake4 - # packages: - # - clang-3.8 - # - premake4 - - env: C_COMPILER=clang-3.7 - addons: - apt: - sources: - - llvm-toolchain-precise-3.7 - - ubuntu-toolchain-r-test - - premake4 - packages: - - clang-3.7 - - premake4 - # - env: C_COMPILER=clang-3.6 - # addons: - # apt: - # sources: - # - llvm-toolchain-precise-3.6 - # - ubuntu-toolchain-r-test - # - premake4 - # packages: - # - clang-3.6 - # - premake4 - - env: C_COMPILER=clang-3.5 - addons: - apt: - sources: - - llvm-toolchain-precise-3.5 - - ubuntu-toolchain-r-test - - premake4 - packages: - - clang-3.5 - - premake4 - - ### - ## PGI - ### - - env: C_COMPILER=pgcc ENABLE_OPENMP=y - addons: - apt: - sources: - - premake4 - packages: - - premake4 - - ### - ## OS X - ### - # - os: osx - -before_install: -### -## If we use the matrix to set CC/CXX Travis, overwrites the values, -## so instead we use C/CXX_COMPILER, then copy the values to CC/CXX -## here (after Travis has set CC/CXX). -### -- if [ "${C_COMPILER}" = "pgcc" ]; then wget -q -O /dev/stdout 'https://raw.githubusercontent.com/nemequ/pgi-travis/master/install-pgi.sh' | /bin/sh; fi -- if [ -n "${C_COMPILER}" ]; then export CC="${C_COMPILER}"; fi -- premake4 gmake - -script: -- cd gmake && make - diff --git a/contrib/lzsse/LICENSE b/contrib/lzsse/LICENSE deleted file mode 100644 index d13dc62210f..00000000000 --- a/contrib/lzsse/LICENSE +++ /dev/null @@ -1,23 +0,0 @@ -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE -FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER -CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, -OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/contrib/lzsse/README.md b/contrib/lzsse/README.md deleted file mode 100644 index 6b777eecb52..00000000000 --- a/contrib/lzsse/README.md +++ /dev/null @@ -1,15 +0,0 @@ -# LZSSE -[LZSS](https://en.wikipedia.org/wiki/Lempel%E2%80%93Ziv%E2%80%93Storer%E2%80%93Szymanski) designed for a branchless SSE decompression implementation. - -Three variants: -- LZSSE2, for high compression files with small literal runs. -- LZSSE4, for a more balanced mix of literals and matches. -- LZSSE8, for lower compression data with longer runs of matches. - -All three variants have an optimal parser implementation, which uses a quite strong match finder (very similar to LzFind) combined with a Storer-Szymanski style parse. LZSSE4 and LZSSE8 have "fast" compressor implementations, which use a simple hash table based matching and a greedy parse. - -Currently LZSSE8 is the recommended variant to use in the general case, as it generally performs well in most cases (and you have the option of both optimal parse and fast compression). LZSSE2 is recommended if you are only using text, especially heavily compressible text, but is slow/doesn't compress as well on less compressible data and binaries. - -The code is approaching production readiness and LZSSE2 and LZSSE8 have received a reasonable amount of testing. - -See these blog posts [An LZ Codec Designed for SSE Decompression](http://conorstokes.github.io/compression/2016/02/15/an-LZ-codec-designed-for-SSE-decompression) and [Compressor Improvements and LZSSE2 vs LZSSE8](http://conorstokes.github.io/compression/2016/02/24/compressor-improvements-and-lzsse2-vs-lzsse8) for a description of how the compression algorithm and implementation function. There are also benchmarks, but these may not be upto date (in particular the figures in the initial blog post no longer represent compression performance). diff --git a/contrib/lzsse/example/main.cpp b/contrib/lzsse/example/main.cpp deleted file mode 100644 index 9287f683c7d..00000000000 --- a/contrib/lzsse/example/main.cpp +++ /dev/null @@ -1,389 +0,0 @@ -// LZSSE.cpp : Defines the entry point for the console application. -// -#define _CRT_SECURE_NO_WARNINGS 1 - -#include -#include -#include -#include -#include "../lzsse2/lzsse2.h" -#include "../lzsse4/lzsse4.h" -#include "../lzsse8/lzsse8.h" - -static const uint32_t MAGIC_NUMBER = 0x28F19732; - -void DisplayUsage() -{ - printf( "Usage:\n" ); - printf( " lzsse [args] input_file output_file\n" ); - printf( "\n" ); - printf( "Arguments:\n" ); - printf( " -2 Compress in lzsse2 mode (default)\n" ); - printf( " -4 Compress in lzsse4 mode\n" ); - printf( " -8 Compress in lzsse8 mode\n" ); - printf( " -f Optimal parse (default)\n" ); - printf( " -o Fast parse (not available for lzsse2)\n" ); - printf( " -d Decompress\n" ); - printf( " -lN Compression level for optimal parse, where N is 1 to 17 (default 16)\n" ); - printf( " -bN Block size in KiB, default 131,072\n" ); - printf( "\n" ); -} - -static size_t CompressorFastLZSSE4( LZSSE4_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int ) -{ - return LZSSE4_CompressFast( state, input, inputLength, output, outputLength ); -} - -static size_t CompressorFastLZSSE8( LZSSE8_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int ) -{ - return LZSSE8_CompressFast( state, input, inputLength, output, outputLength ); -} - -template -void Compress( FILE* inputFile, FILE* outputFile, uint64_t blockSize, uint8_t mode, unsigned int level, State* state, size_t (*compressor)( State*, const void*, size_t, void*, size_t, unsigned int ) ) -{ - if ( state == nullptr ) - { - printf( "Couldn't allocate parse state\n" ); - exit( 1 ); - } - - if ( fwrite( &MAGIC_NUMBER, sizeof( uint32_t ), 1, outputFile ) == 0 ) - { - printf( "Couldn't write magic number\n" ); - exit( 1 ); - } - - if ( fwrite( &mode, sizeof( uint8_t ), 1, outputFile ) == 0 ) - { - printf( "Couldn't write stream type\n" ); - exit( 1 ); - } - - if ( fwrite( &blockSize, sizeof( uint64_t ), 1, outputFile ) == 0 ) - { - printf( "Couldn't write block size\n" ); - exit( 1 ); - } - - size_t typedBlockSize = static_cast< size_t >( blockSize ); - uint8_t* inputBuffer = reinterpret_cast< uint8_t* >( malloc( typedBlockSize ) ); - uint8_t* outputBuffer = reinterpret_cast< uint8_t* >( malloc( typedBlockSize ) ); - - if ( inputBuffer == nullptr || outputBuffer == nullptr ) - { - printf( "Couldn't allocate buffer memory\n" ); - exit( 1 ); - } - - for ( ;; ) - { - size_t readSize = fread( inputBuffer, 1, blockSize, inputFile ); - - if ( readSize == 0 ) - { - break; - } - - size_t compressedSize = compressor( state, inputBuffer, readSize, outputBuffer, typedBlockSize, level ); - - if ( compressedSize == 0 ) - { - printf( "Compression function failed\n" ); - exit( 1 ); - } - - uint32_t compressedLength = static_cast< uint32_t >( compressedSize ); - uint32_t uncompressedLength = static_cast< uint32_t >( readSize ); - - if ( fwrite( &uncompressedLength, sizeof( uint32_t ), 1, outputFile ) < 1 ) - { - break; - } - - if ( fwrite( &compressedLength, sizeof( uint32_t ), 1, outputFile ) < 1 ) - { - printf( "Error writing compressed length from block\n" ); - exit( 1 ); - } - - if ( fwrite( outputBuffer, 1, compressedLength, outputFile ) != compressedLength ) - { - printf( "Error writing block\n" ); - exit( 1 ); - } - } - - free( inputBuffer ); - free( outputBuffer ); -} - -void Decompress( FILE* inputFile, FILE* outputFile ) -{ - uint32_t magicNumber; - uint64_t blockSize = 128 * 1024 * 1024; - - if ( fread( &magicNumber, sizeof( uint32_t ), 1, inputFile ) < 1 || magicNumber != MAGIC_NUMBER ) - { - printf( "Couldn't read magic number, or magic number incorrect\n" ); - exit( 1 ); - } - - uint8_t streamType; - - if ( fread( &streamType, sizeof( uint8_t ), 1, inputFile ) < 1 ) - { - printf( "Couldn't read stream type\n" ); - exit( 1 ); - } - - if ( fread( &blockSize, sizeof( uint64_t ), 1, inputFile ) < 1 ) - { - printf( "Couldn't read block size\n" ); - exit( 1 ); - } - - uint8_t* inputBuffer = reinterpret_cast< uint8_t* >( malloc( static_cast< size_t >( blockSize ) ) ); - uint8_t* outputBuffer = reinterpret_cast< uint8_t* >( malloc( static_cast< size_t >( blockSize ) ) ); - - if ( inputBuffer == nullptr || outputBuffer == nullptr ) - { - printf( "Couldn't allocate buffer memory\n" ); - exit( 1 ); - } - - size_t( *decompressor )( const void*, size_t, void*, size_t ); - - switch ( streamType ) - { - case 2: - - decompressor = LZSSE2_Decompress; - break; - - case 4: - - decompressor = LZSSE4_Decompress; - break; - - case 8: - - decompressor = LZSSE8_Decompress; - break; - - default: - - printf( "Invalid stream type\n" ); - exit( 1 ); - - } - - memset( inputBuffer, 0, blockSize ); - memset( outputBuffer, 0, blockSize ); - - for ( ;; ) - { - uint32_t compressedLength; - uint32_t uncompressedLength; - - if ( fread( &uncompressedLength, sizeof( uint32_t ), 1, inputFile ) < 1 ) - { - break; - } - - if ( fread( &compressedLength, sizeof( uint32_t ), 1, inputFile ) < 1 ) - { - printf( "Error reading compressed length from block\n" ); - exit( 1 ); - } - - if ( fread( inputBuffer, 1, compressedLength, inputFile ) != compressedLength ) - { - printf( "Error reading block\n" ); - exit( 1 ); - } - - size_t decompressedSize = 0; - - decompressedSize = - decompressor( inputBuffer, - compressedLength, - outputBuffer, - uncompressedLength ); - - if ( decompressedSize != size_t( uncompressedLength ) ) - { - printf( "Error in decompression stream\n" ); - exit( 1 ); - } - - if ( fwrite( outputBuffer, 1, uncompressedLength, outputFile ) != uncompressedLength ) - { - printf( "Couldn't write block to output file\n" ); - exit( 1 ); - } - } - - free( inputBuffer ); - free( outputBuffer ); -} - -int main( int argc, const char** argv ) -{ - bool decompression = false; - bool optimal = true; - uint64_t blockSize = 128 * 1024 * 1024; - uint8_t mode = 2; - unsigned int level = 16; - - if ( argc < 3 ) - { - DisplayUsage(); - exit( 1 ); - } - - for ( int argIndex = 1; argIndex < argc - 2; ++argIndex ) - { - const char* arg = argv[ argIndex ]; - - if ( arg[ 0 ] == '-' ) - { - switch ( arg[ 1 ] ) - { - case 'd': - - decompression = true; - break; - - case '2': - - mode = 2; - break; - - case '4': - - mode = 4; - break; - - case '8': - - mode = 8; - break; - - case 'l': - - level = static_cast< unsigned int >( strtoul( arg + 2, nullptr, 10 ) ); - break; - - case 'b': - - blockSize = strtoull( arg + 2, nullptr, 10 ) * 1024; - break; - - case 'o': - - optimal = true; - break; - - case 'f': - - optimal = false; - break; - - } - } - } - - FILE* inputFile = fopen( argv[ argc - 2 ], "rb" ); - - if ( inputFile == nullptr ) - { - perror( argv[ argc - 2 ] ); - exit( 1 ); - } - - FILE* outputFile = fopen( argv[ argc - 1 ], "wb+" ); - - if ( outputFile == nullptr ) - { - perror( argv[ argc - 2 ] ); - exit( 1 ); - } - - if ( decompression ) - { - Decompress( inputFile, outputFile ); - } - else - { - switch ( mode ) - { - case 2: - { - LZSSE2_OptimalParseState* state = LZSSE2_MakeOptimalParseState( static_cast< size_t >( blockSize ) ); - - Compress( inputFile, outputFile, blockSize, mode, level, state, LZSSE2_CompressOptimalParse ); - - LZSSE2_FreeOptimalParseState( state ); - - break; - } - - case 4: - { - if ( optimal ) - { - LZSSE4_OptimalParseState* state = LZSSE4_MakeOptimalParseState( static_cast( blockSize ) ); - - Compress( inputFile, outputFile, blockSize, mode, level, state, LZSSE4_CompressOptimalParse ); - - LZSSE4_FreeOptimalParseState( state ); - } - else - { - LZSSE4_FastParseState* state = LZSSE4_MakeFastParseState(); - - Compress( inputFile, outputFile, blockSize, mode, level, state, CompressorFastLZSSE4 ); - - LZSSE4_FreeFastParseState( state ); - } - - break; - } - - case 8: - { - if ( optimal ) - { - LZSSE8_OptimalParseState* state = LZSSE8_MakeOptimalParseState( static_cast( blockSize ) ); - - Compress( inputFile, outputFile, blockSize, mode, level, state, LZSSE8_CompressOptimalParse ); - - LZSSE8_FreeOptimalParseState( state ); - } - else - { - LZSSE8_FastParseState* state = LZSSE8_MakeFastParseState(); - - Compress( inputFile, outputFile, blockSize, mode, level, state, CompressorFastLZSSE8 ); - - LZSSE8_FreeFastParseState( state ); - } - - break; - } - - default: - - printf( "Invalid stream type\n" ); - exit( 1 ); - - } - } - - fclose( inputFile ); - fclose( outputFile ); - - return 0; -} - diff --git a/contrib/lzsse/lzsse2/lzsse2.cpp b/contrib/lzsse/lzsse2/lzsse2.cpp deleted file mode 100644 index d521a84553a..00000000000 --- a/contrib/lzsse/lzsse2/lzsse2.cpp +++ /dev/null @@ -1,1080 +0,0 @@ -/* -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#include -#include -#include -#include - -#include "lzsse2_platform.h" -#include "lzsse2.h" - -#pragma warning ( disable : 4127 ) - -namespace -{ - // Constants - most of these should not be changed without corresponding code changes because it will break many things in unpredictable ways. - const uint32_t WINDOW_BITS = 16; - const uint32_t MIN_MATCH_LENGTH = 3; - const uint32_t LZ_WINDOW_SIZE = 1 << WINDOW_BITS; - const uint32_t LZ_WINDOW_MASK = LZ_WINDOW_SIZE - 1; - const uint32_t OPTIMAL_HASH_BITS = 20; - const uint32_t OPTIMAL_BUCKETS_COUNT = 1 << OPTIMAL_HASH_BITS; - const uint32_t OPTIMAL_HASH_MASK = OPTIMAL_BUCKETS_COUNT - 1; - const uint32_t MIN_COMPRESSION_SIZE = 32; - const uint32_t END_PADDING_LITERALS = 16; - const int32_t NO_MATCH = -1; - const int32_t EMPTY_NODE = -1; - const uint32_t MIN_LITERAL_COUNT = 8; - const uint32_t HASH_MULTIPLIER = 4013; - const uint32_t HASH_REMOVAL_MULTIPLIER = HASH_MULTIPLIER * HASH_MULTIPLIER; - const uint32_t CONTROL_BITS = 4; - const uint32_t LITERAL_BITS = 8; - const uint32_t OFFSET_BITS = 16; - const uint32_t BASE_MATCH_BITS = OFFSET_BITS + CONTROL_BITS; - const uint32_t SINGLE_LITERAL_COST = CONTROL_BITS + LITERAL_BITS; - const uint32_t DOUBLE_LITERAL_COST = SINGLE_LITERAL_COST + LITERAL_BITS; - const uint32_t EXTENDED_MATCH_BOUND = ( 1 << CONTROL_BITS ) - 1; - const uint32_t CONTROL_BLOCK_SIZE = sizeof( __m128i ); - const uint32_t CONTROLS_PER_BLOCK = 32; - const uint32_t LITERALS_PER_CONTROL = 2; - const uint32_t MAX_INPUT_PER_CONTROL = 2; - const size_t OUTPUT_BUFFER_SAFE = EXTENDED_MATCH_BOUND * CONTROLS_PER_BLOCK; - const size_t INPUT_BUFFER_SAFE = MAX_INPUT_PER_CONTROL * CONTROLS_PER_BLOCK; - const uint16_t INITIAL_OFFSET = MIN_MATCH_LENGTH; - const size_t SKIP_MATCH_LENGTH = 128; - const uint32_t NO_SKIP_LEVEL = 17; -} - -struct Arrival -{ - size_t cost; - int32_t from; - int32_t to; - uint16_t offset; -}; - -struct TreeNode -{ - int32_t children[ 2 ]; -}; - -struct LZSSE2_OptimalParseState -{ - // Note, we should really replace this with a BST, hash chaining works but is *slooooooooooooooow* for optimal parse. - int32_t roots[ OPTIMAL_BUCKETS_COUNT ]; - - TreeNode window[ LZ_WINDOW_SIZE ]; - - Arrival* arrivals; - - size_t bufferSize; -}; - - -LZSSE2_OptimalParseState* LZSSE2_MakeOptimalParseState( size_t bufferSize ) -{ - if ( bufferSize > 0 && ( SIZE_MAX / sizeof( Arrival ) ) < bufferSize ) - { - return nullptr; - } - - LZSSE2_OptimalParseState* result = reinterpret_cast< LZSSE2_OptimalParseState* >( ::malloc( sizeof( LZSSE2_OptimalParseState ) ) ); - - result->bufferSize = bufferSize; - - if ( result != nullptr ) - { - result->arrivals = reinterpret_cast< Arrival* >( ::malloc( sizeof( Arrival ) * bufferSize ) ); - - if ( result->arrivals == nullptr ) - { - LZSSE2_FreeOptimalParseState( result ); - - result = nullptr; - } - } - - return result; -} - - -void LZSSE2_FreeOptimalParseState( LZSSE2_OptimalParseState* toFree ) -{ - ::free( toFree->arrivals ); - - toFree->arrivals = nullptr; - - ::free( toFree ); -} - - -inline uint32_t CalculateHash( const uint8_t* inputCursor ) -{ - return ( uint32_t( inputCursor[ 0 ] ) * HASH_MULTIPLIER * HASH_MULTIPLIER + uint32_t( inputCursor[ 1 ] ) * HASH_MULTIPLIER + uint32_t( inputCursor[ 2 ] ) ) & OPTIMAL_HASH_MASK; -} - - -struct Match -{ - size_t length; - int32_t position; - uint16_t offset; -}; - - -inline Match SearchAndUpdateFinder( LZSSE2_OptimalParseState& state, const uint8_t* input, const uint8_t* inputCursor, const uint8_t* inputEnd, uint32_t cutOff ) -{ - Match result; - - int32_t position = static_cast( inputCursor - input ); - - result.position = NO_MATCH; - result.length = MIN_MATCH_LENGTH; - result.offset = 0; - - size_t lengthToEnd = inputEnd - inputCursor; - int32_t lastPosition = position - ( LZ_WINDOW_SIZE - 1 ); - uint32_t hash = CalculateHash( inputCursor ); - - lastPosition = lastPosition > 0 ? lastPosition : 0; - - int32_t treeCursor = state.roots[ hash ]; - - state.roots[ hash ] = position; - - int32_t* left = &state.window[ position & LZ_WINDOW_MASK ].children[ 1 ]; - int32_t* right = &state.window[ position & LZ_WINDOW_MASK ].children[ 0 ]; - size_t leftLength = 0; - size_t rightLength = 0; - - for ( ;; ) - { - if ( cutOff-- == 0 || treeCursor < lastPosition ) - { - *left = *right = EMPTY_NODE; - break; - } - - TreeNode& currentNode = state.window[ treeCursor & LZ_WINDOW_MASK ]; - const uint8_t* key = input + treeCursor; - size_t matchLength = leftLength < rightLength ? leftLength : rightLength; - - uint16_t matchOffset = static_cast( position - treeCursor ); - size_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; - - while ( matchLength < lengthToEnd ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); - - unsigned long matchBytes; - - _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != 16 ) - { - break; - } - } - - matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; - - size_t truncatedMatchLength = matchLength < maxLength ? matchLength : maxLength; - - if ( truncatedMatchLength >= result.length ) - { - result.length = truncatedMatchLength; - result.offset = matchOffset; - result.position = treeCursor; - } - - if ( matchLength == lengthToEnd ) - { - *left = currentNode.children[ 1 ]; - *right = currentNode.children[ 0 ]; - break; - } - - if ( inputCursor[ matchLength ] < key[ matchLength ] || ( matchLength == lengthToEnd ) ) - { - *left = treeCursor; - left = currentNode.children; - treeCursor = *left; - leftLength = matchLength; - } - else - { - *right = treeCursor; - right = currentNode.children + 1; - treeCursor = *right; - rightLength = matchLength; - } - } - - // Special RLE overlapping match case, the LzFind style match above doesn't work very well with our - // restriction of overlapping matches having offsets of at least 16. - // Suffix array seems like a better option to handling this. - { - // Note, we're detecting long RLE here, but if we have an offset too close, we'll sacrifice a fair - // amount of decompression performance to load-hit-stores. - int32_t matchPosition = position - ( sizeof( __m128i ) * 2 ); - - if ( matchPosition >= 0 ) - { - uint16_t matchOffset = static_cast( position - matchPosition ); - const uint8_t* key = input + matchPosition; - size_t matchLength = 0; - - while ( matchLength < lengthToEnd ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); - - unsigned long matchBytes; - - _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != 16 ) - { - break; - } - - } - - matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; - - if ( matchLength >= result.length ) - { - result.length = matchLength; - result.offset = matchOffset; - result.position = matchPosition; - } - } - } - - return result; -} - - -size_t LZSSE2_CompressOptimalParse( LZSSE2_OptimalParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength, unsigned int level ) -{ - if ( outputLength < inputLength || state->bufferSize < inputLength ) - { - // error case, output buffer not large enough. - return 0; - } - - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - if ( inputLength < MIN_COMPRESSION_SIZE ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - const uint8_t* inputEnd = input + inputLength; - Arrival* arrivalWatermark = state->arrivals; - Arrival* arrival = state->arrivals; - uint32_t cutOff = 1 << level; - - for ( int32_t* rootCursor = state->roots, *end = rootCursor + OPTIMAL_BUCKETS_COUNT; rootCursor < end; rootCursor += 4 ) - { - rootCursor[ 0 ] = EMPTY_NODE; - rootCursor[ 1 ] = EMPTY_NODE; - rootCursor[ 2 ] = EMPTY_NODE; - rootCursor[ 3 ] = EMPTY_NODE; - } - - for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) - { - /*Match dummy = */ SearchAndUpdateFinder( *state, input, inputCursor, inputEnd - END_PADDING_LITERALS, cutOff ); - - ++inputCursor; - } - - arrival->cost = LITERAL_BITS * MIN_MATCH_LENGTH; - arrival->from = -1; - arrival->offset = 0; - - // loop through each character and project forward the matches at that character to calculate the cheapest - // path of arrival for each individual character. - for ( const uint8_t* earlyEnd = inputEnd - END_PADDING_LITERALS; inputCursor < earlyEnd; ++inputCursor, ++arrival ) - { - uint32_t lengthToEnd = static_cast< uint32_t >( earlyEnd - inputCursor ); - int32_t currentPosition = static_cast< int32_t >( inputCursor - input ); - Arrival* literalFirst = arrival + 1; - Arrival* literalSecond = arrival + 2; - size_t arrivalCost = arrival->cost; - - // NOTE - we currently assume only 2 literals filled in here, because the minimum match length is 3. - // If we wanted to go with a higher minimum match length, we would need to fill in more literals before hand. - // Also, because there is a maximum of 2 literals per control block assumed. - - // project forward the cost of a single literal - if ( literalFirst > arrivalWatermark || literalFirst->cost > ( arrival->cost + SINGLE_LITERAL_COST ) ) - { - literalFirst->cost = arrival->cost + SINGLE_LITERAL_COST; - literalFirst->from = currentPosition; - literalFirst->offset = 0; - - arrivalWatermark = literalFirst > arrivalWatermark ? literalFirst : arrivalWatermark; - } - - // project forward the cost of two literals - if ( lengthToEnd > 1 ) - { - if ( literalSecond > arrivalWatermark || literalFirst->cost > ( arrival->cost + DOUBLE_LITERAL_COST ) ) - { - literalSecond->cost = arrival->cost + DOUBLE_LITERAL_COST; - literalSecond->from = currentPosition; - literalSecond->offset = 0; - - arrivalWatermark = literalSecond > arrivalWatermark ? literalSecond : arrivalWatermark; - } - } - else - { - continue; - } - - Match match = SearchAndUpdateFinder( *state, input, inputCursor, earlyEnd, cutOff ); - - if ( match.position != NO_MATCH ) - { - for ( size_t matchedLength = MIN_MATCH_LENGTH, end = match.length + 1; matchedLength < end; ++matchedLength ) - { - Arrival* matchArrival = arrival + matchedLength; - size_t matchCost = arrivalCost + BASE_MATCH_BITS; - - if ( matchedLength > EXTENDED_MATCH_BOUND ) - { - matchCost += ( ( matchedLength - 1 ) / EXTENDED_MATCH_BOUND ) * CONTROL_BITS; - } - - if ( matchArrival > arrivalWatermark || matchArrival->cost > matchCost ) - { - matchArrival->cost = matchCost; - matchArrival->from = currentPosition; - matchArrival->offset = match.offset; - - arrivalWatermark = matchArrival > arrivalWatermark ? matchArrival : arrivalWatermark; - } - } - - if ( match.length > SKIP_MATCH_LENGTH && level < NO_SKIP_LEVEL ) - { - arrival += match.length - LITERALS_PER_CONTROL; - inputCursor += match.length - LITERALS_PER_CONTROL; - } - } - } - - // If this would cost more to encode than it would if it were just literals, encode it with no control blocks, - // just literals - if ( ( arrivalWatermark->cost + END_PADDING_LITERALS * LITERAL_BITS + CONTROLS_PER_BLOCK * CONTROL_BITS ) > ( inputLength * LITERAL_BITS ) ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - Arrival* previousPathNode; - - // now trace the actual optimal parse path back, connecting the nodes in the other direction. - for ( const Arrival* pathNode = arrivalWatermark; pathNode->from > 0; pathNode = previousPathNode ) - { - previousPathNode = state->arrivals + ( pathNode->from - MIN_MATCH_LENGTH ); - - previousPathNode->to = static_cast( ( pathNode - state->arrivals ) + MIN_MATCH_LENGTH ); - } - - uint8_t* outputCursor = output; - - memcpy( outputCursor, input, MIN_MATCH_LENGTH ); - - outputCursor += MIN_MATCH_LENGTH; - - uint8_t* currentControlBlock = outputCursor; - uint32_t currentControlCount = 0; - uint32_t totalControlCount = 0; - - outputCursor += CONTROL_BLOCK_SIZE; - - Arrival* nextPathNode; - - size_t totalPathLength = MIN_MATCH_LENGTH; - uint16_t previousOffset = INITIAL_OFFSET; - - bool lastControlIsNop = false; - - // Now walk forwards again and actually write out the data. - for ( const Arrival* pathNode = state->arrivals; pathNode < arrivalWatermark; pathNode = nextPathNode ) - { - int32_t currentPosition = static_cast< int32_t >( ( pathNode - state->arrivals ) + MIN_MATCH_LENGTH ); - - nextPathNode = state->arrivals + ( pathNode->to - MIN_MATCH_LENGTH ); - - size_t pathDistance = nextPathNode - pathNode; - - totalPathLength += pathDistance; - - lastControlIsNop = false; - - if ( pathDistance < MIN_MATCH_LENGTH ) - { - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - currentControlCount = 0; - } - - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - ( static_cast( pathDistance ) - 1 ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - ( static_cast< uint8_t >( pathDistance ) - 1 ) << CONTROL_BITS; - } - - // output the literals. - for ( int32_t where = 0; where < pathDistance; ++where ) - { - const uint8_t* currentInput = input + currentPosition + where; - - outputCursor[ where ] = *currentInput ^ *( currentInput - previousOffset ); - } - - outputCursor += pathDistance; - - ++totalControlCount; - ++currentControlCount; - } - else - { - size_t toEncode = pathDistance - 1; //note, we always subtract one here, because the first control block of the match encodes this way - - // make sure the control block for the first part of the match has been allocated - // note, this is idempontent if we have not actually incremented the control count and we try this again. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - currentControlCount = 0; - } - - // output the offset (after control block containing the first control nibble for this match). - *reinterpret_cast< uint16_t* >( outputCursor ) = nextPathNode->offset ^ previousOffset; - - previousOffset = nextPathNode->offset; - - outputCursor += sizeof( uint16_t ); - - for ( ;; ) - { - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - currentControlCount = 0; - } - - if ( toEncode >= EXTENDED_MATCH_BOUND ) - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( EXTENDED_MATCH_BOUND ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; - } - - toEncode -= EXTENDED_MATCH_BOUND; - - ++totalControlCount; - ++currentControlCount; - - } - else - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( toEncode ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( toEncode ) << CONTROL_BITS; - } - - if ( toEncode == 0 && currentControlCount == 0 ) - { - lastControlIsNop = true; - } - - ++totalControlCount; - ++currentControlCount; - - break; - } - } - } - } - - if ( lastControlIsNop ) - { - outputCursor -= CONTROL_BLOCK_SIZE; - } - - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy remaining literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - - return outputCursor - output; -} - - -size_t LZSSE2_Decompress( const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) -{ - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - // Length it not work compressing, just copy initial values - if ( outputLength == inputLength ) - { - memcpy( output, input, outputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - uint8_t* outputCursor = output; - - // The offset starts off as the minimum match length. We actually need it at least two - // characters back because we need them to be set to xor out the literals from the match data. - size_t offset = INITIAL_OFFSET; - __m128i previousCarryHi = _mm_setzero_si128(); - - *( outputCursor++ ) = *( inputCursor++ ); - *( outputCursor++ ) = *( inputCursor++ ); - *( outputCursor++ ) = *( inputCursor++ ); - - // What these macros do: - // Decode a single literal run or match run for a single control nibble. - // How they do it: - // - Read the *unaligned* input (in the case of LZSSE-F - twice, for LZSSE-O we read once) - one goes into an SSE register, - // because it could either be literals or an offset (or nothing at all). The low byte of streamBytesRead controls how much we advance - // the input cursor. - // - Used a contived set of casts to sign extend the "read offset" control mask and then use it to mask the input word, - // which is then xor'd against the offset, for a "branchless" conditional move into the offset which - // has been carried over from the previous literal/match block. Note, this ends up doing better than a cmov on most - // modern processors. But we need to pre-xor the input offset. - // - We then load the match data from output buffer (offset back from the current output point). Unconditional load here. - // - We broadcast the "from literal" control mask from the current least significant byte of the SSE register using a shuffle epi-8 - // - We mask the literals with that SSE register wide mask. - // - The literals have been pre-xor'd with the data read in as match data, so we use an xor to branchlessly choose between the two. - // In this case, it ends up a better option than a blendv on most processors. - // - Store the block. We store all 16 bytes of the SSE register (due to some constraints in the format of the data, we won't - // go past the end of the buffer), but we may overlap this. - // - bytesOut controls how much we advance the output cursor. - // - We use 8 bit shifts to advance all the controls up to the next byte. There is some variable sized register trickery that - // x86/x64 is great for as long as we don't anger the register renamer. - -#define DECODE_STEP( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - \ - uint64_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_cvtsi64_si128( inputWord ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - readOffsetHalf##HILO >>= 8; \ - \ - const uint8_t* matchPointer = outputCursor - offset; \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( fromLiteral, literals ); \ - \ - fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - bytesOutHalf##HILO >>= 8; \ - streamBytesReadHalf##HILO >>= 8; \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_HALF( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast< const uint16_t* >( inputCursor ); \ - __m128i literals = _mm_cvtsi64_si128( inputWord ); \ - \ - offset ^= static_cast< size_t >( static_cast< ptrdiff_t >( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - const uint8_t* matchPointer = outputCursor - offset; \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( fromLiteral, literals ); \ - \ - fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_END( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast< const uint16_t* >( inputCursor ); \ - __m128i literals = _mm_cvtsi64_si128( inputWord ); \ - \ - offset ^= static_cast< size_t >( static_cast< ptrdiff_t >( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - const uint8_t* matchPointer = outputCursor - offset; \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( fromLiteral, literals ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Hi, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HALF_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HALF_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Hi, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_END_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_END_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Hi, CHECKMATCH, CHECKBUFFERS ) - - __m128i nibbleMask = _mm_set1_epi8( 0xF ); - __m128i literalsPerControl = _mm_set1_epi8( LITERALS_PER_CONTROL ); - - // Note, we use this block here because it allows the "fake" inputEarlyEnd/outputEarlyEnd not to cause register spills - // in the decompression loops. And yes, that did actually happen. - { -#pragma warning ( push ) -#pragma warning ( disable : 4101 ) - - const uint8_t* inputEarlyEnd; //= ( input + inputLength ) - END_PADDING_LITERALS; - uint8_t* outputEarlyEnd;// = ( output + outputLength ) - END_PADDING_LITERALS; - -#pragma warning ( pop ) - - // "Safe" ends to the buffer, before the input/output cursors hit these, we can loop without overflow checks. - const uint8_t* inputSafeEnd = ( input + inputLength ) - INPUT_BUFFER_SAFE; - uint8_t* outputSafeEnd = ( output + outputLength ) - OUTPUT_BUFFER_SAFE; - - // Decoding loop with offset output buffer underflow test, but no buffer overflow tests, assumed to end at a safe distance - // from overflows - while ( ( outputCursor - output ) < LZ_WINDOW_SIZE && outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) - { - // load the control block - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - - // split the control block into high and low nibbles. - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation - // this will essentially be ignored later on. - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - // Work out the carry for the low nibbles (which will be used with the high controls to put them into - // match without offset read mode). - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - - // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save - // the calculated carry to use that byte next iteration. - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); - - previousCarryHi = carryHi; - - // I want 128 set bits please. - __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - // Calcualting the bytes to output to the stream. Basically, we are subtracting negative one from the control value if the - // carry is not set. This works because the masks produced by comparisons are the equivalent to negative one, which - // make this a conditional increment. - __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_xor_si128( shiftedCarryHi, allSet ) ); - __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_xor_si128( carryLo, allSet ) ); - - // Calculate the number of bytes to read per control. - // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_min_epi8( literalsPerControl, bytesOutLo ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_min_epi8( literalsPerControl, bytesOutHi ) ); - - // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); - - // Masks whether we are reading literals - set if the carry is not set and these are literals. - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - // Advance the input past the control block. - inputCursor += CONTROL_BLOCK_SIZE; - - { - // Pull out the bottom halves off the SSE registers from before - we want these - // things in GPRs for the more linear logic. - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_HALF_LO( true, false ); - DECODE_STEP_HALF_HI( true, false ); - } - - { - // Now the top halves. - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_END_LO( true, false ); - DECODE_STEP_END_HI( true, false ); - } - } - - // Decoding loop with no buffer checks, but will end at a safe distance from the end of the buffers. - // Note, when we get here we have already reached the point in the output buffer which is *past* where we can underflow - // due to a bad match offset. - while ( outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) - { - // This code is the same as the loop above, see comments there - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); // where we take the carry from the previous hi values - - previousCarryHi = carryHi; - - __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_xor_si128( shiftedCarryHi, neg1 ) ); - __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_xor_si128( carryLo, neg1 ) ); - - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_min_epi8( literalsPerControl, bytesOutLo ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_min_epi8( literalsPerControl, bytesOutHi ) ); - - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); - - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - inputCursor += CONTROL_BLOCK_SIZE; - - { - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_HALF_LO( false, false ); - DECODE_STEP_HALF_HI( false, false ); - } - - { - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_END_LO( false, false ); - DECODE_STEP_END_HI( false, false ); - } - } - } - - // Decoding loop with all buffer checks. - { - const uint8_t* inputEarlyEnd; - uint8_t* outputEarlyEnd; - inputEarlyEnd = (( input + inputLength ) - END_PADDING_LITERALS); - outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; - - while ( outputCursor < outputEarlyEnd && inputCursor < inputEarlyEnd ) - { - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); // where we take the carry from the previous hi values - - previousCarryHi = carryHi; - - __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_xor_si128( shiftedCarryHi, neg1 ) ); - __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_xor_si128( carryLo, neg1 ) ); - - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_min_epi8( literalsPerControl, bytesOutLo ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_min_epi8( literalsPerControl, bytesOutHi ) ); - - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); - - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - inputCursor += CONTROL_BLOCK_SIZE; - - if ( inputCursor > inputEarlyEnd ) - goto BUFFER_END; - - { - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_HALF_LO( true, true ); - DECODE_STEP_HALF_HI( true, true ); - } - - { - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_END_LO( true, true ); - DECODE_STEP_END_HI( true, true ); - } - } - -BUFFER_END: - - // When we get here, we have either advanced the right amount on both cursors - // or something bad happened, so leave it as is, so we can tell where - // the error happened. - if ( inputCursor == inputEarlyEnd && outputCursor == outputEarlyEnd ) - { - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy any trailing literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - } - } - -MATCH_UNDERFLOW: - - return outputCursor - output; -} diff --git a/contrib/lzsse/lzsse2/lzsse2.h b/contrib/lzsse/lzsse2/lzsse2.h deleted file mode 100644 index ea908b85fb5..00000000000 --- a/contrib/lzsse/lzsse2/lzsse2.h +++ /dev/null @@ -1,90 +0,0 @@ -/* -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef LZSSE2_H__ -#define LZSSE2_H__ - -#pragma once - -/* LZSSE2 - x64/SSE targeted codec for better performance with high compression ratio data/more optimal compressors. - * Supports minimum 3 byte matches, maximum 16 bytes of match per control word and 2 byte literal runs per control word. - */ - -#ifdef __cplusplus -extern "C" -{ -#endif - - /* Re-usable parse state object for compression. */ -typedef struct LZSSE2_OptimalParseState LZSSE2_OptimalParseState; - -/* Allocate the parse state for compression - returns null on failure. Note - Buffersize has to be greater or equal to any inputLength used with LZSSE2_CompressOptimalParse */ -LZSSE2_OptimalParseState* LZSSE2_MakeOptimalParseState( size_t bufferSize ); - -/* De-allocate the parse state for compression */ -void LZSSE2_FreeOptimalParseState( LZSSE2_OptimalParseState* toFree ); - -/* "Optimal" compression routine. -* Will compress data into LZSSE2 format, uses hash BST matching to find matches and run an optimal parse (high relative memory usage). Requires SSE 4.1. -* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. -* input : Buffer containing uncompressed data to be compressed. May not be null. -* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. -* output : Buffer that will receive the compressed output. -* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, -* The compressed data should never be longer than inputLength, as in this case the data is stored raw. -* level : The compression level to use for this file 1->17, 17 is highest compression, 0 is least -* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state -* concurrently. -* -* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). -*/ -size_t LZSSE2_CompressOptimalParse( LZSSE2_OptimalParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int level ); - -/* Decompression routine. -* This routine will decompress data in the LZSSE2 format and currently requires SSE 4.1 and is targeted at x64. -* It will perform poorly on x86 due to hunger for registers. -* input : Buffer containing compressed input block. May not be null. -* inputLength : Length of the compressed data in the input buffer - note, this should be under 2GB -* output : Buffer that will received the de-compressed output. Note, that this needs to be at least outputLength long. -* May not be null. -* outputLength : The length of the compressed output - note, this should be under 2GB -* -* Provided that input and output are valid pointers to buffers of at least their specified size, this routine -* should be memory safe - both match pointer checks and input/output buffer checks exist. -* -* Returns the size of the decompressed data, which will be less than outputLength in the event of an error (number of bytes -* will indicate where in the output stream the error occured). -* -* Note that this data is not hash verified, errors that occur are either from a misformed stream or bad buffer sizes. -* Remember, corrupt data can still be valid to decompress. -*/ -size_t LZSSE2_Decompress( const void* input, size_t inputLength, void* output, size_t outputLength ); - -#ifdef __cplusplus -} -#endif - -#endif /* -- LZSSE2_H__ */ diff --git a/contrib/lzsse/lzsse2/lzsse2_platform.h b/contrib/lzsse/lzsse2/lzsse2_platform.h deleted file mode 100644 index 5cbab6b0600..00000000000 --- a/contrib/lzsse/lzsse2/lzsse2_platform.h +++ /dev/null @@ -1,73 +0,0 @@ -/* -Copyright (c) 2016, Brian Marshall -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef LZSSE2_PLATFORM_H__ -#define LZSSE2_PLATFORM_H__ - -#pragma once - -/* - Compiler/Platform detection based on the table from: - https://blogs.msdn.microsoft.com/vcblog/2015/12/04/clang-with-microsoft-codegen-in-vs-2015-update-1/ -*/ - -#ifdef _MSC_VER - -/* - Microsoft Visual Studio Support. - C1xx/C2, Clang/C2 and Clang/LLVM all support the Microsoft header files and _BitScanForward - - Note: if you receive errors with the intrinsics make sure that you have SSE4.1 support enabled. - For example with Clang include "-msse4.1" on the command line -*/ -#include - -#else /* _MSC_VER */ - -#ifdef __GNUC__ - -/* - GCC -*/ - -/* - Note: including just would be sufficient, but including x86intrin is a better match to intrin.h on Visual Studio as - both include all intrinsics for the enabled processor, rather than just SSE4.1. -*/ -#include -/* _BitScanForward is Visual Studio specific. */ -#define _BitScanForward(x, m) *(x) = __builtin_ctz(m) - -#else - -/* -If you hit the error below, then add detection for your compiler/platform to this header file. -*/ -#error Platform not supported - -#endif /* __GNUC__ */ -#endif /* _MSC_VER */ - -#endif /* -- LZSSE2_PLATFORM_H__ */ diff --git a/contrib/lzsse/lzsse4/lzsse4.cpp b/contrib/lzsse/lzsse4/lzsse4.cpp deleted file mode 100644 index cf9f6fc6e60..00000000000 --- a/contrib/lzsse/lzsse4/lzsse4.cpp +++ /dev/null @@ -1,1499 +0,0 @@ -/* -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#include -#include -#include -#include - -#include "lzsse4_platform.h" -#include "lzsse4.h" - -#pragma warning ( disable : 4127 ) - -namespace -{ - // Constants - most of these should not be changed without corresponding code changes because it will break many things in unpredictable ways. - const uint32_t WINDOW_BITS = 16; - const uint32_t MIN_MATCH_LENGTH = 4; - const uint32_t LZ_WINDOW_SIZE = 1 << WINDOW_BITS; - const uint32_t LZ_WINDOW_MASK = LZ_WINDOW_SIZE - 1; - const uint32_t FAST_HASH_BITS = 20; // You can change this - more bits = more matches, less bits = more cache hits - const uint32_t FAST_BUCKETS_COUNT = 1 << FAST_HASH_BITS; - const uint32_t FAST_HASH_MASK = FAST_BUCKETS_COUNT - 1; - const uint32_t MIN_COMPRESSION_SIZE = 32; - const uint32_t END_PADDING_LITERALS = 16; - const int32_t NO_MATCH = -1; - const int32_t EMPTY_NODE = -1; - const uint32_t MIN_LITERAL_COUNT = 8; - const uint32_t CONTROL_BITS = 4; - const uint32_t LITERAL_BITS = 8; - const uint32_t OFFSET_BITS = 16; - const uint32_t BASE_MATCH_BITS = OFFSET_BITS + CONTROL_BITS; - const uint32_t OFFSET_SIZE = 2; - const uint32_t EXTENDED_MATCH_BOUND = ( 1 << CONTROL_BITS ) - 1; - const uint32_t CONTROL_BLOCK_SIZE = sizeof( __m128i ); - const uint32_t CONTROLS_PER_BLOCK = 32; - const uint32_t LITERALS_PER_CONTROL = 4; - const uint32_t MAX_INPUT_PER_CONTROL = 4; - const size_t OUTPUT_BUFFER_SAFE = EXTENDED_MATCH_BOUND * CONTROLS_PER_BLOCK; - const size_t INPUT_BUFFER_SAFE = MAX_INPUT_PER_CONTROL * CONTROLS_PER_BLOCK; - const uint16_t INITIAL_OFFSET = MIN_MATCH_LENGTH; - const uint32_t OPTIMAL_HASH_BITS = 20; - const uint32_t OPTIMAL_BUCKETS_COUNT = 1 << OPTIMAL_HASH_BITS; - const uint32_t OPTIMAL_HASH_MASK = OPTIMAL_BUCKETS_COUNT - 1; - const size_t SKIP_MATCH_LENGTH = 128; - const uint32_t NO_SKIP_LEVEL = 17; -} - - -struct LZSSE4_FastParseState -{ - int32_t buckets[ FAST_BUCKETS_COUNT ]; // stores the first matching position, we can then look at the rest of the matches by tracing through the window. -}; - - -LZSSE4_FastParseState* LZSSE4_MakeFastParseState() -{ - return new LZSSE4_FastParseState(); -} - - -void LZSSE4_FreeFastParseState( LZSSE4_FastParseState* toFree ) -{ - delete toFree; -} - - -inline void SetHash( LZSSE4_FastParseState* state, uint32_t hash, const uint8_t* input, const uint8_t* inputCursor ) -{ - int32_t position = static_cast( inputCursor - input ); - - state->buckets[ hash & FAST_HASH_MASK ] = position; -} - -// Simple fast hash function - actually what is used in snappy and derivatives -// There's probably better, but I haven't spent time focusing on this area yet. -inline uint32_t HashFast( const uint8_t* inputCursor ) -{ - return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - FAST_HASH_BITS ); -} - -size_t LZSSE4_CompressFast( LZSSE4_FastParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) -{ - if ( outputLength < inputLength ) - { - // error case, output buffer not large enough. - return 0; - } - - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - if ( inputLength < MIN_COMPRESSION_SIZE ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - const uint8_t* inputEnd = input + inputLength; - const uint8_t* inputEarlyEnd = inputEnd - END_PADDING_LITERALS; - uint8_t* outputCursor = output; - uint8_t* outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; - uint32_t hash = 0; - - // initialize hash to empty - for ( int32_t* where = state->buckets, *end = state->buckets + FAST_BUCKETS_COUNT; where < end; where += 4 ) - { - where[ 0 ] = -1; - where[ 1 ] = -1; - where[ 2 ] = -1; - where[ 3 ] = -1; - } - - // initial literals that wont be compressed - for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) - { - hash = HashFast( inputCursor ); - - SetHash( state, hash, input, inputCursor ); - - *( outputCursor++ ) = *( inputCursor++ ); - } - - uint8_t* currentControlBlock = outputCursor; - uint32_t currentControlCount = 0; - uint16_t previousOffset = INITIAL_OFFSET; - size_t literalsToFlush = 0; - - outputCursor += CONTROL_BLOCK_SIZE; - - bool lastControlIsNop = false; - - // Loop through the data until we hit the end of one of the buffers (minus the end padding literals) - while ( inputCursor < inputEarlyEnd && outputCursor <= outputEarlyEnd ) - { - lastControlIsNop = false; - - hash = HashFast( inputCursor ); - - int matchPosition = state->buckets[ hash & FAST_HASH_MASK ]; - int currentPosition = static_cast< int32_t >( inputCursor - input ); - uint32_t matchLength = 0; - uint16_t matchOffset = static_cast< uint16_t >( currentPosition - matchPosition ); - - // If we had a hit in the hash and it wasn't outside the window. - if ( matchPosition >= 0 && ( currentPosition - matchPosition ) < ( LZ_WINDOW_SIZE - 1 ) ) - { - const uint8_t* matchCandidate = input + matchPosition; - uint32_t lengthToEnd = static_cast< uint32_t >( inputEarlyEnd - inputCursor ); - // Here we limit the hash length to prevent overlap matches with offset less than 16 bytes - uint32_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; - - // Find how long the match is 16 bytes at a time. - while ( matchLength < maxLength ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( matchCandidate + matchLength ) ); - - unsigned long matchBytes; - - // Finds the number of equal bytes at the start of the 16 - _BitScanForward( &matchBytes, ( static_cast< unsigned long >( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != sizeof( __m128i ) ) - { - break; - } - } - - matchLength = matchLength < maxLength ? matchLength : maxLength; - } - - // If we have at least the minimum match length (4 bytes) - if ( matchLength >= MIN_MATCH_LENGTH ) - { - // Do we have literals to flush before the match? - if ( literalsToFlush > 0 ) - { - // Start a new control block if we need one. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - // Would be larger than compressed size, get out! - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); - - // flush the literals. - // note the xor against the data that would be read in the match. - for ( uint32_t where = 0; where < literalsToFlush; ++where ) - { - const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); - - *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); - } - - ++currentControlCount; - - literalsToFlush = 0; - - // Would be larger than compressed size, get out! - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - // Start a new control block if the previous one is full. - // Note this is done before the offset is written out - the offset - // is always written after the control block containing the first - // control in the match. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - // The match length value we are encoding. - size_t toEncode = matchLength; - - // Write the offset out - note the xor with the previous offset. - *reinterpret_cast< uint16_t* >( outputCursor ) = matchOffset ^ previousOffset; - - previousOffset = matchOffset; - outputCursor += sizeof( uint16_t ); - - for ( ;; ) - { - // Check if we need to start a new control block - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - // If the encode size is greater than we can hold in a control, write out a full match length - // control, subtract full control value from the amount to encode and loop around again. - if ( toEncode >= EXTENDED_MATCH_BOUND ) - { - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( EXTENDED_MATCH_BOUND ) << 4 ); - - toEncode -= EXTENDED_MATCH_BOUND; - - ++currentControlCount; - } - else // Write out the remaining match length control. Could potentially be zero. - { - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( toEncode ) << 4 ); - - if ( currentControlCount == 0 && toEncode == 0 ) - { - lastControlIsNop = true; - } - - ++currentControlCount; - - break; - } - } - - // Update the value into the hash for future matches. - SetHash( state, hash, input, inputCursor ); - - ++inputCursor; - - // Hash all the other values in the match too. - for ( const uint8_t* nextArrival = inputCursor + matchLength - 1; inputCursor < nextArrival; ++inputCursor ) - { - hash = HashFast( inputCursor ); - SetHash( state, hash, input, inputCursor ); - } - } - else - { - // One more literal to write out. - ++literalsToFlush; - - // If we have reached the maximum number of literals allowed in the control, flush them out. - if ( literalsToFlush == LITERALS_PER_CONTROL ) - { - // Check if the control block is full and we need start a new one. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( ( static_cast( LITERALS_PER_CONTROL - 1 ) ) << 4 ); - - ++currentControlCount; - - *reinterpret_cast< uint32_t* >( outputCursor ) = - *reinterpret_cast< const uint32_t* >( inputCursor - 3 ) ^ - *reinterpret_cast< const uint32_t* >( ( inputCursor - 3 ) - previousOffset ); - - outputCursor += 4; - - //*( outputCursor++ ) = *( inputCursor - 3 ) ^ *( ( inputCursor - 3 ) - previousOffset ); - //*( outputCursor++ ) = *( inputCursor - 2 ) ^ *( ( inputCursor - 2 ) - previousOffset ); - //*( outputCursor++ ) = *( inputCursor - 1 ) ^ *( ( inputCursor - 1 ) - previousOffset ); - //*( outputCursor++ ) = *inputCursor ^ *( inputCursor - previousOffset ); - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - - literalsToFlush = 0; - } - - // Update the hash with this byte - SetHash( state, hash, input, inputCursor ); - - ++inputCursor; - } - } - - // If we would create a compression output bigger than or equal to the input, just copy the input to the output and return equal size. - if ( ( ( outputCursor + literalsToFlush + ( currentControlCount == CONTROLS_PER_BLOCK ? CONTROL_BLOCK_SIZE : 0 ) ) ) >= output + inputLength - END_PADDING_LITERALS ) - { - memcpy( output, input, inputLength ); - - outputCursor = output + inputLength; - } - else - { - // Flush any remaining literals. - if ( literalsToFlush > 0 ) - { - lastControlIsNop = false; - - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - } - - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); - - for ( uint32_t where = 0; where < literalsToFlush; ++where ) - { - const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); - - *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); - } - - ++currentControlCount; - } - - // Need to finish off shifting the final control block into the low nibble if there is no second nibble - if ( ( currentControlCount & 1 ) > 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] >>= 4; - } - - if ( lastControlIsNop ) - { - outputCursor -= CONTROL_BLOCK_SIZE; - } - - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy remaining literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - } - - // Return the size of the compressed data. - return outputCursor - output; -} - - -struct Arrival -{ - size_t cost; - int32_t from; - int32_t to; - uint16_t offset; -}; - -struct TreeNode -{ - int32_t children[ 2 ]; -}; - -struct LZSSE4_OptimalParseState -{ - // Note, we should really replace this with a BST, hash chaining works but is *slooooooooooooooow* for optimal parse. - int32_t roots[ OPTIMAL_BUCKETS_COUNT ]; - - TreeNode window[ LZ_WINDOW_SIZE ]; - - Arrival* arrivals; - - size_t bufferSize; -}; - - -LZSSE4_OptimalParseState* LZSSE4_MakeOptimalParseState( size_t bufferSize ) -{ - if ( bufferSize > 0 && ( SIZE_MAX / sizeof( Arrival ) ) < bufferSize ) - { - return nullptr; - } - - LZSSE4_OptimalParseState* result = reinterpret_cast< LZSSE4_OptimalParseState* >( ::malloc( sizeof( LZSSE4_OptimalParseState ) ) ); - - result->bufferSize = bufferSize; - - if ( result != nullptr ) - { - result->arrivals = reinterpret_cast< Arrival* >( ::malloc( sizeof( Arrival ) * bufferSize ) ); - - if ( result->arrivals == nullptr ) - { - LZSSE4_FreeOptimalParseState( result ); - - result = nullptr; - } - } - - return result; -} - - -void LZSSE4_FreeOptimalParseState( LZSSE4_OptimalParseState* toFree ) -{ - ::free( toFree->arrivals ); - - toFree->arrivals = nullptr; - - ::free( toFree ); -} - - -inline uint32_t HashOptimal( const uint8_t* inputCursor ) -{ - return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - OPTIMAL_HASH_BITS ); -} - - -struct Match -{ - size_t length; - int32_t position; - uint16_t offset; -}; - - -inline Match SearchAndUpdateFinder( LZSSE4_OptimalParseState& state, const uint8_t* input, const uint8_t* inputCursor, const uint8_t* inputEnd, uint32_t cutOff ) -{ - Match result; - - int32_t position = static_cast( inputCursor - input ); - - result.position = NO_MATCH; - result.length = MIN_MATCH_LENGTH; - result.offset = 0; - - size_t lengthToEnd = inputEnd - inputCursor; - int32_t lastPosition = position - ( LZ_WINDOW_SIZE - 1 ); - uint32_t hash = HashOptimal( inputCursor ); - - lastPosition = lastPosition > 0 ? lastPosition : 0; - - int32_t treeCursor = state.roots[ hash ]; - - state.roots[ hash ] = position; - - int32_t* left = &state.window[ position & LZ_WINDOW_MASK ].children[ 1 ]; - int32_t* right = &state.window[ position & LZ_WINDOW_MASK ].children[ 0 ]; - size_t leftLength = 0; - size_t rightLength = 0; - - for ( ;; ) - { - if ( cutOff-- == 0 || treeCursor < lastPosition ) - { - *left = *right = EMPTY_NODE; - break; - } - - TreeNode& currentNode = state.window[ treeCursor & LZ_WINDOW_MASK ]; - const uint8_t* key = input + treeCursor; - size_t matchLength = leftLength < rightLength ? leftLength : rightLength; - - uint16_t matchOffset = static_cast< uint16_t >( position - treeCursor ); - size_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; - - while ( matchLength < lengthToEnd ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); - - unsigned long matchBytes; - - _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != 16 ) - { - break; - } - } - - matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; - - size_t truncatedMatchLength = matchLength < maxLength ? matchLength : maxLength; - - if ( truncatedMatchLength >= result.length && matchOffset >= LITERALS_PER_CONTROL ) - { - result.length = truncatedMatchLength; - result.offset = matchOffset; - result.position = treeCursor; - } - - if ( matchLength == lengthToEnd ) - { - *left = currentNode.children[ 1 ]; - *right = currentNode.children[ 0 ]; - break; - } - - if ( inputCursor[ matchLength ] < key[ matchLength ] || ( matchLength == lengthToEnd ) ) - { - *left = treeCursor; - left = currentNode.children; - treeCursor = *left; - leftLength = matchLength; - } - else - { - *right = treeCursor; - right = currentNode.children + 1; - treeCursor = *right; - rightLength = matchLength; - } - } - - // Special RLE overlapping match case, the LzFind style match above doesn't work very well with our - // restriction of overlapping matches having offsets of at least 16. - // Suffix array seems like a better option to handling this. - { - // Note, we're detecting long RLE here, but if we have an offset too close, we'll sacrifice a fair - // amount of decompression performance to load-hit-stores. - int32_t matchPosition = position - ( sizeof( __m128i ) * 2 ); - - if ( matchPosition >= 0 ) - { - uint16_t matchOffset = static_cast( position - matchPosition ); - const uint8_t* key = input + matchPosition; - size_t matchLength = 0; - - while ( matchLength < lengthToEnd ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); - - unsigned long matchBytes; - - _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != 16 ) - { - break; - } - - } - - matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; - - if ( matchLength >= result.length ) - { - result.length = matchLength; - result.offset = matchOffset; - result.position = matchPosition; - } - } - } - - return result; -} - - -size_t LZSSE4_CompressOptimalParse( LZSSE4_OptimalParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength, unsigned int level ) -{ - if ( outputLength < inputLength || state->bufferSize < inputLength ) - { - // error case, output buffer not large enough. - return 0; - } - - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - if ( inputLength < MIN_COMPRESSION_SIZE ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - const uint8_t* inputEnd = input + inputLength; - Arrival* arrivalWatermark = state->arrivals; - Arrival* arrival = state->arrivals; - uint32_t cutOff = 1 << level; - - for ( int32_t* rootCursor = state->roots, *end = rootCursor + OPTIMAL_BUCKETS_COUNT; rootCursor < end; rootCursor += 4 ) - { - rootCursor[ 0 ] = EMPTY_NODE; - rootCursor[ 1 ] = EMPTY_NODE; - rootCursor[ 2 ] = EMPTY_NODE; - rootCursor[ 3 ] = EMPTY_NODE; - } - - for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) - { - SearchAndUpdateFinder( *state, input, inputCursor, inputEnd - END_PADDING_LITERALS, cutOff ); - - ++inputCursor; - } - - arrival->cost = LITERAL_BITS * LITERALS_PER_CONTROL; - arrival->from = -1; - arrival->offset = 0; - - // loop through each character and project forward the matches at that character to calculate the cheapest - // path of arrival for each individual character. - for ( const uint8_t* earlyEnd = inputEnd - END_PADDING_LITERALS; inputCursor < earlyEnd; ++inputCursor, ++arrival ) - { - uint32_t lengthToEnd = static_cast< uint32_t >( earlyEnd - inputCursor ); - int32_t currentPosition = static_cast< int32_t >( inputCursor - input ); - size_t literalsForward = LITERALS_PER_CONTROL < lengthToEnd ? LITERALS_PER_CONTROL : lengthToEnd; - size_t arrivalCost = arrival->cost; - - // NOTE - we currently assume only 2 literals filled in here, because the minimum match length is 3. - // If we wanted to go with a higher minimum match length, we would need to fill in more literals before hand. - // Also, because there is a maximum of 2 literals per control block assumed. - - // project forward the cost of a single literal - - for ( size_t where = 1; where <= literalsForward; ++where ) - { - Arrival* literalArrival = arrival + where; - size_t literalCost = arrivalCost + CONTROL_BITS + ( where * LITERAL_BITS ); - - if ( literalArrival > arrivalWatermark || literalArrival->cost > literalCost ) - { - literalArrival->cost = literalCost; - literalArrival->from = currentPosition; - literalArrival->offset = 0; - - arrivalWatermark = literalArrival > arrivalWatermark ? literalArrival : arrivalWatermark; - } - } - - Match match = SearchAndUpdateFinder( *state, input, inputCursor, earlyEnd, cutOff ); - - if ( match.position != NO_MATCH ) - { - for ( size_t matchedLength = MIN_MATCH_LENGTH, end = match.length + 1; matchedLength < end; ++matchedLength ) - { - Arrival* matchArrival = arrival + matchedLength; - size_t matchCost = arrivalCost + BASE_MATCH_BITS; - - if ( matchedLength >= EXTENDED_MATCH_BOUND ) - { - matchCost += ( matchedLength / EXTENDED_MATCH_BOUND ) * CONTROL_BITS; - } - - if ( matchArrival > arrivalWatermark || matchArrival->cost > matchCost ) - { - matchArrival->cost = matchCost; - matchArrival->from = currentPosition; - matchArrival->offset = match.offset; - - arrivalWatermark = matchArrival > arrivalWatermark ? matchArrival : arrivalWatermark; - } - } - - if ( match.length > SKIP_MATCH_LENGTH && level < NO_SKIP_LEVEL ) - { - arrival += match.length - LITERALS_PER_CONTROL; - inputCursor += match.length - LITERALS_PER_CONTROL; - } - } - } - - // If this would cost more to encode than it would if it were just literals, encode it with no control blocks, - // just literals - if ( ( arrivalWatermark->cost + END_PADDING_LITERALS * LITERAL_BITS + CONTROLS_PER_BLOCK * CONTROL_BITS ) > ( inputLength * LITERAL_BITS ) ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - Arrival* previousPathNode; - - // now trace the actual optimal parse path back, connecting the nodes in the other direction. - for ( const Arrival* pathNode = arrivalWatermark; pathNode->from > 0; pathNode = previousPathNode ) - { - previousPathNode = state->arrivals + ( pathNode->from - LITERALS_PER_CONTROL ); - - previousPathNode->to = static_cast( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); - } - - uint8_t* outputCursor = output; - - memcpy( outputCursor, input, MIN_MATCH_LENGTH ); - - outputCursor += MIN_MATCH_LENGTH; - - uint8_t* currentControlBlock = outputCursor; - uint32_t currentControlCount = 0; - uint32_t totalControlCount = 0; - - outputCursor += CONTROL_BLOCK_SIZE; - - Arrival* nextPathNode; - - size_t totalPathLength = MIN_MATCH_LENGTH; - uint16_t previousOffset = INITIAL_OFFSET; - - bool lastControlIsNop = false; - - // Now walk forwards again and actually write out the data. - for ( const Arrival* pathNode = state->arrivals; pathNode < arrivalWatermark; pathNode = nextPathNode ) - { - int32_t currentPosition = static_cast< int32_t >( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); - - nextPathNode = state->arrivals + ( pathNode->to - LITERALS_PER_CONTROL ); - - size_t pathDistance = nextPathNode - pathNode; - - totalPathLength += pathDistance; - - lastControlIsNop = false; - - if ( nextPathNode->offset == 0 ) - { - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - currentControlCount = 0; - } - - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - ( static_cast( pathDistance ) - 1 ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - ( static_cast< uint8_t >( pathDistance ) - 1 ) << CONTROL_BITS; - } - - // output the literals. - for ( int32_t where = 0; where < pathDistance; ++where ) - { - const uint8_t* currentInput = input + currentPosition + where; - - outputCursor[ where ] = *currentInput ^ *( currentInput - previousOffset ); - } - - outputCursor += pathDistance; - - ++totalControlCount; - ++currentControlCount; - } - else - { - // Check if we need to start a new control block - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - } - - // Write the offset out - note the xor with the previous offset. - *reinterpret_cast< uint16_t* >( outputCursor ) = nextPathNode->offset ^ previousOffset; - - previousOffset = nextPathNode->offset; - outputCursor += sizeof( uint16_t ); - - if ( pathDistance < EXTENDED_MATCH_BOUND ) - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( pathDistance ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( pathDistance ) << CONTROL_BITS; - } - - ++currentControlCount; - } - else - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( EXTENDED_MATCH_BOUND ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; - } - - ++currentControlCount; - - size_t toEncode = pathDistance - EXTENDED_MATCH_BOUND; - - for ( ;; ) - { - // Check if we need to start a new control block - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - } - - // If the encode size is greater than we can hold in a control, write out a full match length - // control, subtract full control value from the amount to encode and loop around again. - if ( toEncode >= EXTENDED_MATCH_BOUND ) - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( EXTENDED_MATCH_BOUND ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; - } - - toEncode -= EXTENDED_MATCH_BOUND; - - ++currentControlCount; - } - else // Write out the remaining match length control. Could potentially be zero. - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( toEncode ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( toEncode ) << CONTROL_BITS; - } - - if ( toEncode == 0 && currentControlCount == 0 ) - { - lastControlIsNop = true; - } - - ++currentControlCount; - - break; - } - } - } - } - } - - if ( lastControlIsNop ) - { - outputCursor -= CONTROL_BLOCK_SIZE; - } - - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy remaining literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - - return outputCursor - output; -} - - -size_t LZSSE4_Decompress( const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) -{ - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - // Data was not compressible, just copy initial values - if ( outputLength == inputLength ) - { - memcpy( output, input, outputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - uint8_t* outputCursor = output; - - // The offset starts off as the minimum match length. We actually need it least four - // characters back because we need them to be set to xor out the literals from the match data. - size_t offset = INITIAL_OFFSET; - __m128i previousCarryHi = _mm_setzero_si128(); - - // Copy the initial literals to the output. - for ( uint32_t where = 0; where < MIN_MATCH_LENGTH; ++where ) - { - *( outputCursor++ ) = *( inputCursor++ ); - } - - // Let me be clear, I am usually anti-macro, but they work for this particular (very unusual) case. - // DECODE_STEP is a regular decoding step, DECODE_STEP_HALF and DECODE_STEP_END are because the compiler couldn't - // seem to remove some of the dead code where values were updated and then never used. - - // What these macros do: - // Decode a single literal run or match run for a single control nibble. - // How they do it: - // - Read the *unaligned* input (in the case of LZSSE-F - twice), it goes into both a regular variable and an SSE register, - // because it could either be literals or an offset (or nothing at all). The low byte of streamBytesRead controls how much we advance - // the input cursor. - // - Used a contived set of casts to sign extend the "read offset" control mask and then use it to mask the input word, - // which is then xor'd against the offset, for a "branchless" conditional move into the offset which - // has been carried over from the previous literal/match block. Note, this ends up doing better than a cmov on most - // modern processors. But we need to pre-xor the input offset. - // - We then load the match data from output buffer (offset back from the current output point). Unconditional load here. - // - We broadcast the "from literal" control mask from the current least significant byte of the SSE register using a shuffle epi-8 - // - We mask the literals with that SSE register wide mask. - // - The literals have been pre-xor'd with the data read in as match data, so we use an xor to branchlessly choose between the two. - // In this case, it ends up a better option than a blendv on most processors. - // - Store the block. We store all 16 bytes of the SSE register (due to some constraints in the format of the data, we won't - // go past the end of the buffer), but we may overlap this. - // - bytesOut controls how much we advance the output cursor. - // - We use 8 bit shifts to advance all the controls up to the next byte. There is some variable sized register trickery that - // x86/x64 is great for as long as we don't anger the register renamer. - -#define DECODE_STEP( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - readOffsetHalf##HILO >>= 8; \ - \ - const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( literals, fromLiteral ); \ - \ - fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - bytesOutHalf##HILO >>= 8; \ - streamBytesReadHalf##HILO >>= 8; \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_HALF( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( literals, fromLiteral ); \ - \ - fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_END( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( literals, fromLiteral ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Hi, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HALF_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HALF_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Hi, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_END_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_END_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Hi, CHECKMATCH, CHECKBUFFERS ) - - __m128i nibbleMask = _mm_set1_epi8( 0xF ); - __m128i offsetSize = _mm_set1_epi8( OFFSET_SIZE ); - - // Note, we use this block here because it allows the "fake" inputEarlyEnd/outputEarlyEnd not to cause register spills - // in the decompression loops. And yes, that did actually happen. - { - -#pragma warning ( push ) -#pragma warning ( disable : 4101 ) - - // These variables are not actually ever used in this block, because we use - // a constant conditional expression to take out the branches that would hit them. - // But unfortunately, we need them to compile. - const uint8_t* inputEarlyEnd; - uint8_t* outputEarlyEnd; - -#pragma warning ( pop ) - - // "Safe" ends to the buffer, before the input/output cursors hit these, we can loop without overflow checks. - const uint8_t* inputSafeEnd = ( input + inputLength ) - INPUT_BUFFER_SAFE; - uint8_t* outputSafeEnd = ( output + outputLength ) - OUTPUT_BUFFER_SAFE; - - // Decoding loop with offset output buffer underflow test, but no buffer overflow tests, assumed to end at a safe distance - // from overflows - while ( ( outputCursor - output ) < LZ_WINDOW_SIZE && outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) - { - // load the control block - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - - // split the control block into high and low nibbles - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will - // be an extension of the current match operation. - - // Work out the carry for the low nibbles (which will be used with the high controls to put them into - // match without offset read mode). - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - - // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save - // the calculated carry to use that byte next iteration. - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); - - previousCarryHi = carryHi; - - // We make the implicit assumption that the maximum number of literals to controls here is twice the offset size (4 vs 2), - // we are doing this here to save keeping the value around (spilling or fetching it each time) - __m128i literalsPerControl = _mm_add_epi8( offsetSize, offsetSize ); - - // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation - // this will essentially be ignored later on. - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - // Here we're calculating the number of bytes that will be output, we are actually subtracting negative one from the control - // (handy trick where comparison result masks are negative one) if carry is not set and it is a literal. - __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_andnot_si128( shiftedCarryHi, isLiteralLo ) ); - __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_andnot_si128( carryLo, isLiteralHi ) ); - - // Calculate the number of bytes to read per control. - // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_blendv_epi8( offsetSize, bytesOutLo, isLiteralLo ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_blendv_epi8( offsetSize, bytesOutHi, isLiteralHi ) ); - - // I want 128 set bits please. - __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); - - // Masks whether we are reading literals - set if the carry is not set and these are literals. - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - // Advance the input past the control block - inputCursor += CONTROL_BLOCK_SIZE; - - { - // Pull out the bottom halves off the SSE registers from before - we want these - // things in GPRs for the more linear logic. - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_HALF_LO( true, false ); - DECODE_STEP_HALF_HI( true, false ); - } - - { - // Now the top halves. - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_END_LO( true, false ); - DECODE_STEP_END_HI( true, false ); - } - } - - // Decoding loop with no buffer checks, but will end at a safe distance from the end of the buffers. - // Note, when we get here we have already reached the point in the output buffer which is *past* where we can underflow - // due to a bad match offset. - while ( outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) - { - // This code is the same as the loop above, see comments there - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); // where we take the carry from the previous hi values - - previousCarryHi = carryHi; - - __m128i literalsPerControl = _mm_add_epi8( offsetSize, offsetSize ); - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_andnot_si128( shiftedCarryHi, isLiteralLo ) ); - __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_andnot_si128( carryLo, isLiteralHi ) ); - - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_blendv_epi8( offsetSize, bytesOutLo, isLiteralLo ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_blendv_epi8( offsetSize, bytesOutHi, isLiteralHi ) ); - - __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); - - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - inputCursor += CONTROL_BLOCK_SIZE; - - { - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_HALF_LO( false, false ); - DECODE_STEP_HALF_HI( false, false ); - } - - { - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_END_LO( false, false ); - DECODE_STEP_END_HI( false, false ); - } - } - } - - // Decoding loop with all buffer checks. - { - const uint8_t* inputEarlyEnd; - uint8_t* outputEarlyEnd; - inputEarlyEnd = ( input + inputLength ) - END_PADDING_LITERALS; - outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; - - while ( outputCursor < outputEarlyEnd && inputCursor < inputEarlyEnd ) - { - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); - - previousCarryHi = carryHi; - - __m128i literalsPerControl = _mm_add_epi8( offsetSize, offsetSize ); - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - __m128i bytesOutLo = _mm_sub_epi8( controlLo, _mm_andnot_si128( shiftedCarryHi, isLiteralLo ) ); - __m128i bytesOutHi = _mm_sub_epi8( controlHi, _mm_andnot_si128( carryLo, isLiteralHi ) ); - - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_blendv_epi8( offsetSize, bytesOutLo, isLiteralLo ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_blendv_epi8( offsetSize, bytesOutHi, isLiteralHi ) ); - - __m128i neg1 = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), neg1 ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), neg1 ); - - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - inputCursor += CONTROL_BLOCK_SIZE; - - if ( inputCursor > inputEarlyEnd ) - goto BUFFER_END; - - { - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_HALF_LO( true, true ); - DECODE_STEP_HALF_HI( true, true ); - } - - { - // Now the top halves. - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_END_LO( true, true ); - DECODE_STEP_END_HI( true, true ); - } - } - -BUFFER_END: - - // When we get here, we have either advanced the right amount on both cursors - // or something bad happened, so leave it as is, so we can tell where - // the error happened. - if ( inputCursor == inputEarlyEnd && outputCursor == outputEarlyEnd ) - { - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy any trailing literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - } - } - -MATCH_UNDERFLOW: - - return outputCursor - output; -} diff --git a/contrib/lzsse/lzsse4/lzsse4.h b/contrib/lzsse/lzsse4/lzsse4.h deleted file mode 100644 index 0fa5e6c8ed0..00000000000 --- a/contrib/lzsse/lzsse4/lzsse4.h +++ /dev/null @@ -1,117 +0,0 @@ -/* -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef LZSSE4_H__ -#define LZSSE4_H__ - -#pragma once - -/* LZSSE4 - x64/SSE targeted codec for better performance with lower compression ratio data/less optimal compressors. - * Supports minimum 4 byte matches, maximum 15 bytes of match per control word and 4 byte literal runs per control word. - */ - -#ifdef __cplusplus -extern "C" -{ -#endif - -/* Re-usable parse state object for compression. */ -typedef struct LZSSE4_FastParseState LZSSE4_FastParseState; - -/* Allocate the parse state for compression - returns null on failure */ -LZSSE4_FastParseState* LZSSE4_MakeFastParseState(); - -/* De-allocate the parse state for compression */ -void LZSSE4_FreeFastParseState( LZSSE4_FastParseState* toFree ); - -/* Re-usable parse state object for compression. */ -typedef struct LZSSE4_OptimalParseState LZSSE4_OptimalParseState; - -/* Allocate the parse state for compression - returns null on failure. Note - Buffersize has to be greater or equal to any inputLength used with LZSSE2_CompressOptimalParse */ -LZSSE4_OptimalParseState* LZSSE4_MakeOptimalParseState( size_t bufferSize ); - -/* De-allocate the parse state for compression */ -void LZSSE4_FreeOptimalParseState( LZSSE4_OptimalParseState* toFree ); - - -/* "Optimal" compression routine. -* Will compress data into LZSSE4 format, uses hash BST matching to find matches and run an optimal parse (high relative memory usage). Requires SSE 4.1. -* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. -* input : Buffer containing uncompressed data to be compressed. May not be null. -* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. -* output : Buffer that will receive the compressed output. -* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, -* The compressed data should never be longer than inputLength, as in this case the data is stored raw. -* level : The compression level to use for this file 1->17, 17 delivers the highest compression, 1 delivers the least. -* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state -* concurrently. -* -* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). -*/ -size_t LZSSE4_CompressOptimalParse( LZSSE4_OptimalParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int level ); - - -/* "Fast" compression routine. - * Will compress data into LZSSE4 format, uses a simple single entry hash/greedy matching to find matches. Requires SSE 4.1. - * state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. - * input : Buffer containing uncompressed data to be compressed. May not be null. - * inputLength : Length of the compressed data in the input buffer - note should be under 2GB. - * output : Buffer that will receive the compressed output. - * outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, - * The compressed data should never be longer than inputLength, as in this case the data is stored raw. - * - * Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state - * concurrently. - * - * Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). - */ -size_t LZSSE4_CompressFast( LZSSE4_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength ); - -/* Decompression routine. - * This routine will decompress data in the LZSSE4 format and currently requires SSE 4.1 and is targeted at x64. - * It will perform poorly on x86 due to hunger for registers. - * input : Buffer containing compressed input block. May not be null. - * inputLength : Length of the compressed data in the input buffer - note, this should be under 2GB - * output : Buffer that will received the de-compressed output. Note, that this needs to be at least outputLength long. - * May not be null. - * outputLength : The length of the compressed output - note, this should be under 2GB - * - * Provided that input and output are valid pointers to buffers of at least their specified size, this routine - * should be memory safe - both match pointer checks and input/output buffer checks exist. - * - * Returns the size of the decompressed data, which will be less than outputLength in the event of an error (number of bytes - * will indicate where in the output stream the error occured). - * - * Note that this data is not hash verified, errors that occur are either from a misformed stream or bad buffer sizes. - * Remember, corrupt data can still be valid to decompress. - */ -size_t LZSSE4_Decompress( const void* input, size_t inputLength, void* output, size_t outputLength ); - -#ifdef __cplusplus -} -#endif - -#endif /* -- LZSSE4_H__ */ diff --git a/contrib/lzsse/lzsse4/lzsse4_platform.h b/contrib/lzsse/lzsse4/lzsse4_platform.h deleted file mode 100644 index d1d3f1cced2..00000000000 --- a/contrib/lzsse/lzsse4/lzsse4_platform.h +++ /dev/null @@ -1,73 +0,0 @@ -/* -Copyright (c) 2016, Brian Marshall -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef LZSSE4_PLATFORM_H__ -#define LZSSE4_PLATFORM_H__ - -#pragma once - -/* - Compiler/Platform detection based on the table from: - https://blogs.msdn.microsoft.com/vcblog/2015/12/04/clang-with-microsoft-codegen-in-vs-2015-update-1/ -*/ - -#ifdef _MSC_VER - -/* - Microsoft Visual Studio Support. - C1xx/C2, Clang/C2 and Clang/LLVM all support the Microsoft header files and _BitScanForward - - Note: if you receive errors with the intrinsics make sure that you have SSE4.1 support enabled. - For example with Clang include "-msse4.1" on the command line -*/ -#include - -#else /* _MSC_VER */ - -#ifdef __GNUC__ - -/* - GCC -*/ - -/* - Note: including just would be sufficient, but including x86intrin is a better match to intrin.h on Visual Studio as - both include all intrinsics for the enabled processor, rather than just SSE4.1. -*/ -#include -/* _BitScanForward is Visual Studio specific. */ -#define _BitScanForward(x, m) *(x) = __builtin_ctz(m) - -#else - -/* -If you hit the error below, then add detection for your compiler/platform to this header file. -*/ -#error Platform not supported - -#endif /* __GNUC__ */ -#endif /* _MSC_VER */ - -#endif /* -- LZSSE4_PLATFORM_H__ */ diff --git a/contrib/lzsse/lzsse8/lzsse8.cpp b/contrib/lzsse/lzsse8/lzsse8.cpp deleted file mode 100644 index a5682a1de81..00000000000 --- a/contrib/lzsse/lzsse8/lzsse8.cpp +++ /dev/null @@ -1,1568 +0,0 @@ -/* -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#include -#include -#include -#include - -#include "lzsse8_platform.h" -#include "lzsse8.h" - -#pragma warning ( disable : 4127 ) - -namespace -{ - // Constants - most of these should not be changed without corresponding code changes because it will break many things in unpredictable ways. - const uint32_t WINDOW_BITS = 16; - const uint32_t MIN_MATCH_LENGTH = 4; - const uint32_t LZ_WINDOW_SIZE = 1 << WINDOW_BITS; - const uint32_t LZ_WINDOW_MASK = LZ_WINDOW_SIZE - 1; - const uint32_t FAST_HASH_BITS = 20; // You can change this - more bits = more matches, less bits = more cache hits - const uint32_t FAST_BUCKETS_COUNT = 1 << FAST_HASH_BITS; - const uint32_t FAST_HASH_MASK = FAST_BUCKETS_COUNT - 1; - const uint32_t MIN_COMPRESSION_SIZE = 32; - const uint32_t END_PADDING_LITERALS = 16; - const int32_t NO_MATCH = -1; - const int32_t EMPTY_NODE = -1; - const uint32_t CONTROL_BITS = 4; - const uint32_t OFFSET_SIZE = 2; - const uint32_t EXTENDED_MATCH_BOUND = ( 1 << CONTROL_BITS ) - 1; - const uint32_t CONTROL_BLOCK_SIZE = sizeof( __m128i ); - const uint32_t CONTROLS_PER_BLOCK = 32; - const uint32_t LITERALS_PER_CONTROL = 8; - const uint32_t MAX_INPUT_PER_CONTROL = 8; - const uint32_t INITIAL_MATCH_BOUND = 11; - const size_t OUTPUT_BUFFER_SAFE = EXTENDED_MATCH_BOUND * CONTROLS_PER_BLOCK; - const size_t INPUT_BUFFER_SAFE = MAX_INPUT_PER_CONTROL * CONTROLS_PER_BLOCK; - const uint16_t INITIAL_OFFSET = LITERALS_PER_CONTROL; - const uint32_t OFFSET_BITS = 16; - const uint32_t BASE_MATCH_BITS = OFFSET_BITS + CONTROL_BITS; - const uint32_t OPTIMAL_HASH_BITS = 20; - const uint32_t OPTIMAL_BUCKETS_COUNT = 1 << OPTIMAL_HASH_BITS; - const uint32_t OPTIMAL_HASH_MASK = OPTIMAL_BUCKETS_COUNT - 1; - const uint32_t LITERAL_BITS = 8; - const size_t SKIP_MATCH_LENGTH = 128; - const uint32_t NO_SKIP_LEVEL = 17; - -} - - -struct LZSSE8_FastParseState -{ - int32_t buckets[ FAST_BUCKETS_COUNT ]; // stores the first matching position, we can then look at the rest of the matches by tracing through the window. -}; - - -LZSSE8_FastParseState* LZSSE8_MakeFastParseState() -{ - return new LZSSE8_FastParseState(); -} - - -void LZSSE8_FreeFastParseState( LZSSE8_FastParseState* toFree ) -{ - delete toFree; -} - - -inline void SetHash( LZSSE8_FastParseState* state, uint32_t hash, const uint8_t* input, const uint8_t* inputCursor ) -{ - int32_t position = static_cast( inputCursor - input ); - - state->buckets[ hash & FAST_HASH_MASK ] = position; -} - -// Simple fast hash function - actually what is used in snappy and derivatives -// There's probably better, but I haven't spent time focusing on this area yet. -inline uint32_t HashFast( const uint8_t* inputCursor ) -{ - return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - FAST_HASH_BITS ); -} - -struct Arrival -{ - size_t cost; - int32_t from; - int32_t to; - uint16_t offset; -}; - -struct TreeNode -{ - int32_t children[ 2 ]; -}; - -struct LZSSE8_OptimalParseState -{ - int32_t roots[ OPTIMAL_BUCKETS_COUNT ]; - - TreeNode window[ LZ_WINDOW_SIZE ]; - - Arrival* arrivals; - - size_t bufferSize; -}; - - -LZSSE8_OptimalParseState* LZSSE8_MakeOptimalParseState( size_t bufferSize ) -{ - if ( bufferSize > 0 && ( SIZE_MAX / sizeof( Arrival ) ) < bufferSize ) - { - return nullptr; - } - - LZSSE8_OptimalParseState* result = reinterpret_cast< LZSSE8_OptimalParseState* >( ::malloc( sizeof( LZSSE8_OptimalParseState ) ) ); - - result->bufferSize = bufferSize; - - if ( result != nullptr ) - { - result->arrivals = reinterpret_cast< Arrival* >( ::malloc( sizeof( Arrival ) * bufferSize ) ); - - if ( result->arrivals == nullptr ) - { - LZSSE8_FreeOptimalParseState( result ); - - result = nullptr; - } - } - - return result; -} - - -void LZSSE8_FreeOptimalParseState( LZSSE8_OptimalParseState* toFree ) -{ - ::free( toFree->arrivals ); - - toFree->arrivals = nullptr; - - ::free( toFree ); -} - - -inline uint32_t HashOptimal( const uint8_t* inputCursor ) -{ - return *reinterpret_cast( inputCursor ) * 0x1e35a7bd >> ( 32 - OPTIMAL_HASH_BITS ); -} - - -struct Match -{ - size_t length; - int32_t position; - uint16_t offset; -}; - - -inline Match SearchAndUpdateFinder( LZSSE8_OptimalParseState& state, const uint8_t* input, const uint8_t* inputCursor, const uint8_t* inputEnd, uint32_t cutOff ) -{ - Match result; - - int32_t position = static_cast( inputCursor - input ); - - result.position = NO_MATCH; - result.length = MIN_MATCH_LENGTH; - result.offset = 0; - - size_t lengthToEnd = inputEnd - inputCursor; - int32_t lastPosition = position - ( LZ_WINDOW_SIZE - 1 ); - uint32_t hash = HashOptimal( inputCursor ); - - lastPosition = lastPosition > 0 ? lastPosition : 0; - - int32_t treeCursor = state.roots[ hash ]; - - state.roots[ hash ] = position; - - int32_t* left = &state.window[ position & LZ_WINDOW_MASK ].children[ 1 ]; - int32_t* right = &state.window[ position & LZ_WINDOW_MASK ].children[ 0 ]; - size_t leftLength = 0; - size_t rightLength = 0; - - for ( ;; ) - { - if ( cutOff-- == 0 || treeCursor < lastPosition ) - { - *left = *right = EMPTY_NODE; - break; - } - - TreeNode& currentNode = state.window[ treeCursor & LZ_WINDOW_MASK ]; - const uint8_t* key = input + treeCursor; - size_t matchLength = leftLength < rightLength ? leftLength : rightLength; - - uint16_t matchOffset = static_cast( position - treeCursor ); - size_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; - - while ( matchLength < lengthToEnd ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); - - unsigned long matchBytes; - - _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != 16 ) - { - break; - } - } - - matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; - - size_t truncatedMatchLength = matchLength < maxLength ? matchLength : maxLength; - - if ( truncatedMatchLength >= result.length && matchOffset >= LITERALS_PER_CONTROL ) - { - result.length = truncatedMatchLength; - result.offset = matchOffset; - result.position = treeCursor; - } - - if ( matchLength == lengthToEnd ) - { - *left = currentNode.children[ 1 ]; - *right = currentNode.children[ 0 ]; - break; - } - - if ( inputCursor[ matchLength ] < key[ matchLength ] || ( matchLength == lengthToEnd ) ) - { - *left = treeCursor; - left = currentNode.children; - treeCursor = *left; - leftLength = matchLength; - } - else - { - *right = treeCursor; - right = currentNode.children + 1; - treeCursor = *right; - rightLength = matchLength; - } - } - - // Special RLE overlapping match case, the LzFind style match above doesn't work very well with our - // restriction of overlapping matches having offsets of at least 16. - // Suffix array seems like a better option to handling this. - { - // Note, we're detecting long RLE here, but if we have an offset too close, we'll sacrifice a fair - // amount of decompression performance to load-hit-stores. - int32_t matchPosition = position - ( sizeof( __m128i ) * 2 ); - - if ( matchPosition >= 0 ) - { - uint16_t matchOffset = static_cast( position - matchPosition ); - const uint8_t* key = input + matchPosition; - size_t matchLength = 0; - - while ( matchLength < lengthToEnd ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( key + matchLength ) ); - - unsigned long matchBytes; - - _BitScanForward( &matchBytes, ( static_cast( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != 16 ) - { - break; - } - - } - - matchLength = matchLength < lengthToEnd ? matchLength : lengthToEnd; - - if ( matchLength >= result.length ) - { - result.length = matchLength; - result.offset = matchOffset; - result.position = matchPosition; - } - } - } - - return result; -} - - -size_t LZSSE8_CompressOptimalParse( LZSSE8_OptimalParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength, unsigned int level ) -{ - if ( outputLength < inputLength || state->bufferSize < inputLength ) - { - // error case, output buffer not large enough. - return 0; - } - - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - if ( inputLength < MIN_COMPRESSION_SIZE ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - const uint8_t* inputEnd = input + inputLength; - Arrival* arrivalWatermark = state->arrivals; - Arrival* arrival = state->arrivals; - uint32_t cutOff = 1 << level; - - for ( int32_t* rootCursor = state->roots, *end = rootCursor + OPTIMAL_BUCKETS_COUNT; rootCursor < end; rootCursor += 4 ) - { - rootCursor[ 0 ] = EMPTY_NODE; - rootCursor[ 1 ] = EMPTY_NODE; - rootCursor[ 2 ] = EMPTY_NODE; - rootCursor[ 3 ] = EMPTY_NODE; - } - - for ( uint32_t where = 0; where < LITERALS_PER_CONTROL; ++where ) - { - SearchAndUpdateFinder( *state, input, inputCursor, inputEnd - END_PADDING_LITERALS, cutOff ); - - ++inputCursor; - } - - arrival->cost = LITERAL_BITS * LITERALS_PER_CONTROL; - arrival->from = -1; - arrival->offset = 0; - - // loop through each character and project forward the matches at that character to calculate the cheapest - // path of arrival for each individual character. - for ( const uint8_t* earlyEnd = inputEnd - END_PADDING_LITERALS; inputCursor < earlyEnd; ++inputCursor, ++arrival ) - { - uint32_t lengthToEnd = static_cast< uint32_t >( earlyEnd - inputCursor ); - int32_t currentPosition = static_cast< int32_t >( inputCursor - input ); - size_t literalsForward = LITERALS_PER_CONTROL < lengthToEnd ? LITERALS_PER_CONTROL : lengthToEnd; - size_t arrivalCost = arrival->cost; - - // NOTE - we currently assume only 2 literals filled in here, because the minimum match length is 3. - // If we wanted to go with a higher minimum match length, we would need to fill in more literals before hand. - // Also, because there is a maximum of 2 literals per control block assumed. - - // project forward the cost of a single literal - - for ( size_t where = 1; where <= literalsForward; ++where ) - { - Arrival* literalArrival = arrival + where; - size_t literalCost = arrivalCost + CONTROL_BITS + ( where * LITERAL_BITS ); - - if ( literalArrival > arrivalWatermark || literalArrival->cost > literalCost ) - { - literalArrival->cost = literalCost; - literalArrival->from = currentPosition; - literalArrival->offset = 0; - - arrivalWatermark = literalArrival > arrivalWatermark ? literalArrival : arrivalWatermark; - } - } - - Match match = SearchAndUpdateFinder( *state, input, inputCursor, earlyEnd, cutOff ); - - if ( match.position != NO_MATCH ) - { - for ( size_t matchedLength = MIN_MATCH_LENGTH, end = match.length + 1; matchedLength < end; ++matchedLength ) - { - Arrival* matchArrival = arrival + matchedLength; - size_t matchCost = arrivalCost + BASE_MATCH_BITS; - - if ( matchedLength >= INITIAL_MATCH_BOUND ) - { - matchCost += ( ( ( matchedLength - INITIAL_MATCH_BOUND ) / EXTENDED_MATCH_BOUND ) + 1 ) * CONTROL_BITS; - } - - if ( matchArrival > arrivalWatermark || matchArrival->cost > matchCost ) - { - matchArrival->cost = matchCost; - matchArrival->from = currentPosition; - matchArrival->offset = match.offset; - - arrivalWatermark = matchArrival > arrivalWatermark ? matchArrival : arrivalWatermark; - } - } - - if ( match.length > SKIP_MATCH_LENGTH && level < NO_SKIP_LEVEL ) - { - arrival += match.length - LITERALS_PER_CONTROL; - inputCursor += match.length - LITERALS_PER_CONTROL; - } - } - } - - // If this would cost more to encode than it would if it were just literals, encode it with no control blocks, - // just literals - if ( ( arrivalWatermark->cost + END_PADDING_LITERALS * LITERAL_BITS + CONTROLS_PER_BLOCK * CONTROL_BITS ) > ( inputLength * LITERAL_BITS ) ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - Arrival* previousPathNode; - - // now trace the actual optimal parse path back, connecting the nodes in the other direction. - for ( const Arrival* pathNode = arrivalWatermark; pathNode->from > 0; pathNode = previousPathNode ) - { - previousPathNode = state->arrivals + ( pathNode->from - LITERALS_PER_CONTROL ); - - previousPathNode->to = static_cast( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); - } - - uint8_t* outputCursor = output; - - memcpy( outputCursor, input, LITERALS_PER_CONTROL ); - - outputCursor += LITERALS_PER_CONTROL; - - uint8_t* currentControlBlock = outputCursor; - uint32_t currentControlCount = 0; - uint32_t totalControlCount = 0; - - outputCursor += CONTROL_BLOCK_SIZE; - - Arrival* nextPathNode; - - size_t totalPathLength = LITERALS_PER_CONTROL; - uint16_t previousOffset = INITIAL_OFFSET; - - bool lastControlIsNop = false; - - // Now walk forwards again and actually write out the data. - for ( const Arrival* pathNode = state->arrivals; pathNode < arrivalWatermark; pathNode = nextPathNode ) - { - int32_t currentPosition = static_cast< int32_t >( ( pathNode - state->arrivals ) + LITERALS_PER_CONTROL ); - - nextPathNode = state->arrivals + ( pathNode->to - LITERALS_PER_CONTROL ); - - size_t pathDistance = nextPathNode - pathNode; - - totalPathLength += pathDistance; - - lastControlIsNop = false; - - if ( nextPathNode->offset == 0 ) - { - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - currentControlCount = 0; - } - - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - ( static_cast( pathDistance ) - 1 ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - ( static_cast< uint8_t >( pathDistance ) - 1 ) << CONTROL_BITS; - } - - // output the literals. - for ( int32_t where = 0; where < pathDistance; ++where ) - { - const uint8_t* currentInput = input + currentPosition + where; - - outputCursor[ where ] = *currentInput ^ *( currentInput - previousOffset ); - } - - outputCursor += pathDistance; - - ++totalControlCount; - ++currentControlCount; - } - else - { - // Check if we need to start a new control block - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - } - - // Write the offset out - note the xor with the previous offset. - *reinterpret_cast< uint16_t* >( outputCursor ) = nextPathNode->offset ^ previousOffset; - - previousOffset = nextPathNode->offset; - outputCursor += sizeof( uint16_t ); - - if ( pathDistance < INITIAL_MATCH_BOUND ) - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( pathDistance + MIN_MATCH_LENGTH ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( pathDistance + MIN_MATCH_LENGTH ) << CONTROL_BITS; - } - - ++currentControlCount; - } - else - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( EXTENDED_MATCH_BOUND ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; - } - - ++currentControlCount; - - size_t toEncode = pathDistance - INITIAL_MATCH_BOUND; - - for ( ;; ) - { - // Check if we need to start a new control block - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - } - - // If the encode size is greater than we can hold in a control, write out a full match length - // control, subtract full control value from the amount to encode and loop around again. - if ( toEncode >= EXTENDED_MATCH_BOUND ) - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( EXTENDED_MATCH_BOUND ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( EXTENDED_MATCH_BOUND ) << CONTROL_BITS; - } - - toEncode -= EXTENDED_MATCH_BOUND; - - ++currentControlCount; - } - else // Write out the remaining match length control. Could potentially be zero. - { - if ( ( currentControlCount & 1 ) == 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] = - static_cast( toEncode ); - } - else - { - currentControlBlock[ currentControlCount >> 1 ] |= - static_cast< uint8_t >( toEncode ) << CONTROL_BITS; - } - - if ( toEncode == 0 && currentControlCount == 0 ) - { - lastControlIsNop = true; - } - - ++currentControlCount; - - break; - } - } - } - } - } - - if ( lastControlIsNop ) - { - outputCursor -= CONTROL_BLOCK_SIZE; - } - - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy remaining literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - - return outputCursor - output; -} - - -size_t LZSSE8_CompressFast( LZSSE8_FastParseState* state, const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) -{ - if ( outputLength < inputLength ) - { - // error case, output buffer not large enough. - return 0; - } - - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - if ( inputLength < MIN_COMPRESSION_SIZE ) - { - memcpy( output, input, inputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - const uint8_t* inputEnd = input + inputLength; - const uint8_t* inputEarlyEnd = inputEnd - END_PADDING_LITERALS; - uint8_t* outputCursor = output; - uint8_t* outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; - uint32_t hash = 0; - - // initialize hash to empty - for ( int32_t* where = state->buckets, *end = state->buckets + FAST_BUCKETS_COUNT; where < end; where += 4 ) - { - where[ 0 ] = -1; - where[ 1 ] = -1; - where[ 2 ] = -1; - where[ 3 ] = -1; - } - - // initial literals that wont be compressed - for ( uint32_t where = 0; where < LITERALS_PER_CONTROL; ++where ) - { - hash = HashFast( inputCursor ); - - SetHash( state, hash, input, inputCursor ); - - *( outputCursor++ ) = *( inputCursor++ ); - } - - uint8_t* currentControlBlock = outputCursor; - uint32_t currentControlCount = 0; - uint16_t previousOffset = INITIAL_OFFSET; - size_t literalsToFlush = 0; - - outputCursor += CONTROL_BLOCK_SIZE; - - bool lastControlIsNop = false; - - // Loop through the data until we hit the end of one of the buffers (minus the end padding literals) - while ( inputCursor < inputEarlyEnd && outputCursor <= outputEarlyEnd ) - { - lastControlIsNop = false; - - hash = HashFast( inputCursor ); - - int matchPosition = state->buckets[ hash & FAST_HASH_MASK ]; - int currentPosition = static_cast< int32_t >( inputCursor - input ); - uint32_t matchLength = 0; - uint16_t matchOffset = static_cast< uint16_t >( currentPosition - matchPosition ); - - // If we had a hit in the hash and it wasn't outside the window. - if ( matchPosition >= 0 && ( currentPosition - matchPosition ) < ( LZ_WINDOW_SIZE - 1 ) && matchOffset >= LITERALS_PER_CONTROL ) - { - const uint8_t* matchCandidate = input + matchPosition; - uint32_t lengthToEnd = static_cast< uint32_t >( inputEarlyEnd - inputCursor ); - // Here we limit the hash length to prevent overlap matches with offset less than 16 bytes - uint32_t maxLength = matchOffset <= ( EXTENDED_MATCH_BOUND + 1 ) && matchOffset < lengthToEnd ? matchOffset : lengthToEnd; - - // Find how long the match is 16 bytes at a time. - while ( matchLength < maxLength ) - { - __m128i input16 = _mm_loadu_si128( reinterpret_cast( inputCursor + matchLength ) ); - __m128i match16 = _mm_loadu_si128( reinterpret_cast( matchCandidate + matchLength ) ); - - unsigned long matchBytes; - - // Finds the number of equal bytes at the start of the 16 - _BitScanForward( &matchBytes, ( static_cast< unsigned long >( ~_mm_movemask_epi8( _mm_cmpeq_epi8( input16, match16 ) ) ) | 0x10000 ) ); - - matchLength += matchBytes; - - if ( matchBytes != sizeof( __m128i ) ) - { - break; - } - } - - matchLength = matchLength < maxLength ? matchLength : maxLength; - } - - // If we have at least the minimum match length (4 bytes) - if ( matchLength >= MIN_MATCH_LENGTH ) - { - // Do we have literals to flush before the match? - if ( literalsToFlush > 0 ) - { - // Start a new control block if we need one. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - // Would be larger than compressed size, get out! - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); - - // flush the literals. - // note the xor against the data that would be read in the match. - for ( uint32_t where = 0; where < literalsToFlush; ++where ) - { - const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); - - *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); - } - - ++currentControlCount; - - literalsToFlush = 0; - - // Would be larger than compressed size, get out! - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - // Start a new control block if the previous one is full. - // Note this is done before the offset is written out - the offset - // is always written after the control block containing the first - // control in the match. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - // The match length value we are encoding. - - // Write the offset out - note the xor with the previous offset. - *reinterpret_cast< uint16_t* >( outputCursor ) = matchOffset ^ previousOffset; - - previousOffset = matchOffset; - outputCursor += sizeof( uint16_t ); - - if ( matchLength < INITIAL_MATCH_BOUND ) - { - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( matchLength + MIN_MATCH_LENGTH ) << 4 ); - - ++currentControlCount; - } - else - { - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( EXTENDED_MATCH_BOUND ) << 4 ); - - ++currentControlCount; - - size_t toEncode = matchLength - INITIAL_MATCH_BOUND; - - for ( ;; ) - { - // Check if we need to start a new control block - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - // If the encode size is greater than we can hold in a control, write out a full match length - // control, subtract full control value from the amount to encode and loop around again. - if ( toEncode >= EXTENDED_MATCH_BOUND ) - { - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( EXTENDED_MATCH_BOUND ) << 4 ); - - toEncode -= EXTENDED_MATCH_BOUND; - - ++currentControlCount; - } - else // Write out the remaining match length control. Could potentially be zero. - { - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( toEncode ) << 4 ); - - if ( currentControlBlock == 0 && toEncode == 0 ) - { - lastControlIsNop = true; - } - - ++currentControlCount; - - break; - } - } - } - - // Update the value into the hash for future matches. - SetHash( state, hash, input, inputCursor ); - - ++inputCursor; - - // Hash all the other values in the match too. - for ( const uint8_t* nextArrival = inputCursor + matchLength - 1; inputCursor < nextArrival; ++inputCursor ) - { - hash = HashFast( inputCursor ); - SetHash( state, hash, input, inputCursor ); - } - } - else - { - // One more literal to write out. - ++literalsToFlush; - - // If we have reached the maximum number of literals allowed in the control, flush them out. - if ( literalsToFlush == LITERALS_PER_CONTROL ) - { - // Check if the control block is full and we need start a new one. - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - } - - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( ( static_cast( LITERALS_PER_CONTROL - 1 ) ) << 4 ); - - ++currentControlCount; - - *reinterpret_cast< uint64_t* >( outputCursor ) = - *reinterpret_cast< const uint64_t* >( inputCursor - ( sizeof( uint64_t ) - 1 ) ) ^ - *reinterpret_cast< const uint64_t* >( ( inputCursor - ( sizeof( uint64_t ) - 1 ) ) - previousOffset ); - - outputCursor += sizeof( uint64_t ); - - //*( outputCursor++ ) = *( inputCursor - 3 ) ^ *( ( inputCursor - 3 ) - previousOffset ); - //*( outputCursor++ ) = *( inputCursor - 2 ) ^ *( ( inputCursor - 2 ) - previousOffset ); - //*( outputCursor++ ) = *( inputCursor - 1 ) ^ *( ( inputCursor - 1 ) - previousOffset ); - //*( outputCursor++ ) = *inputCursor ^ *( inputCursor - previousOffset ); - - if ( outputCursor > outputEarlyEnd ) - { - break; - } - - literalsToFlush = 0; - } - - // Update the hash with this byte - SetHash( state, hash, input, inputCursor ); - - ++inputCursor; - } - } - - // If we would create a compression output bigger than or equal to the input, just copy the input to the output and return equal size. - if ( ( ( outputCursor + literalsToFlush + ( currentControlCount == CONTROLS_PER_BLOCK ? CONTROL_BLOCK_SIZE : 0 ) ) ) >= output + inputLength - END_PADDING_LITERALS ) - { - memcpy( output, input, inputLength ); - - outputCursor = output + inputLength; - } - else - { - // Flush any remaining literals. - if ( literalsToFlush > 0 ) - { - lastControlIsNop = false; - - if ( currentControlCount == CONTROLS_PER_BLOCK ) - { - currentControlBlock = outputCursor; - outputCursor += CONTROL_BLOCK_SIZE; - - _mm_storeu_si128( reinterpret_cast< __m128i* >( outputCursor ), _mm_setzero_si128() ); - - currentControlCount = 0; - } - - currentControlBlock[ currentControlCount >> 1 ] = - ( currentControlBlock[ currentControlCount >> 1 ] >> 4 ) | ( static_cast( literalsToFlush - 1 ) << 4 ); - - for ( uint32_t where = 0; where < literalsToFlush; ++where ) - { - const uint8_t* currentInput = inputCursor - ( literalsToFlush - where ); - - *( outputCursor++ ) = *currentInput ^ *( currentInput - previousOffset ); - } - - ++currentControlCount; - } - - // Need to finish off shifting the final control block into the low nibble if there is no second nibble - if ( ( currentControlCount & 1 ) > 0 ) - { - currentControlBlock[ currentControlCount >> 1 ] >>= 4; - } - - if ( lastControlIsNop ) - { - outputCursor -= CONTROL_BLOCK_SIZE; - } - - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy remaining literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - } - - // Return the size of the compressed data. - return outputCursor - output; -} - - -size_t LZSSE8_Decompress( const void* inputChar, size_t inputLength, void* outputChar, size_t outputLength ) -{ - const uint8_t* input = reinterpret_cast< const uint8_t* >( inputChar ); - uint8_t* output = reinterpret_cast< uint8_t* >( outputChar ); - - // Data was not compressible, just copy initial values - if ( outputLength == inputLength ) - { - memcpy( output, input, outputLength ); - - return inputLength; - } - - const uint8_t* inputCursor = input; - uint8_t* outputCursor = output; - - // The offset starts off as the minimum match length. We actually need it least four - // characters back because we need them to be set to xor out the literals from the match data. - size_t offset = INITIAL_OFFSET; - __m128i previousCarryHi = _mm_setzero_si128(); - - // Copy the initial literals to the output. - for ( uint32_t where = 0; where < LITERALS_PER_CONTROL; ++where ) - { - *( outputCursor++ ) = *( inputCursor++ ); - } - - // Let me be clear, I am usually anti-macro, but they work for this particular (very unusual) case. - // DECODE_STEP is a regular decoding step, DECODE_STEP_HALF and DECODE_STEP_END are because the compiler couldn't - // seem to remove some of the dead code where values were updated and then never used. - - // What these macros do: - // Decode a single literal run or match run for a single control nibble. - // How they do it: - // - Read the *unaligned* input (in the case of LZSSE-F - twice), it goes into both a regular variable and an SSE register, - // because it could either be literals or an offset (or nothing at all). The low byte of streamBytesRead controls how much we advance - // the input cursor. - // - Used a contived set of casts to sign extend the "read offset" control mask and then use it to mask the input word, - // which is then xor'd against the offset, for a "branchless" conditional move into the offset which - // has been carried over from the previous literal/match block. Note, this ends up doing better than a cmov on most - // modern processors. But we need to pre-xor the input offset. - // - We then load the match data from output buffer (offset back from the current output point). Unconditional load here. - // - We broadcast the "from literal" control mask from the current least significant byte of the SSE register using a shuffle epi-8 - // - We mask the literals with that SSE register wide mask. - // - The literals have been pre-xor'd with the data read in as match data, so we use an xor to branchlessly choose between the two. - // In this case, it ends up a better option than a blendv on most processors. - // - Store the block. We store all 16 bytes of the SSE register (due to some constraints in the format of the data, we won't - // go past the end of the buffer), but we may overlap this. - // - bytesOut controls how much we advance the output cursor. - // - We use 8 bit shifts to advance all the controls up to the next byte. There is some variable sized register trickery that - // x86/x64 is great for as long as we don't anger the register renamer. - -#define DECODE_STEP( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - readOffsetHalf##HILO >>= 8; \ - \ - const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( literals, fromLiteral ); \ - \ - fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - bytesOutHalf##HILO >>= 8; \ - streamBytesReadHalf##HILO >>= 8; \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_HALF( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( literals, fromLiteral ); \ - \ - fromLiteral##HILO = _mm_srli_si128( fromLiteral##HILO, 1 ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_END( HILO, CHECKMATCH, CHECKBUFFERS ) \ - { \ - size_t inputWord = *reinterpret_cast( inputCursor ); \ - __m128i literals = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); \ - \ - offset ^= static_cast( static_cast( static_cast( readOffsetHalf##HILO ) ) ) & inputWord; \ - \ - const uint8_t* matchPointer = reinterpret_cast( outputCursor - offset ); \ - \ - if ( CHECKMATCH && matchPointer < output ) \ - goto MATCH_UNDERFLOW; \ - \ - __m128i fromLiteral = _mm_shuffle_epi8( fromLiteral##HILO, _mm_setzero_si128() ); \ - __m128i matchData = _mm_loadu_si128( reinterpret_cast( matchPointer ) ); \ - \ - literals = _mm_and_si128( literals, fromLiteral ); \ - \ - __m128i toStore = _mm_xor_si128( matchData, literals ); \ - \ - _mm_storeu_si128( reinterpret_cast<__m128i*>( outputCursor ), toStore ); \ - \ - outputCursor += static_cast< uint8_t >( bytesOutHalf##HILO ); \ - inputCursor += static_cast< uint8_t >( streamBytesReadHalf##HILO ); \ - \ - if ( CHECKBUFFERS && ( outputCursor >= outputEarlyEnd || inputCursor > inputEarlyEnd ) ) \ - goto BUFFER_END; \ - } - -#define DECODE_STEP_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP( Hi, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HALF_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_HALF_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_HALF( Hi, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_END_LO(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Lo, CHECKMATCH, CHECKBUFFERS ) -#define DECODE_STEP_END_HI(CHECKMATCH, CHECKBUFFERS ) DECODE_STEP_END( Hi, CHECKMATCH, CHECKBUFFERS ) - - __m128i nibbleMask = _mm_set1_epi8( 0xF ); - __m128i literalsPerControl = _mm_set1_epi8( static_cast< char >( LITERALS_PER_CONTROL ) ); - __m128i bytesInOutLUT = _mm_set_epi8( '\x2B', '\x2A', '\x29', '\x28', '\x27', '\x26', '\x25', '\x24', '\x88', '\x77', '\x66', '\x55', '\x44', '\x33', '\x22', '\x11' ); - - // Note, we use this block here because it allows the "fake" inputEarlyEnd/outputEarlyEnd not to cause register spills - // in the decompression loops. And yes, that did actually happen. - { - -#pragma warning ( push ) -#pragma warning ( disable : 4101 ) - - // These variables are not actually ever used in this block, because we use - // a constant conditional expression to take out the branches that would hit them. - // But unfortunately, we need them to compile. - const uint8_t* inputEarlyEnd; - uint8_t* outputEarlyEnd; - -#pragma warning ( pop ) - - // "Safe" ends to the buffer, before the input/output cursors hit these, we can loop without overflow checks. - const uint8_t* inputSafeEnd = ( input + inputLength ) - INPUT_BUFFER_SAFE; - uint8_t* outputSafeEnd = ( output + outputLength ) - OUTPUT_BUFFER_SAFE; - - // Decoding loop with offset output buffer underflow test, but no buffer overflow tests, assumed to end at a safe distance - // from overflows - while ( ( outputCursor - output ) < LZ_WINDOW_SIZE && outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) - { - // load the control block - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - - // split the control block into high and low nibbles - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will - // be an extension of the current match operation. - - // Work out the carry for the low nibbles (which will be used with the high controls to put them into - // match without offset read mode). - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - - // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save - // the calculated carry to use that byte next iteration. - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); - - previousCarryHi = carryHi; - - // We make the implicit assumption that the maximum number of literals to controls here is twice the offset size (4 vs 2), - // we are doing this here to save keeping the value around (spilling or fetching it each time) - - __m128i streamBytesLo = _mm_shuffle_epi8( bytesInOutLUT, controlLo ); - __m128i streamBytesHi = _mm_shuffle_epi8( bytesInOutLUT, controlHi ); - - // Here we're calculating the number of bytes that will be output, we are actually subtracting negative one from the control - // (handy trick where comparison result masks are negative one) if carry is not set and it is a literal. - __m128i bytesOutLo = _mm_blendv_epi8( _mm_and_si128( streamBytesLo, nibbleMask ), controlLo, shiftedCarryHi ); - __m128i bytesOutHi = _mm_blendv_epi8( _mm_and_si128( streamBytesHi, nibbleMask ), controlHi, carryLo ); - - // Calculate the number of bytes to read per control. - // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_and_si128( _mm_srli_epi32( streamBytesLo, 4 ), nibbleMask ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_and_si128( _mm_srli_epi32( streamBytesHi, 4 ), nibbleMask ) ); - - // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation - // this will essentially be ignored later on. - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - // I want 128 set bits please. - __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); - - // Masks whether we are reading literals - set if the carry is not set and these are literals. - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - // Advance the input past the control block - inputCursor += CONTROL_BLOCK_SIZE; - - { - // Pull out the bottom halves off the SSE registers from before - we want these - // things in GPRs for the more linear logic. - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_HALF_LO( true, false ); - DECODE_STEP_HALF_HI( true, false ); - } - - { - // Now the top halves. - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - - DECODE_STEP_LO( true, false ); - DECODE_STEP_HI( true, false ); - DECODE_STEP_END_LO( true, false ); - DECODE_STEP_END_HI( true, false ); - } - } - - // Decoding loop with no buffer checks, but will end at a safe distance from the end of the buffers. - // Note, when we get here we have already reached the point in the output buffer which is *past* where we can underflow - // due to a bad match offset. - while ( outputCursor < outputSafeEnd && inputCursor < inputSafeEnd ) - { - // This code is the same as the loop above, see comments there - - // load the control block - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - - // split the control block into high and low nibbles - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will - // be an extension of the current match operation. - - // Work out the carry for the low nibbles (which will be used with the high controls to put them into - // match without offset read mode). - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - - // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save - // the calculated carry to use that byte next iteration. - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); - - previousCarryHi = carryHi; - - // We make the implicit assumption that the maximum number of literals to controls here is twice the offset size (4 vs 2), - // we are doing this here to save keeping the value around (spilling or fetching it each time) - - __m128i streamBytesLo = _mm_shuffle_epi8( bytesInOutLUT, controlLo ); - __m128i streamBytesHi = _mm_shuffle_epi8( bytesInOutLUT, controlHi ); - - // Here we're calculating the number of bytes that will be output, we are actually subtracting negative one from the control - // (handy trick where comparison result masks are negative one) if carry is not set and it is a literal. - __m128i bytesOutLo = _mm_blendv_epi8( _mm_and_si128( streamBytesLo, nibbleMask ), controlLo, shiftedCarryHi ); - __m128i bytesOutHi = _mm_blendv_epi8( _mm_and_si128( streamBytesHi, nibbleMask ), controlHi, carryLo ); - - // Calculate the number of bytes to read per control. - // In the case the carry is set, no bytes. Otherwise, the offset size (2 bytes) for matches or the number of output bytes for literals. - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_and_si128( _mm_srli_epi32( streamBytesLo, 4 ), nibbleMask ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_and_si128( _mm_srli_epi32( streamBytesHi, 4 ), nibbleMask ) ); - - // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation - // this will essentially be ignored later on. - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - // I want 128 set bits please. - __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); - - // Masks whether we are reading literals - set if the carry is not set and these are literals. - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - inputCursor += CONTROL_BLOCK_SIZE; - - { - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_HALF_LO( false, false ); - DECODE_STEP_HALF_HI( false, false ); - } - - { - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - - DECODE_STEP_LO( false, false ); - DECODE_STEP_HI( false, false ); - DECODE_STEP_END_LO( false, false ); - DECODE_STEP_END_HI( false, false ); - } - } - } - - // Decoding loop with all buffer checks. - { - const uint8_t* inputEarlyEnd; - uint8_t* outputEarlyEnd; - inputEarlyEnd = ( input + inputLength ) - END_PADDING_LITERALS; - outputEarlyEnd = ( output + outputLength ) - END_PADDING_LITERALS; - - while ( outputCursor < outputEarlyEnd && inputCursor < inputEarlyEnd ) - { - // This code is the same as the loop above, see comments there - - // load the control block - __m128i controlBlock = _mm_loadu_si128( reinterpret_cast( inputCursor ) ); - - // split the control block into high and low nibbles - __m128i controlHi = _mm_and_si128( _mm_srli_epi32( controlBlock, CONTROL_BITS ), nibbleMask ); - __m128i controlLo = _mm_and_si128( controlBlock, nibbleMask ); - - // Note, the carries are set when the nibble is at its highest value, 15, meaning the operation after will - // be an extension of the current match operation. - - // Work out the carry for the low nibbles (which will be used with the high controls to put them into - // match without offset read mode). - __m128i carryLo = _mm_cmpeq_epi8( controlLo, nibbleMask ); - - // The carry for the high nibbles is used with the low controls, but needs one byte from the previous iteration. We save - // the calculated carry to use that byte next iteration. - __m128i carryHi = _mm_cmpeq_epi8( controlHi, nibbleMask ); - __m128i shiftedCarryHi = _mm_alignr_epi8( carryHi, previousCarryHi, 15 ); - - previousCarryHi = carryHi; - - // Note, unlike the other compressors, we are essentially doing an in register lookup table to implement the logic here. - __m128i streamBytesLo = _mm_shuffle_epi8( bytesInOutLUT, controlLo ); - __m128i streamBytesHi = _mm_shuffle_epi8( bytesInOutLUT, controlHi ); - - // Either use the value from the lookup, or in the case the carry is set, use the control value. - __m128i bytesOutLo = _mm_blendv_epi8( _mm_and_si128( streamBytesLo, nibbleMask ), controlLo, shiftedCarryHi ); - __m128i bytesOutHi = _mm_blendv_epi8( _mm_and_si128( streamBytesHi, nibbleMask ), controlHi, carryLo ); - - // Calculate the number of bytes to read per control. - // We use the value from the lookup table to . - __m128i streamBytesReadLo = _mm_andnot_si128( shiftedCarryHi, _mm_and_si128( _mm_srli_epi32( streamBytesLo, 4 ), nibbleMask ) ); - __m128i streamBytesReadHi = _mm_andnot_si128( carryLo, _mm_and_si128( _mm_srli_epi32( streamBytesHi, 4 ), nibbleMask ) ); - - // Here we are testing if the runs will be literals or matches. Note that if the carries are set from the previous operation - // this will essentially be ignored later on. - __m128i isLiteralHi = _mm_cmplt_epi8( controlHi, literalsPerControl ); - __m128i isLiteralLo = _mm_cmplt_epi8( controlLo, literalsPerControl ); - - // I want 128 set bits please. - __m128i allSet = _mm_cmpeq_epi8( shiftedCarryHi, shiftedCarryHi ); - - // Masks to read the offset (or keep the previous one) - set in the case that this is not a literal and the carry is not set - __m128i readOffsetLo = _mm_xor_si128( _mm_or_si128( isLiteralLo, shiftedCarryHi ), allSet ); - __m128i readOffsetHi = _mm_xor_si128( _mm_or_si128( isLiteralHi, carryLo ), allSet ); - - // Masks whether we are reading literals - set if the carry is not set and these are literals. - __m128i fromLiteralLo = _mm_andnot_si128( shiftedCarryHi, isLiteralLo ); - __m128i fromLiteralHi = _mm_andnot_si128( carryLo, isLiteralHi ); - - inputCursor += CONTROL_BLOCK_SIZE; - - if ( inputCursor > inputEarlyEnd ) - goto BUFFER_END; - - { - uint64_t bytesOutHalfLo = static_cast( _mm_cvtsi128_si64( bytesOutLo ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_cvtsi128_si64( bytesOutHi ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_cvtsi128_si64( streamBytesReadLo ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_cvtsi128_si64( streamBytesReadHi ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_cvtsi128_si64( readOffsetLo ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_cvtsi128_si64( readOffsetHi ) ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_HALF_LO( true, true ); - DECODE_STEP_HALF_HI( true, true ); - } - - { - // Now the top halves. - uint64_t bytesOutHalfLo = static_cast( _mm_extract_epi64( bytesOutLo, 1 ) ); - uint64_t bytesOutHalfHi = static_cast( _mm_extract_epi64( bytesOutHi, 1 ) ); - - uint64_t streamBytesReadHalfLo = static_cast( _mm_extract_epi64( streamBytesReadLo, 1 ) ); - uint64_t streamBytesReadHalfHi = static_cast( _mm_extract_epi64( streamBytesReadHi, 1 ) ); - - uint64_t readOffsetHalfLo = static_cast( _mm_extract_epi64( readOffsetLo, 1 ) ); - uint64_t readOffsetHalfHi = static_cast( _mm_extract_epi64( readOffsetHi, 1 ) ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - - DECODE_STEP_LO( true, true ); - DECODE_STEP_HI( true, true ); - DECODE_STEP_END_LO( true, true ); - DECODE_STEP_END_HI( true, true ); - } - } - -BUFFER_END: - - // When we get here, we have either advanced the right amount on both cursors - // or something bad happened, so leave it as is, so we can tell where - // the error happened. - if ( inputCursor == inputEarlyEnd && outputCursor == outputEarlyEnd ) - { - size_t remainingLiterals = ( input + inputLength ) - inputCursor; - - // copy any trailing literals - memcpy( outputCursor, inputCursor, remainingLiterals ); - - outputCursor += remainingLiterals; - } - } - -MATCH_UNDERFLOW: - - return outputCursor - output; -} diff --git a/contrib/lzsse/lzsse8/lzsse8.h b/contrib/lzsse/lzsse8/lzsse8.h deleted file mode 100644 index dec5297a17c..00000000000 --- a/contrib/lzsse/lzsse8/lzsse8.h +++ /dev/null @@ -1,117 +0,0 @@ -/* -Copyright (c) 2016, Conor Stokes -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef LZSSE8_H__ -#define LZSSE8_H__ - -#pragma once - -/* LZSSE8 - x64/SSE targeted codec for better performance with lower compression ratio data/less optimal compressors. -* Supports minimum 4 byte matches, maximum 15 bytes of match per control word and 8 byte literal runs per control word. -*/ - -#ifdef __cplusplus -extern "C" -{ -#endif - -/* Re-usable parse state object for compression. */ -typedef struct LZSSE8_FastParseState LZSSE8_FastParseState; - -/* Allocate the parse state for compression - returns null on failure */ -LZSSE8_FastParseState* LZSSE8_MakeFastParseState(); - -/* De-allocate the parse state for compression */ -void LZSSE8_FreeFastParseState( LZSSE8_FastParseState* toFree ); - -/* Re-usable parse state object for compression. */ -typedef struct LZSSE8_OptimalParseState LZSSE8_OptimalParseState; - -/* Allocate the parse state for compression - returns null on failure. Note -Buffersize has to be greater or equal to any inputLength used with LZSSE2_CompressOptimalParse */ -LZSSE8_OptimalParseState* LZSSE8_MakeOptimalParseState( size_t bufferSize ); - -/* De-allocate the parse state for compression */ -void LZSSE8_FreeOptimalParseState( LZSSE8_OptimalParseState* toFree ); - - -/* "Optimal" compression routine. -* Will compress data into LZSSE8 format, uses hash BST matching to find matches and run an optimal parse (high relative memory usage). Requires SSE 4.1. -* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. -* input : Buffer containing uncompressed data to be compressed. May not be null. -* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. -* output : Buffer that will receive the compressed output. -* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, -* The compressed data should never be longer than inputLength, as in this case the data is stored raw. -* level : The compression level to use for this file 1->17, 17 delivers the highest compression, 1 delivers the least. -* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state -* concurrently. -* -* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). -*/ -size_t LZSSE8_CompressOptimalParse( LZSSE8_OptimalParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength, unsigned int level ); - - -/* "Fast" compression routine. -* Will compress data into LZSSE8 format, uses a simple single entry hash/greedy matching to find matches. Requires SSE 4.1. -* state : Contains the hash table for matching, passed as a parameter so that allocations can be re-used. -* input : Buffer containing uncompressed data to be compressed. May not be null. -* inputLength : Length of the compressed data in the input buffer - note should be under 2GB. -* output : Buffer that will receive the compressed output. -* outputLength : The length reserved in the buffer for compressed data. This should be at least inputLength. Note, -* The compressed data should never be longer than inputLength, as in this case the data is stored raw. -* -* Thread Safety - state can not be used on multiple threads with calls running concurrently. Can run multiple threads with separate state -* concurrently. -* -* Returns the size of the compressed data, or 0 in the case of error (e.g. outputLength is less than inputLength). -*/ -size_t LZSSE8_CompressFast( LZSSE8_FastParseState* state, const void* input, size_t inputLength, void* output, size_t outputLength ); - -/* Decompression routine. -* This routine will decompress data in the LZSSE8 format and currently requires SSE 4.1 and is targeted at x64. -* It will perform poorly on x86 due to hunger for registers. -* input : Buffer containing compressed input block. May not be null. -* inputLength : Length of the compressed data in the input buffer - note, this should be under 2GB -* output : Buffer that will received the de-compressed output. Note, that this needs to be at least outputLength long. -* May not be null. -* outputLength : The length of the compressed output - note, this should be under 2GB -* -* Provided that input and output are valid pointers to buffers of at least their specified size, this routine -* should be memory safe - both match pointer checks and input/output buffer checks exist. -* -* Returns the size of the decompressed data, which will be less than outputLength in the event of an error (number of bytes -* will indicate where in the output stream the error occured). -* -* Note that this data is not hash verified, errors that occur are either from a misformed stream or bad buffer sizes. -* Remember, corrupt data can still be valid to decompress. -*/ -size_t LZSSE8_Decompress( const void* input, size_t inputLength, void* output, size_t outputLength ); - -#ifdef __cplusplus -} -#endif - -#endif /* -- LZSSE8_H__ */ diff --git a/contrib/lzsse/lzsse8/lzsse8_platform.h b/contrib/lzsse/lzsse8/lzsse8_platform.h deleted file mode 100644 index 5ad2e0a8853..00000000000 --- a/contrib/lzsse/lzsse8/lzsse8_platform.h +++ /dev/null @@ -1,73 +0,0 @@ -/* -Copyright (c) 2016, Brian Marshall -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef LZSSE8_PLATFORM_H__ -#define LZSSE8_PLATFORM_H__ - -#pragma once - -/* - Compiler/Platform detection based on the table from: - https://blogs.msdn.microsoft.com/vcblog/2015/12/04/clang-with-microsoft-codegen-in-vs-2015-update-1/ -*/ - -#ifdef _MSC_VER - -/* - Microsoft Visual Studio Support. - C1xx/C2, Clang/C2 and Clang/LLVM all support the Microsoft header files and _BitScanForward - - Note: if you receive errors with the intrinsics make sure that you have SSE4.1 support enabled. - For example with Clang include "-msse4.1" on the command line -*/ -#include - -#else /* _MSC_VER */ - -#ifdef __GNUC__ - -/* - GCC -*/ - -/* - Note: including just would be sufficient, but including x86intrin is a better match to intrin.h on Visual Studio as - both include all intrinsics for the enabled processor, rather than just SSE4.1. -*/ -#include -/* _BitScanForward is Visual Studio specific. */ -#define _BitScanForward(x, m) *(x) = __builtin_ctz(m) - -#else - -/* -If you hit the error below, then add detection for your compiler/platform to this header file. -*/ -#error Platform not supported - -#endif /* __GNUC__ */ -#endif /* _MSC_VER */ - -#endif /* -- LZSSE8_PLATFORM_H__ */ diff --git a/contrib/lzsse/premake4.exe b/contrib/lzsse/premake4.exe deleted file mode 100644 index 50f4c86b528f881a6e56a76228e0fcd32ad512c7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 518144 zcmeFaeSDME`S_nUX&WFk0RvR2TC{4lh}EJ}gP0bQ7LbxsNvKHCsVs|3863ig8J|O{ZAj+r6)dG>UeO(A17@s zxcHBgE{T6_RmF;x*L`*66<@Eo@``J(y)Ietm8&aOrmn5{+O-w4=C@XS{kp5Jo-%s$ z$jV&Pzcv?NWPNb%^8Dw$t$$kn7vA67`oZ#F>+e^V*YSPcxFgFC=yFq+|B!F>U#~2G zTjzhc`~&@N>F)(!yD~0q{huI@gdC2G3yK_5pP73}zRW&HVZo>ZkHfJXOnhH3;c)a^ zT;Oo{c<2=D1nuWShohJ``IrB81l>0BNOL-Z8+AdcCzbX0;FlyiX8pRv5z$X>$3D9F zw|_pjqce}3f6E+7=>Ol6qvr@XAOBD7<*H0cUcENS^Lf`W9)U~7Y7cd=(F%v-@+m8? zx*~an!_i(xri}dpzE=nihW-UX$P~MnWBGZccb7OE?{r`{uA7Egic_5Z>Bnq-Aw;EaPa2Pv0 zW`8U@&fPiF(bb(gLDH`~(cy^2eK*Z?$lIeQNGV^UaFOM;>t&C3INEBv+k#7%Y^S}} zXv|#imPTXhkuw|)o(f}TpZW#Q#*Q*`Z^F?U&5rZfJscNPUfOSsPo@2sx@@TR(bB-Z zgKf7j)~!c@A)E4K$N8vl_8op};-SNDB|V9=1h&bgK+CiybOt~{%15ahMap9kc`dK!cZENf5PWtWSNL2k681#6o!rE?g-azdi+tm+g z#r(SsK`g!*SmSpKLdKN?MkG6~LZC^x%{>V>RgF^}nGV;dzle6dlyqfZvWm0g{1jN4 zhiu~k#oP@+)oLyQs;dAcekU0T;QBosG8w&Hmy>!oi5dMwS7p-GccifQkJ=1M`UgYh%ozh*qQ=Z+K7i6B8&Qn`r#HHplX5XhjS=yc&yR7azl9(DnqdREBT7Edx@914XnY03{tqP9*L5g)oN@JlBNG~c7AiIM{TcEt%BBXa!h~kgYmdtjaC@#j zCt z-Lv0y_(kKLf);7BDc7dYZqqJ!2L+Apsk+mjF;3|;S0HMaSUJ>b%2nHKoo;lv94TKW z?2ZTsHGsf$yk{(P<;3c~vixjl*b$GluO}QcTLp&iodT6K7058ukD&pxpYAgA-7N^n z$+Q%sP`V_&)TRClXxrMBwwv2+IXEb_FQEx7wG)&cH&u6U%k4*~5{q|2NYYgyZ!|tk zZa3xuv*mV6t4$e&i=*nyar740?!84sMQc>u56KEGcsJcbf(UT+MrFR=KumQ3JJMB? zOX;N00Gx7-2G|;1-5cTUL;3>BL^i5^N=E!1po*$S9uoyob%%UAqw4Qtq;V;90y=^D z|1z63cGN(FXD7als;wvkGc;iK2M@p9HZj|Fa)0Wk=uFk=7hv~#Z*7HKpvm^dLiULx zQFU`(dsrU~S9Fu|po6e!!w~jOfQ-fO%OUK=3IB?)@uXtI|%(J7ZDS3*aMkUMC{k+GrHSvQQC5!D|tqHOC4ic>5SbU$(XwVrR znZGDF0(+-|t$5WYLF5rKv#A47)x?9KN@PyFM(6#NBoTi5j2)H31lj|+lIK%WsN5ff zqv7Xv_4}#A*m?TVfg9}fBNOmVfk}N{`^z^9F#ww%AZf< z&&TqoPyT!)e?F8y|ByciBAL*R_%r~HC=34O4NZI+FanjaBU0O~B~957Xts7QCt(Oe z^r$D(_2pCO01p0C>>u7CO9cE!s`}#}-M@M<0FsfEU zS~9d`X-r;!uJs=FCDJ3Nvl0%YzbJKe?dzm0!gh-)CxS;L!joeKiDg!UEe@`uYAj+S zk7+_I2z%d;*vO?t)fY&CnXN+mzd-Rz8#m}SrXxKb4O*ws#py@2vXxc_Yd=A4V@HL# zeQ?yjJbT!vJ251GOTw^GW3olH5N=g3BCLS3u+{8WPwQv1&u$r#X%S!r%ne{Uq8jX8 zc^MqeW-TM80p}z=jjie|k!U#$%M=6`s}pn3KvGn#OpUM`j94!{IYgi|fAM*m6z#q) z;zcVk}xa&E1PyF%eUN7Bs6-#i>Wc4pP%KO)feo1T&FlvuXP>r z-r7ZR<49p@MCTJ<`#;3&Q_n)=&mrf{dG0L*)6Ys)OusByKD{zoHvObjap&~~mC4dg-AQ@6Ac^K) zvK@mYO$uSr<`d*=)9}+eS`XCL^7DXFYFz9Oz4q?Ci*%m2~YaJ}YcDg%qt4 z&TexaHZ^=R13wVRytgvutX&yV)e=*WGTfty?&nj6SimiV{}l;CEP#23F$3}n7=|AD zF1UJdet%nA;5g@Xy9d;Z`PsA|2&BoWopqOaA0Cr9tKB>6&k$Y8b-`bf&dn1V?EwXL zB@4G4Ti;pdNd4Jzch;SiD&3hUl-Yq@YyOcawDJAeEf83%xieywBnpk4g)sQ9G$(nG zIRA)d9}m; z8xJC0eiXEVVCf{9-{Q^9)8>qzDWaVJ*8JN?hRMet&@A@DF!`97Jcz><+jOjbJ!XDU zcf5nOcCu!xd}V8}11oH*PsLgjS$dCaaAd6!Zp^rzkYaUZsaN8QHnZUQaIdJM4?#|A zqJY8yUAUaW$BI?B67BHjlKxM;lSAmPoS{dvQ~?K(col7^22`7E&{%IG4RXN6YO}Gp zdwdYAUq_?y6*r6Bv=gP1J$K(LjXdMXbi_B2tS*3z;G3V3kSGwl^kLk6#)Qc$5s4uZy*!pqD^9wi&WC zK8w;%$?(Nk#{U(+m;~DTt2_&CNdGg>d1yTX+?RmI43E?8094YIa9Dn7%?5%YL|MlQ zRL>qWh@umTH%q@qO~X;+yh5N#0j<@t)N0W8{6Eew-(Ba6gavG55d<@dtI@*?Q80 z@yB^%)IXJ+JvwKC)bfFLl5dyfNbgEtBM82@l zK9kiDYl+lIslJu!mRkhCJwV^qrki>Xrf2Se%L3k`rQYjRgB+_(HaG%k_R)-d@8!D@ zAyl{C)3MSF`ApYVEf7=Apf$Slvoll2OlvAgl^WfJG`tYD4ndwN_&%bpt?kJcAVP=g+UgeO+Aa`Dd>K=PfM~zQ zFQFLzJG6zdLnPM;QFRE+iyeGSG=3SmX)SToU0N?=aLT~M(7xa_dxBH}=73l-s8@9_ z(%S5cs@rZ98t59xW?AHz{n^^6Iv!Bsab!bIZmzen+LcXt#k&wyr%+n!pr{J{nHe*C z;djjz;)OUA&ABulLT&S0NF8&p`YCT>3ipG%I!upqTh)2MFD~3EGLE)D-?~ZFeKF^! zQT0bKb9nE>?w~f94&krY_GVCgJNd56g>K^pcd>19nPWR|D0U<(&3PW&e(Mf+A8sfK z?Aq`yw9@mA%jj`CpGmv73Z|EAhe$m!^9G-Bq+s3gfi;yI%FJ26^Wny!aN2D)mYcJF zGhlT4?9TMC%#bI$!j-KNG4}l>niX>CZnMcBXsSvTnZc$d+s%5H=`e#Hohkiv9qO0} zpj+486n6By=Q8JbjGjK{^MN^TZ$ppOv9%A$9O&y9jcrM`^BZn+rxlNB;T1TXjBPyf z5e1U9fj(%F(FOh0N6ethbep|B`&>r1+xu`!iE~%rbKQZrxyL%g4El`j%i)rF?o9n6 zbGF}{?J`H0vps2dZ_v#elC?jL8Qo1eNe0SP{Cjf^>t+taqmy+(y{#v!xvs zC}u`{X2YWNoJIE7?Rv=f%yi7VX4r2wy8}(`mCkgNJJ;Ae z&!a(>GmuKQG4RZW2I-y3VNXxL%h?mywxOnLcgGYnTwWb^x0xhW#$M>8*BTEQzx3izRtD9j?{SZwQ7~Q!q=jDmq}&+ z(3(OhaPfS4kcEaUN}acm7WoF^oE1*a@Ja7hG(V2LF>9! z)pjkFtz~A%te%fXTJ3yqd_Rld!fG|=`ON67o_9tDo>?=(+*aKqHJs^=U|lr025$Y{ zb?l>cjB~UlN1EYaAUtceWu3qjTF06l5#gP1#40sAF42#dSWe>wJu3)?n@yh;)quz9{Za57QJ=%V=DtrcXU~GFq0$m|FcZ z=vw1Whx#(OkfO2PP`}Kv`fQ@mMwLBXIfwTkl*ijpM%?Bz#{QzDZ}#k@#|-<-TW-IT zx>gYYHM!ZB5KrzgfIrB5$T7Gon4{JFpr8K&@!RZ1HngB$n z9pw9-T(>+73wpevlphScSx^_Sn#4)l!444PDEbJZW~aY<6lv&lU}Q;ZU_m<6 zU)@-NN47r_f9}*EPRbv}G{nY{Y&*JQzTb?w11+K}THH%>^N;qd?XC-Xyx-|I=h*nf z=yDhhZeL#BQOHh85_Rof8Z!zq=H0?u~vU>Z!&FfbM zj!ejBPionZWvHG(o)*`+NBrh6H*Fz-9c&*ujQZTT7&o##KV&S)VMG2moip$?x)RdQ z>6{-!esuPq6jn~>AWnz6n&!Y)JzQ;In&pg5l{cAlaRjocj3sKE{vT+Pr zZKVY44Q`Dip?!9{=lEp8>RMKK@38gOAPv8Kh-r$tA<6$BsdB_C2EyPz(%$6cmv1Y^A=hAfBjjBM{E%)57$i z7N-3}yror_+wL@5qvr1)R_EX3uU!pO8DOUmsc~3~fpAr7bbJvaII6;pG9W$4zdQ(r_O!K0u`|V})JAX^o!^(PNAucf zS)519fE+-bN0Sh-bphfAeLb9-)Kb(9(N!%uV6tq!8gkS~}qzkbHSR%xG5 zi1OuZ7(}gDG-{bYZ^|%ndAS;LFEg8_B4d`No2CwlQjP8zbbufS#J1Yqbxr+ACkQfc zVA)})qU2OI8O7R3-?CL(JG0C}XtQf~GN5sIu34zDccx%(@s%2TRz<$d1YKrKuFMFE z98VGGmeI1f`&Y~8!B3dGs5>*KuO%~QpPr9nWH4ov-tRdE&?$WW+GNW#$c_sMyN)MbvjbruV(#9C;mA zc_Z_&kWe2uwG^u=tXE3db`2~5-0jBoZr8dK%n^Egk?b;(yW8kVxnK7@bo z!@*(=0yr0;wTs@6gS%+SksOT)!~!Q(V&JhxMh!3cLtXG_8w>3%s{~!lI8dHzGL zZLD{(vF3w5ZDWl^=8MV+9egu%@bx7FVl3t&k!7v>g^xMYZ|BlVs9In?s zvys_Vz96dWZQcRhgKftkM`;z_ayU5xiSLJK@F5E+9hH~c) zYtSb)z23?(iP|WGXlQPzXoW6%p%|B`B4byU{Gd%saFfxWO_gk2fdC`f)pg)jd7!TB zA&a03e43nr{`Otd+gBj=N5@&U-C}lNSV!H?J(+qR@m4SkD;$?z;ka4h=moovV)Ijf z&Wqlb7QIPSdnjDD#?^5-jE|=-0_mD>sga_=QdshdBUD;M2nBrhPc5R&`7RhcaoskGX;duKocbcOI_<;#IO4sMp=8PEnNL+!5UZF;z7ZojgOLQB!MS+ zTr`qdaMm8n{z8o!hf_+gY|TC)_3w(eOT0Ut_=>NK zx$Rbe$Z1a=@5PQJk||NV$R-oI^cWX85R)OFahjvoAr|1)8-Z$}L-zx6Oc|L7T*X|DNn%I5v&Kz`NU`1SAFUc~E*_!7|k1;NH5k+Bh z5eBX>rbpEwKWUn0zr3EI{dT&H%p^ACS^3$`eA7d>^XrONmPFOLvSu+$w>UQ!*y-8g znK;qjX|XDb zQAH#aWN*C|TNwU+k&*GuS}%V@_n^~SmK;|foMUf`-0V)3 z&Q6V@^4;+V$hCTm^>;hA$`UK)4TbKGw8Wp2`svW!>Q3FPAIGH*7uf{==Vc|fJ1(m$ zTn>04Q}0O^XwZSylli8XV6zM5&l)X6pMh+{ZBD4nhjHRo4OrHo4J=I5@11agY;Q>3 zdHBTbEgc&d3{jHD>czb<@-gwLnmTpz3t*F$?5=Y~Zv)w~p2BG~DVJ2Hg(d^{C7ijW zt8h?LGasf29KKx6t-8LzcZl+?TC3W1rKON^G|#p=}?0pJiEt4q}Bg&u4i_ zn4`(klg4+y%MAHzvvt#w$Ftxhj3%c_&Lp`J_Q|-m8f@DCjfUbL5EoVNLUa+@_%;-W z($=!e?6nRL2~*1wvINp&q*g^v7~BGb=W(4w$~wl5lQ`r{7BAU9mcis{z%8W7Zik=} zkq0|Y*9jMYXt$g^Q(6{JE4vrWc;bj`j=X@014Ph8sRVaCsH1WKZ48BV^_Rf|APnvf z*d%j^)ZCg*@Z3%-+QF?hG7d>ubU@mz(7@T7xw^Fx6ojjg$WvnKS3=r)7a=t=rfSB3 z#!PvMxn0M!foC-Vw8?(PGHD)QJGDR*um5yOE&*7XiQV;`fhokaYF@og1K?x1`x+D| z#l9o03`ErfNLKcb*g>YLb7hT=Bc&*HoRpb{`AQ?18w=XnyqlIX1`%|EksnhFYh=*1 zKWOS7WQ5v+r~_3|^_N_4k4kTJ@F&*Y8nG$e%wn3M$x+hTvkV8_(AH+N+r|Usu>6qp zVMx%UPuNBH>%$w3|+>8=BOW%M>mYn1Q^U8+w?RT4?Sq**oQ z3)RoJ5V*1ks>nVhEo8c+L_`@vCl5&r&cjgVC-#O4fpt0``MiT_?G(ya$hg?aBC6pb zfkwT-co0#Hu=bCTvwCy%pqOfDL7|`{LUU< zAS1GgB@|?jv@Nr?pzC#S*KM#vqBy=u^VH|;FEU2;IZhdSTPmWOac45AC#TNt?GhY` zf@D`;0Is#OvdJQ&e`M+d`QyoDmDyPny8mt$;o6cI*3nyuW+WkO(vtsdk z$grlg#&6Nk6ar2Bc3pNF?f_Y`>a8+@I(6iEEPjVB_mOQM?Ef)6MAZM#EbwNNqmBNF zNw=x1SnubCZG%UQ6M9{OXkY}Sb|j;R7KzUVqikq`Cc7XpF*48>@E+o@wW$MxeQa)& z$B}Mol4*-Y;rH>X7dm8zS;H06aQUs#>P!gP)J;M5`5?6;qXwrJKIXsL$b`3v(u!bhT%s)C}S ziOG!jFkW@`V~Ay~X_BMkvu6ngu6$2(=+1V%jM2h|XaaH9hyqU>S>)X$*sk4;+iTuH zoj2*p1l{!_p%3$(D2UGj13AIijN6v}VTfHSD-j8!HPn>|iey+o{x)C{3r?Bbu3$Me zYypbM^u#aL*!&ZG18q{j%pQ?k+a`w<6+-zEv1JPYq+=Xu)NQlBs~F~ini#vBoecRqc+fwnVxanY2e*ZXKJ^%!`l zULp|-Z_xK(NXL-YISa8TGHgQ;oVsK?1CZG~8PU~d2O;EAO7i7S2Bm&f%Q(@R$x%o? zwHDIQ?j!u+As;54Z5~QP6SNGpO@$VglDhTBm~`EzSCqF;mR|j6V{3FSf2{63zXrqp zW9MJGhh#A(N6KR+oO2+Gp%zuwl9pZV%C061`|ld@yRH+-bZBaL=scZj ztbf4KF(I3JAQO5#5_jpWXFt&JSU%`m_SvH9WIBa(WZ#-#;cvPJ`A#H&K=l<(MxIFg zH?*Jz#>4Gw5UW=)HmNtbLWVrtY6aAqAHhhF1o`O^Rxz9(k3*(pxY#}Zk&p1m?|Riy(@swK}cU) zn`+G^!aV99s3*<;T7>%=17CF9sm{sQk_C-=h9Vq#%10r^AE2qIT8EUcE>pFRi#n@F3hIF{9Ee3&y%oe=_KO*xh2egP_9u zFtWW1joQ^sW?RaV7|se@-P^^}-D92D((;LSk?YOQRbg>GxGML5oym!+s}Kcpv?7@I zjtG6n?C6(9tZ{%O2G)m25pdtrw2n2DJc0G|aZJ$VAVPBP6_E?f2j7w5mqpbxf2HBv zN=LSvF%_}zVH^E$#=AKzrC?~cdJD;+>Z0js+AWb|rP&EHB)0Zf+fm*C_ zXN!g`JAv_m9NzAM1VNH+O<P1u0%k2EO}@MTSw*Fz+WgiDpm+e>8X zx6$#t^>`l<5e9U@ybsbIodKe38~S?vWT&1*S8%Xf^$S;`4=e343En07Z;+*3Yvy46 zk2If3?wQmcR!L?~Pndc@SBvV^utYl*OJ%+v)K$@`RK)clKx!1iAgtot>RbDA;LL!t z>@}m#$Hk7R8t;R31jzHclGsFJG`84HEHcJ9-2 zl&3m2O9vu$;Zo@nyTy(IQum_b=y*V{#Gip2G!(b#mFd($s?btNnoZip-RjwV(W`V( zdv%pN4;uHc6hReJVM-^8nEpV#wL*?Rl@9rZo8IS0W<#K0$Mc zgr%MsiI;0gqv~$tN+e#x8_O)M)ygeYyqCsuOCy|nS^|oYQT<46TNVbW2(=;$uZ3Q9 z%;$!YQsh`hM$jIFyp)cS#H^J2JH#J$NvpZH`t6zNCB$8I)L{|b0N}!w$;k*m8EDjrr{)sBSoe{+q~F|bjgC*%&>jSowqeddmMz4dw5m(qhAR%4{o!b< zdRY${N2n_`35nCHex?mQPv^9gi4~fkZE01XiJ0O*t#vlpzeMENmhJT}=V7ZTaM(Nl zFu_@zvWW=73TPfWMRpSWG_)KQmInS1wIv2x)b;Z+Jp}fs#mN8K-Dtl%dhl95%$pST z03_kJ6hfNV?5(E@X_%!0k&3?*bDW~@PzaQ3iCKD7V_(ZPHUktz)n&S|=|hcaC4)Ug zd-1D)9_6$^1CJ}r+x0C}E-Akib=PJnM{-qDvYFkKb7Rq_kL0bag+vmO6h}!5{CeXTZWax`Ypl z%Ng&~&Ha|RYEt7>-H=vW zTHe^S3mP{JHRa1S#pyY_p{P3X&1|+wMtdSQt3IaCmA$@9l#~lj5EXJMC1i=f;k4sq zS`ILSINb4*MAZyCFVk2s$jHJnQFS$QQv15uKNDha)Z!dkBpQX+h1Ais?~VHrnxuSg z6Dv<&We^fQ9nn?%<{(SU2!DW?!h%UHu-EdNgxt|dxg(rXVgYQWrP=)H!(CTybwIcp zcH=-UQ7Q6*a>?lyy<%{yb7@YH?2{r@ykJu&j>t=)t(|kKM#hIL93WSLG0K~JW9sL7 zHI9%lqvMb3d7TfzL0(@JB#+BZ{eetU{0*HErD-M{3|d!Fn|7{*RQNi8OqT>TaDf0X zrWLVd1<6q$J*H+*$m)%$>t2U2_Gac5KuuG1uUIqm+OHdbO}mAVrmB?kLRW(r)JxOH zbQC&>J^~a^O$PlEJwAz^<29lji7pr2ToSJ#E2{jIichmAX#moVuLt+>Ij#Jer3*ZE zM0#n8KR%nJ8JF98(MECEIl3mKXcYC4v_5{27alK3JXCFj32Z_Hwb zZ=!>!x(_;uUrr_vZqRwQ>yicJzpjBibtwsf>9*>LixrJ6Z0%tSIN`L8W2UXsWkDq1 zm}lhHNFAa$iL6m4{3V+`yti#)x3TYt(e2DFeatW5hTKjZGalcA@g%ZomsY`e?}6MN z;Oaly1H3|u>nbLLO&p(5;jn6D+)vWHruH&X3>ip|IhMyVO;vu&8$X~8vlHM*G0bc+ zcGL*<)Nv36M0}y`5n-iL-lhRXTigClB%1a(v?gWM|9!nG@>ncR{|p}?7T7Q|MsF-0*I8e+d3Pyz({auV zJ|9AySY=`q2sex67t5?DP`^g1LZ3Ins)Q)V)J1|xwD=_5c8x?F8~X}#;zWnN*r98} zi9J-ZdGcwS^tJH=LD&Z1KS+$J+n_M99zwjy#(Q``RGWB74qaa(JypM!o05sr|Ib9R zvo*6y2o+#2d__pQET)cOV%r9^4*wViMYScIc7(2N z*~sL@)SsArBC})a1|Z=5n_6V!{`PLVnJFZt9N4iMwx>>uQfv~S2*52gPUb2|Np~Gn z^;+&$TC+*sFK~)*tITCSB(PBJG@55*pkgr{YXmUKw;(8_Z`-=y>1VpW41GO3I{X}Z ze#`?9#Ne2BFg|@Zxk*2yVay01a&G5FPU;{N9wdM(Qd% z#S0!)Pw|D`TfprSLUXu&p4a;%{&b_ew`3HUYR^aR*}EUO&8YTfiL}KfEi4Ojk9!D~ zwA;LfFTF!sLB#B6z??v=!1j9cFXo;Ft>QhyoE9;IYgsyB0qe*Qn~~|g0x0%%;4q8z z*2abtFh19hX-XD1B}X--T6&AIR~**t-eNWb9acR)@MNvg{ZCZfnyFO19dz;aH*y)& zYPn?Qn3(z}Av*bgNOH-{Ar5p)}~?J{pwo@9XV7!V0uH(fb`l!Dz&~gdqt?7JRg*5kqNDNm@wa^axDrKFGHY z)|k=A%XWFIdaRW!5__!W`Q^MU=5HjeeV6sl zQEBaxrdObzN|nGv|HeG1;f=5NJufE%%OjEg+U za&4t!{=LF9nBhuoGeSi|5xJdwdmSRSQLB0htf6=b7Ax^?`X$A@x9Y2EE^>Nry-8|a z-yFy z-4{6=XS=>-UBNUox^rvxR(zI#N7xX$rB%JF_nphF3u%bNXoU0JI?2ZwjTE-BaS^wk z5DrQ4!dA6e=lKbsmj+tZ1B?$2POpLBJphy^c>I5|dN0wekj%e9tTr?6wTjS0k z&stSRH%ADrzpl~e-Sk6R$gc4j^9HiUy#8#fD&$Cy2=0_OyT;$DKBWsS$LC2i#l}1j z9;H_GF)W3Y@R&2pw7~(O>6zu~4YM=*%<>qzcV#xw0HB(ok$irM0y^ zD#T^FPk!d`YZG_Z_qp=P7xF=NJ^LqMm9`{~V_hubPr0M<13gsBjVBw>RjUQG_NSH|Xp0C5Q^0#hBv41nkuW6W?SirP9VjTx@w zF?FQ`;$ZJlP1}1cbO$gs? zD^2;4yWzatWxa{8v&sphTi9>RD1^3iXL%(0HKsg=G!8Y;!sw%gzpXB@+=rn39~s#U z*qj)hAe`>R2o;CL z$uO1HSlWu2uG`4Do(Pb_wt;edfqFmZpsxYtR*~}4L{1ioxQUd>Q6lz2J?UBB8Wmhf)_4}`=?sj%DPP#Tdt>|Ht$VUbLjGZN}D zO%|NtkdO>m&S>^$^P?L=(;7;F;&yg4&sS(cEdi)uW0}5nkxA zCIosmJeiqUz=_sJSTkfAvK*7#-qD?@&u+wVzX5B}?cCP0Z^EdZ=~;gmuzqSAmT*bP zBTOK|MeM{r;==6!SIo^oQ<>nx6H(kJ0(mdl-lC}+R!{D!ATdFRQ`QC8m!Ax#mRa12 zsZ$HAv4i<4|CBgZB=Apk_cr1D@hd^cU-=y1bBK=vG>_!d%5w$Zem-aLIgd|}&m2Bu z$eYeb6NqR_Z1ucV*c(zzvJMIF6DQVrP07lp)a5z$c#dhxgcPLYO8})?-K;k*%B+iW z1+E1&IKPHh710}~!?`O%b)dFSTOs9n%eDe56dQD9033n%o& zp_ehcNDQ5aHH;MuyW&kaoj&*~N8p=lD#fdCAIuLtF||Z*!>z)kA4LJ`>(o#FDGg0e zuX6XweJg`THq6=@Z4!X)N}SVn_%p1|+TEE&1sDS;X9Dgvet~RoZ1A;4Vu_OoPtn8m zTjkscrV}0|!3>K@09lRIxfSB5PKPUV28M`jW5(#zBoXq>tT3g2+p=_R&T~FG%R1GV zQEHdsJSVCETL0E5LdIC|jsi>> zP?~Cj=cBNQAWxs*o*-LiCBiSR-{~I2Wd0o~G(@+Jl}E3BRHN2Zd6SxEJZ6a;fw@wq zvWZh7$q7xVCJ-G}Z-Ih{m~3m@i4j({PP_^x49931j?Dsuyc;uyo4 z@>%R4m+ADgC?$P=YJ8DR^)9w2jT|jnwv(yPW{&4j@jDlFel}4DjMXzw8$T08z?t4{ zJ!%4LjWMz=cxK}}vR0^lCSA%A>}?zWD$~ovJ(PtsraF4mm{Hr(#lV~EMn+n$MY@LN zCIHqCCu^Ym_8{N33xhq|NhVvl9Gyq|vN>q>%}l5IV6ndEWgP5kZa9Xwwa>|h3;v`f z2iaz{LWF*B8Ij!R5mTdImO+t{C=T*^=AU?-H_&SIIJp{X`Cr6M%8K!^x4%J%+YIb} zGljc!ayoP{rbOjqMD*>XN@zf8@92dvxE15cU3?#5hx}PVI_C00@hu-CIM+9{yj0$ z?@=yGNs-2x5Y{{$MWfZFau+Y6FKJ3|Mr3?ABEm@M8otLWfL9$oh1@}MLTJbVxU zenJ~D)j_wsSWj!DKrpp`#FGg}shj@>fcZ#KF<6JjA>5^odE5!zQo^`>Wt)|{Tt)Fb#4 zndq84y+hLPHvL#V3K=7;?{Ghi7bW8C3mpD zxLxk@y1pk5`*_}702JmB68QaYoo@rq(6vs^C3s1)qWBu)pWGpl9 z&X&x~k&yPax`cRaopY`85=HS!-9~Ay4ZF~6yBTZjOlj9nJ9n_+2gcR6JN7@qnAzl- ztgFAkL`l(j4X-hEidTZyiNTwya~t(ER_gG)F@)#Er|IMxolJDsJ#vE?a`x`{be%DQ z3~QDdx^IhP`qy#iM|5fdsp$Q)rq4`HoSsZh$fh3DeO#q;|F+FWD|@){%XLb?ZXT+7 zDx0ki-2r`t9@m)5A?r$@gotF)_n>09(CEp0!E(ExCIh+Kh8ql*>>H#69m*==_ih1c zFbc}vF*qKPWj{@&IZeGOy(O!FaBxdjm`nEBywHVqh>eo~*rA}VXAyP4CPi)%_GgK= z9ad%xEt;-yujB)V^xn)#PaFja8-D`dX)jP!KV7IBk+k~|0($2a#7el7qsFi|FI77S=#on%VQd;8`bP-b{D8}*#F(!gg$I4WK zN}ALw&JmiQChIxn?o7-ZyMxj3Al0*}JNC;OG=UE^rXGBhrdnIoci)jxqBiGbQ}@Rs zx}OxI(*(6hzjFUEwI7UMrQcq@M&7o@SLnB=_1k^%PW?8Wl2~_%<*n*M8rIXP$$YcU zzLUZ-m8n*B!m#Y$#P87A*XOdYZB3P;F5&;@x}JwqOAt{J++zJBXuN^@qI%~|!| zqiBmDPdWKop(k`2vrH`ut`)5}4e1EI%1-u30xwAwh8u);=Ntbgc!%hABLo25%1IbH z4GU5JNb?;I>~{LM3C4z(y!knJxi9DrW2cjp*_7t1A9ly(&h`tem>Wz?MahhafNfPj zpfSc&qk%MbOdA68#33+k;ZWg_xz_w<2mbw?UNJ`DMp+yoH|%9cUQqN&HbG~iLL#6~ zB*I6Kvu0&&ztxFl%uQ}lqQ^_rbn@`>Y_f%*7#y(Q_k=lfVJ%MvZG8yRkh5``|zP1A#mv<+L8MJl_bNUBkFZf4=XEg zx<@%fsx84|X-7ESjo&TAu}&@tuThVof{^bc@NI~@V^YrBC({_$FsWu**RnUJ9-@O- zOikDQ;q~EFBt*AXja3pmFkLKevzX|iTVz1$zbR^T_(A$QoRT=8-bBW?aX+=1PmTn3 zv5w0;BmjGIn>}$KHTBJgGA+lvXvgAs?4X)x;!09$52@L33@4!MTOSVL-x2x3~YV(}ZuYK7|SbDQ+$h%vxxyFXKF4ChTJ2!~Tm~ ztQ(Jxfd}?#&hDY#JERt~$0iExC8=C7oo>^Uu^Ub*%s$ z+=iYDi)OVf?t%tDscA}8K_|5@K`DoO#BNQF!s&8^OU|0AtYW=4u*-Y*Hco>di-y7V z8)BCwI9GOAKC+4h|H=1rk^Hi%my3+rJ2!AG)*>+7%BlkD{T{wVZrp+%#B0{#PcQV; zwbgV~W~Y81UOMf5au>2*ucyGwv|Hbr;R`IPp{sRo;GM*KwRrG4CTk?^k|;L@*cXl& z{XRRS@?8uon1PFQhWSA?o5<=8$I`B@-Pg$FmAmUGP!A@Da!NalmCb_CWU-4Afv}8MNY9XT!E8tANP~_wiJdG~4 z+UVdo_6L8=6fn71X9MbRxzIITr%ny5@^cm4OgCE!T%T6Vl~Ej+T*sce&(Tq`bsYwn zhAh8865!^!_0@T0#_aN}F}ot$TkpGta=rDQJEZtIW&`7@;K+Wu9w>R9j@yriNADrq zxW1f$2h;TxTkqu68`_4mHZeW>4PU0dGNN8;(^6cIM)Q5xrEe`LwQq)ErxDKIz>PWw zbn=2+@*Lwsd%gTPlmn)l^$m7DF0nIR8#$Z$&PB#&<;ju8XUB7>?XyWK@#yA#CU%$v zp;p4`!=?dJ3EG|KuBDPI`w-?iYHP{4Zn>^RuC7XGZ==tgrit$|a|1Ul!A6sTCz5F{ zU>;b$ZkVX?o_30<`ZM8EW`kt$`E!8QgChaPVA?(
t@@HVc(0Gm@Y1VqZuM# z>kT2n)n8j5pjadw#$8((=?#1C<5eBecQ$#9Wo0={z+iH`emsP2vpWz4-Nw3IZWrvO zVOz3YgP=))b-B2uML2fv0a^1{TE?r^NTjO0DU!P@)<5K+wl@e@Oy=igv{~noV6DS@ zb#wHZCyI6a>dOfo5XDI;&nXT^C*gJ6A~`tAXf>wS#r;^^I+k)Gx#Hz?3nXs!HL%O1 zp=8xkrT)LDoVsh9BoWN~kj3 z$JAI&B^B*4^-JMlu?yuY(qF$sSGHe6IIi|3yo=BpBMUScWdvmSYNdqlAgh>Ct>$Yqr1Hm9}3765}BBgEP#a6;|)lgxoIBllTy+efNs(ptAQB0Vj4eb~R~oE3P$ zZ1Rw$#LZOJwu$`E!v{2qC=9U~Y@3BV;-b4|#pSs1-LmcBTv5fz47GI?gNE<^x_QDK^1rtk@Ej6QlYkVMDwR#-)j?B)ro@Y+V0IQ(`>sB!jzUylq*tRH@#*< zUwYPSeP+W!zPMScpRaxzkUNC(=rImdf){aOMka4rJ2$g3n@%0<%zo~@`2rYy%c42V z^lb0L&+@Z9&RyPzdjq??w^fiM{<;x(^1O2Ygs5Hb!!x-S<&CcOQtE)&pANli zynj(%U!_CJ$e!xWh}GwPxOHxNVP9aIHzQ`B_u=nJxQMwey>Oqg&sn|GKF{0a-PB2L zleg=2zRb|OX5W^ZrC`s8rIj@s33V)ROt0{EJwa|JTHt(f?c6}?{JRJ4@bH}h`S zOC4vMGd-D0amT)!DZkTn3gjoHe21OlMow`~*m4D)S@%8|NY8uMoCkj2y}uxN&7is( z)^TMGT~Dt`H3wEzt$)jE)LHx<(|>SF=co?0D~OGDY4^=Kzn9E%Dy=ZC;Zb=Fd+y%uiO$_ioz3`~2j!GEC+% zb+DeP3ov!JgpeGj>yiB>(z?zCgbi=jrFm2@ZLOy)NJJwb2kQ| zk^=;XcyF0OJ-~Es)2-Ar&F4TZoX4c$^84OQ|d)8N!aMpI#&A zXM_E@o$omBk8}hbckx}s6l2U82RyR$>DGeO`|!%y=@imNN8|+F;&|8d)z3O#OE2+Z zo9HXjxW;&P>XSnab6nm{5}+aX7=Kz#9pd%|uT8#Z0$PZrY)qYSmo_qJk+W5YI>PJ- zUGs8Zny`m8Zs#ZUE%+Sg9aR7OeHto7W)lx#Z1c$lhAw@9pzycguCbD^>tR=90=>D1OQEY@=BgCnIK|i|7qH3A zS>&J$xH@YnK{1ne6G6gZP1qmqq~Oe0<8{NcP6sc z9pd&<6@`o@EVhwcIjHZmO`T#LkGeGH_=&$rH~6{u?ifyraUsE{B3^cQ4uz)`rrtEe z70&$0D%74{{$M1w#H$JN=yTRr0G-8(%02kbJ#u4Fln2a~Dy#ZmG71Mv6`BpM!IuV? zv8^o7;$OXo2(-b}`kZ?~Z{c~e=c~_lG72p5y=`XHXU>@wn8S_OQT6mVrk3D}Orz@$ zvuogrIP=y7YzKgQoB%kd6to1U5VO)<#+JGtb6G6&L#(^xc6(`N>n=K6mJLr` zRu`VS(wl0EKc}mHs`q&Hx|xM_kjZ7fSUBrLl&}6ydLsU7tNdi~vV!o`JQr}Vg$Q0|VLXWsGp(@(C-3ac?!QgO7A}8E z2(mithKDt^+HtKz0Qvp}AWfS1L-1e}$Q*IIf3H9UB=tV&HaXfpiu+$wq&KKLs>N|O z;>-ErWFJC)7J&0K`d0|{0vDTD{CIF~xx^*k%#(3-pF#lD;R zKqPJ$LDW$ZEr{d4Uv4byi>da2&|;;zbMPiazoBU0B-u(oefDVUQ zzakxCOZ35Lx_(6_v{inE_szqkr|y@CYoLN023a1FdbeY#*uvo%1zrTWpk*e zFL8*P?Nw|IT=?^!|9s?{ao6;gMXm|gl*vxdQx^z$J~>Bzh)2+p?jQ=gg8g3I7ADOeeR*S47k8lhYmLY6Qe63(a9GRb1F2L5%*wbs=tW@0gr$Ee>o?ma=KV z)g=?+_aUG8#jP>%+j)qoeXC?Ro^)}#-Oa=j$W*k3@9LX_^;w??897BjKqMt;8I{na)`q&aKo?eig zxJeFF>N|^_NBuDia5i<#>E)%8_TR8#xmBkjRJ@A?t37S$%~);4raN@qtg z7g+l_@fE_B!exR`dmJ`gEf&KRxu5O8D_V`^YdiSa z*VaVJt-f-;Jm=#d_u5)_Zm0i2K2@c+FNnw=OW3XLZJBUN)*Pt`86WC<{4g-b{#d+7ZR(o#0n{uS8)p`F1eA*;u8IILeK45~o$S#NBbV z`K(^qqs86M9%8WE1S_ilr3oP@xQz(DYIbx+H>IXTvuCRhp?m8jB8m$LV2UQ(5s4>0 zs9Nkk*#f2v#~^j>0lb1-Fg7=Juh}>?s=L06>c)CEv({0@iAxUuRAi^#sVt#4wrK)O z-0RekLLKIItGm&I1F07q6s;bvVGhjIetsLZi3Xvq)}DRxyHU>w7p2%C@gH4&8s#a> zMS1irzd$w9d;uYfghL~@W~UY??p{JTlriMiY_hOe(LGL zF2v9XSAsga*eWBj2Z$Ofr|qrkQPLVYPPyK_?m%7^>ZO^Gtl3z>ni)^@jqG``Ek_$$ zDBqf0GhJO{KLpiRc;FmD6PcqJB9F6qbl8t`c>E`PN9tT2KM?t{X0o|~nOCP%(by4` z6KjP8sp?pI5Jb+~6yxi=w1HzX4;F+(SbG9#xqX5y9wFQ&DGe=*}RHRZkWOhVTrIn@EKPhaNv=`a>N(LzJSne zB)#yB^t^rQl^;s*#~TDC^d$%@`z!iiUgbFbuVv2VBF1w33KA36`Z!Z%t4oo4Mn@cb z+lL*P{HhQXp8R;-QuoS5IO)T2Ry0LeKYrsK=ixoE=%!ul&98?RtSOue%EWxKc=U07 zdn1;RRzLq@7^Y)fvWkaY_p*>U^VB^{bD|n@!LKcG$(e7&W9Id~y75V`SzNcsz0#eW zWA@egQXeNO7A$B{qiTV84+3gugKOlL`Tov#oasy5o97hC{or#cDj62hAV0N8u!7l$t}m2;U9QET3~a0!jin-Yfa`=R64+hl zkfaLfgqmCl7AI3lxTbL$_x}c4DZ({eC+Q-UxgzCJHHL)N#Kp+i5}vr6G*5^ar_szx zUmd7oq3Sn_Wg1FNMqkfaB+w?i9WS4);kJoN(A_O7-EU088E}%>QfBTF&%%07-DP83 z8^#1K8{=N@oS*at;o!i|b^EmkbIJB=y4g+SM^$`QALx?#GPg%3a(G;y))cguMHR-?ffp~3hJ4Gs+0Md|h`b7#8Umu{bmEia~5 zdri8%GTq*kZYSTLZl9KJFH5&iPqzot?G?F(qlt<1kV<{aYG~pySH^k-Ij} zFT!x^+U?yWlI>7^vCG@l$rGE$+28EoAyL?(rhQk2gW)Xev8zQQtPWxVw{E$#Zf8Rs z0q?V%yVFfUeh5W^hzXIWL)M-_eL1>3==;(6af!A3Bukmz^J;Kcu)LPNwFT+`*CEj; zJ<#apOY71rfP6iHI5AL; z3;cnXQ?GHmu1sfmGX4;BK>K9*9fvl4?K`SA>2h4d3qlEB*Tt$Zz9?0tmh#H__hDK8 zMr%-5T#)fnvcep@CG4+sF4!^$1ph zt((mS9jXFK!JyAvS)QyyXH=R;SR0s^dW?^J&bOQSfnp_#BE75SGOlzxxcB;gFtF&BM$B83rgnpD;)W!Ls zUtYNi#qcp?%TMigmqjGaIz#7tn_||h`tih@DD)iC2qLZbWwGZPJVy7J2y*`9Gd1?? zeMO83eko$TedT;Xs<`v=q5qqW-6RVv?1BLzMvWR24Qe#0CSAZl$R^yvhDer(5bJeaZHurA zs38QG*lZrx+E!cJ+MBKZdau1`u~kb%6BI9qR#9xF)#^!?ws@&Tq3rved7e#z_5E8u zAF|JLo^zRV&YYP!bLPx!D9?#IaiSn&EcNJrz#Gu3FIb2bp{WJ+)Guarho@tTl^^{N zj=o7RBb|j+AL#J34`hc~ch)Q^BE{vdFR6G!kdmsnS_`fw8%x=ZCL3p@^UWv$lU(j_ z{9W@WVZGRV{eyL(G3Mg$TET{j-Oz`$Z5dgc^t5GZZMRFqxTCoE+6Nhh>v& zD(;?}*Hh>3o@<9l6E31c7E7D9MCg8BW&Ugnrj_2P9$QM}-nbl60&LVoFN%p>j_C&+$yU4d0wPK|7l3X`qr8(Basn~j$W1e z+MTRAYZM*wIm8kp7$y{&@~FEp{x@gXWYo5`2pfSE%BFM%EMtheHfjM<5*tEp?#tIS z41KDwKg|VGBl_u*p-)Bnsl~`iSu3cPJ7QB<@qYFC(wHx7P)_H#4kR{}=CsD&9zL8F zo&0sBz6XMCMi#n|=iBm4ayG<1$G#`f5Gz2Tb-(m1IIpECIj7XL3egTuZ*9G%vC5Tl zZ*~PMUBPl!aE2>bie$BjyaQXbz_nzz=Fr7!TawpJ@4Pba5TP8kba`T19!#Wh2Fe)m z{nDFW1|_!Zznlq@jWZw>aK$z&wN*M&8)pK+B0J|iUE&P5Tk7sUkqZu*8G*6z?0gQpAK<0Bu{>51Hdo1^nbp`3y?i-`dpC=R+eGhO z$;O5OIoIuOoRn-_G;lMI)~b0?uC-HSljK3n2$qQdYc2u4B0F&~hWLi_%C_eQ8@9@I z5oV*Y$ikJuJcwLj_tMmyT%5cVnHs4~d=Yp9TnTqm4g)*8Ty)Ix^oiAtrLobRgRtAd zgKXG*2MC3pi4qYtWR^Gz<*?}n;wgp&g^2?$Bp~EPpwUvtRv8-U|t%+oGb{eEyk`bA1@s(OYLxctyyYum-Cij`vi2%qDYV>GBbu6`l_?+>O>ckMdZv0!X$RVvVhrE1HO&bu zC~eMt(5`vmf6GqhXoUp>y`oIm9?XOj@sp>e^dd8YtWVv&*RRm@4u%>F5 z&Dq+OL$AOo8O81MV-kZGb&St`9`|?oJcN?U&DcPRS@s@~G0Yu4mN-4n zQuC$emk37mNm6}VgxbGflkf3$-O5wuSE2=txad?uo9$#agfb;GQO0GWSZV8f#g54(>f}@4>y|rj&3zY>PexNA1};>WR{~#yq+0 zFD`ZyEC5kV%@E>o-3+AW@njp1ghW>at-JQd&$C*J`EWM8(E3mW&maeq2Nu|vGck)f zqmf-+vvrNT^wZ%Q4$u9;9y3Gq)7DH$mPQIZkzIJNCrDc&=%8F_KW=R z25}}Rm7F~yS>A8;iekVn(gJ@O;onjjJ^t!GxLweITc5ZCib&)>97Y04*gObiC=$|* z>{_WJY$ku5WUfR^8k3V_M~A?2um}YWinL@tV9JdMZoSbCG*uoaCaD|2elKj2B&IbR zCW)I2dY7t}%^i1H1lIr9<+Mw7Z+0R1+u!~&@>{O`)iYf5QZ`l8KceUzLq*w^4}G)u zhGDawuQ-bT4KsnG_`9x`;R~CG@6;J^iHEy;zeDdU@Bbk=K&@{1Fo6dzy;5CJ2IJo+cHM27wsZy={tvmJZ?cW-J)7X zMyW*NHB5|-jTBKFH)xCv(}R~Zc`b88Md(>jn;oL{p|MNTkyZ+fj^?z+d{TIIVU-kcj&;M1w>Vj7B zr4;}J@5l+tDgjBIh?sV}$Z$Mzdq@LYnEwa^64hDg;uX~fqh-I({i4iOW*3`Mroz@3 zn}aNPwI}eHN!|2fRAlv!YjlK9FJL%b_&0QoNsxmV z^CGCmbt#1>OA7y~UFzD_v8tIO(67fRkUicz4NC_$QFMk-r-DKo%?EivwDE zeApbz6iey9YnMg6?$FiQU=?$-VmRf?bc+<>cM8eS75%VzF+tm$prw{)V4&v1mzlj* z*r8Zlybc_q4kdIV7yZJG6ok4&WwSB=}3q{zt0T3~p2gHz7ZuZ@0ljVUf zkrt%tuTU9{(k4_Oue0im*hIY=%BL&71YC8f_PxGGYH}0iGX&o>V`CK!4e^*<9Ekjs zE)drW`&aT{*r*<%>Ik0BsgxTR_@@C!n!C8^skAH%fg@Xqeua+^UhLIMpNV>S81-bA z8h2`J8`JX}G6T@@c86|(8>PLG$7m0@o@SdvgXyVkz9)zSf)22bTW(70yk(5CxIE4- zRGX-O`X&G`w@Wb>(>l(Llppy6b%*1>L(7)_Eh|uT zBi`^3(X3*$?1lOdT_+r*(!nX2%7<@(-9>Iua0tYp%fLIdfo>T=EZ6W2JBjnm&rr-E z=H(#fovJ=tJf=Z`brzpMzpjh+rbZSy^`eUaY&nFrSNw=-kIPou{}*9$P$&5KNiV&bTPjIFJa!%|f#{ScC%4j|83_0uPY>u7j5m6FI~;iixsbj*ZU)k09I;gKB5)u)`Gb7vq2 z*a74M@A~TBZ3&0W`@hD2kA4Q2vOO_(pf9#7oBvqyL&0N{SeF{B&YaIMDeWCqA;+Gw znU@PStG)wyw+}rx@An@CsvJq#^{FltdjCsSMYe@IK}D|fsA$<@zV)=Mg`)KNSc`oZ zY?8~3v4YO_b24LM;au%ywqV;)1uxfv(5U$1*$gWc$-|~=6U^WVR+z+GJ-7#;ZIFM0 zzD7W{wDb5xv*WjAi=RMooyuo~QjH)6pg!CR;v``PC?ga08IoD*`8E=YqP%7% zLE4eqo`_BAS>|uik3hhw*fwOc(}ZTXizob20jxqdp$w`GMsoayT#$QhPApD-G; zvalsSq=-0n_{1V(hKsC-pVDIbWf#WDtwK#aZ+{b{`eIw$)Lk)727O!3WEm;MrTRc; zj^p7I8~YZP>tY)!V;=v*z+33axpv3j)`7kJwek~Yrf*9f%pR@d8decM-TWlX@xCo5 z>BA|$E%(tcSlm62Qklp|dlNLxePo>M}YtGUWGqi z5It+)bBrFa5RybOXsck#z}K|CDwGnVi z_G6#Cu-^Hac4}#IvN_Lbv^O!kE%p$Yky-7|lyfJH8ufy0&z$@%gwygpuq`h~03YCw zSpRvp;u|<;;1zlOBtUPn+Y{876)T8#46G1@_@q*c>{46BvS?s6e~2U{{lhcm6xlXF zMwv@!)mCC+<_v`RBU{yi=!$_u%u7U4BizZNoRlWxQBAIww_%F2_Nb2^M~)ui6x~5^ z>Ba1UJ80z-OWkaj+DZhyoB=jP957dz>+HnntmVncfnC0?+y56VpXt;j=P~KaKC$6( zM}s?a;7zojpIGW8_7Ze*tL$Qule4D>6?+MCQiL#L-eRet8PONv40p|!k@XX>q_ZKf zq_(>uKXGVM`*Sw^Pwc;Nn5&Ci4%t+5xoF6$uR(2z&CyRo#(x}hD-p$c(W|YQ^_AvR z#ZqyJbwKo#DeDa|Ch@_j^(cJ8=1VLjtA3`1F)KTUivU$T>_jesPD{}bsZWGmm%PmT zqq43nWBf8mdLI;8gR+prm{hQ&+QSi93ia`Ud-#|1w57Q%RYb$TQu9bI ze`gsh7PNYcC0Zi3AIV@9l3!nN-dS7}rz*MA8LScV-B{TlERi;_D9n^MnQ7w%!kP41 zY(-{r8>HlaK`A-QQIyWMkAWy~V6zC(0TWyR}$BB*d7f z^CB0^(66pEo2JUpc+&-z`~Jf9!JMN86sr4@?h`)|3!AcV4e@Wz(huv2lv*7ri!A^k zWmvD!Rb^Pu>BEZHhQ#cZu}`BH8zYqdvj45kKYiI5i}d%9)uaq@I=-qLOy);ggMZ2J{t5v8zt=l&|KC#Db+15o*eLO{@D0dRW|ZyV?%e>!YonT6+bJb??l;W6k zUMPwZWg)c-Uqc!wHy0{?76TH|DB&QpwM#aZ3O$fEEB?1Oe|8(7iM$iwe#sW`u-qJ0 zyzI>4rE(n+{jSoKi(}PN{R_PO4_#`^RzFZnfltJr6SLV9A6~hsJ~o>O+`cWNrClW8 zFO4@uJp*hha-szTeewf>T|iDQV7^0i;o0jx(X$hbC+ zU24v@D~tag*RQldd{kxOer4f5gH(c*nAOVpGrFd;qhpE4IXaym9hV-(0dZ*HyP&xi zfgs5uh#Xe*T5Fd?XsO1%S}U2_R>(vw)OtP<%*Q!_W1gX|$;mADgc%jGa%C^Sn|jq& z(VHA;5@D^M3E{8d40~O}<_-_dZCZm-Q=}4LxRf=h-N16CIaNHG3m{?xHh~B%=Lxe~ zg{%q@Od{~N1ngaQnvv@_`n z8Gr?!SVp&DkJ)8po)F*qxcE@kA9&t3kKM7e%`?B-3$nyLJ&p6a7t5(;ftBNXgeWD^ zvz8;NdAhOgFx%fAu@0Y_EFBehIwi7Dqh~Rm5~VRB!g1slowwgZ?_slcXKfusGu=dc zXRiHAP7c2!fP8n2tqygL2JC!gPQI7l7|OTd`kO=b?YZt_`f-_)Z})J!_kCQxfRpcK=D~rl`{VK*dHL0$d{rOGr{|m7?R@s2Kj0L4>`$^jo$7NxQcO=7Z*g*7G~E4{ z@pm=igvYM&awp&1-w%QD%a7z!FlLSf<7}tUIObpoj2n(FhF(8aH|WGqJ#wSn*@p(i zF*h-J@x1b4r?^ z)lZdg3TAW7__$oZm0aUSS}nICeUY>>PV45ZIU^xywQf>-*J++?wbK2eiqE3rSM#cDq)dAm*2RN5 zK3*!f?8+G(`}hVer@h$j0ossR`bXhD%-Ya?EfPY=h}@-$|5MQdQ(aySb=yn6#eA9y zBPAq=QvNv*TQVGYyk5AqcC^Jzku0T7mYLd70sSxt%ZE#gK{1wq9iQ zaaiP6+ab#r*wKDW;F)%=>*0Z&Xax7me5A)l-Koma%L4~|iw;EpQ?VNdY0D4B3j;gW zpJ4U*eobb2Rwcg0Yak!_^xj)ad z98ONmM@B%kT*|qZt0r9*HoIkAa>BQV@p-VS6Hj+Xa<|@&pJ!iPFPikf@ma`#ca`avK^s5iZPnwp@ajMeJ&(>iI&B;xKZ^SbhBtEvt)P`?$eA#FDy@;o@{~ zPvkxtM~FIeL`^61zK4_J5NbYxuG!Lx-aQCub#23;zf~D**LRcDHyAxDY|iS*mWebI zOP5;iWzm6k+8DV(jQ125yvRP{o;G{3T}j?m(lQYI35ZlqiV<#aRBSX!&jCzpYjZfV z6`)(pn-Qs%^W_@uTx2l{%MtC~yD2!I0k+EoPowWhX@^#5}BpjKp4gQTGW!MJFL^8aPBrJNB2Dw?*?tMa(rT=y*dUk9`(ys=ZP&m@5<=1GF#mvKJ zC@Qv5+%dVtF4(FCizO<~G^yxehD!CpW2AGhk&77NRM-zWz#Jf*j7XOV4SSW-|F2xb z19!5hS;B@gX2&>HAJGvhXGFX@Qdg6cp6l~hFSVQhh2|;Laf%;D;aMymfx}CEnzKSn zaqH09Zj0w5@%^tzqcS@l25-dd5O>2=(t<<_p5e5#+J;8Pb)IzSed@)GA|fG%amv9w z%aYp`D-1^#G0lXx)FiJO1q(zsNRO%TZhIX^b}+V*zVkCS#6G(#)J-j;%~z>kObASh z2L|Yeiu!&jQBFx?-bj8sCh7wqLh+Ai>8G|F=El`D2F)=-DVpDAwM|pY`QdOiK?>~0)oitB7 z^!d*+7Wm~KG+$%<)PfLN%fQfeq3U^85FgN_3zmMWRDbMavV5mUT1Ai;#5pT>=DGSLRr=-Iwl=T1ajijAl%-} zq($VE=*G4e`NN`^Nd~$yZ5omzeLZH6%nxR48`AtEZ~gjON@avsGV(Y zNt~8r{ssO?hKXeONFxG>e7cO`WL4Zq4$KSGU|h+zOpo;7HJeSq+CT<-2%mD2BC7#c z@;qnLWi4kfZ-s&TaNr#%bmx;bEXcX68^Jy%(&NZ# zJ3|v0DsX`fxm`i3@HdJDq-C2pD@0g_P2lYXW}7`0OIiR45QKkcX4R9!XmAr(5Z<05y_A1F37AI{7W zT&zP0?RKck*JPyRs(Fz>klr{1frzN)vLx?ukLZ$#pT_;?9Y9PE$|A1{uI=Lw`00f= z^EL8@{rVpolcu&Z^w3+VKZ7I7inMV$d(OGb=KT_$bNHxA``Pv05%|y_@QIYXf-N+WiuV33EP z1Z8oC{+;&Wj>}8=D2fz-#M{i41atM^f2qF0KP=vLJc7C30s%X6FXbQEtiX;jU0( zU_#*K$X%p|ef2M!?~ofYvvX!ePOJ#?Tk~!5YJFH4>Nnh&ll%p(HS$9#XKv7!tXwK) zBRhIOa3%h7Vr+C`AJ-N<^Pk_RG)(W8r`>a&8>r!)#oB|34?Jse>bP=y%Kf;99wvGR z#}0*kO+1nLog77ruO&~f@dO@S`@Zji&Tfc<6B{;F=NIxRokv)f!O~U|`zwd%SS)v# zXLXpPa$>l@ZWp!Ym~BFsrZt;u8A#m{Jtc++DZZr|592T))i{`3{3`zV%C<|~nQzR{ z7Lxf#c|eK!R|WdS$vF{}eytO{@~K>+6fg1gH0SyrL9sr~(a)RNzAb;HUG(bTm7j*# zRS6@pvyCfO*jM+$v{c;--OK*iSNCF0Xn;Xy9qilM&G;tjUUet>+{ySpjCFi<`RRRp zN#Xc(C_Qst_s?U z6+bT7F3i*}0Dt;Q+8Aztoq4``4zvqNdAs0{1DH`-QXIf=wQgk5zPguldh)yL_N40e z44lHcjohv{GZz3^4)I-t5{gbs^bz|wzRT=mStRS8l7;5$+RPfm`Fea8qkjz9E%7Vm zRVQ7kU@o0J{ZbNzmK0vocc9|wTwE5a7bC0Y0Ef=z*6aCflQ+5T!i}gop1@p>M3Pv! zof}bQeX-LuuDOfQ=2mXmD7EBI7coS+1l=B`LAPBPdRl#H*U(13sYMiAf^!pIrhd$J z>kgxTJRwl$iBog>svb}02OF@${a|@~Mil*{o%zo1EBY{!+hinr&4~mScR${_7Gu4- z!;ENtPYynk)H;%jl-uy8PoORYOWy-b?8fgfS`fuF-L2SGl}lh&4FK*Kpo&`?5Y?P> zgPhunzn=Z4i?umueH6A1O;__T1y^Od9EJlGbA*tE(65l$hK5-?`HhQgWm~cBS~#0) zvqSGfCibRtxuHh$e_Hp@_CnMPEHYBfFF(`zioxZ8g_263H&{n3~eR$vaAT?!4fnkYE~`+xm2#X+muR?#ISjTwPP=wU$L;G;ydsEma^a~A|AfuQIqwaS?iAc5OtM( zUY%L%iFozfOZv?lDb%+oc*CbFA<>HT{Xu=lu~tLgwW6?ylxXtZnp{M->H7X9%$YEA4Em$aM4G3Osb{aXiq2asWVlX|U)L}c8;ARx@!v(&|e@}iTJtlMMa^6s^# z3G!{O;Jw+NC*16s7G4pe@%hv2Ox8Tf&|psyPJm1$?>gRR%G_aXTeshogBM83adt6W z#!#{JPa??1ECwDo!mGWGG_ui?6u9>amu5ZOK?5+KP}`;Pcl!xB@4#U zjUFP=mM@XMHKf>>*w84ogqxZOk*Znk^N};y37NI8(?L=9S%cO)rWL8lTtZSycquV2 zps?&xouh+q*|n>-R~fy~7l&%y!PY5qUl%fBIaDD_X)dL<7Q*A)$p***Oy#3b?C#n7onAyzD}uP{j20ecAIz_U84MNI3;Tu(e( zZ)Bg6seI6wp-KN0k{#J4fUdq-ln5U4ZJHf%oPiP=s--sBH=8?>qP=N0OK#SBR;5pg=Cs!P8a@b< z`{5q#ELwhCO4nydxsvQ`%IwWK5~3}ZtuD`5buPLKIUGK>Y`gEbTY^7jeO$B4%f0c2 zB5sK%=Z$eMjPG+!pcDIRKC;jR>=cgZs`8U{+uHLIRikljTbvXMcg$XCGCBiu$rEhK z7`r%C_f&G@V^BudBpKtUi0~aBPon5;qmXhzUdI_BGwAzmO=a1Rzy}=@BHWbap2WIx z^C63fTzGZCs7-2ueCxzkM`yK|JV>4hU}zMPXQ=`XY$5u*5emE(o7+q)DTEplIa}dU zb9vdJ+zq8-ayF+lvG~1ALt^nDVX^m1flPc`f885ihNH**?C!+p{f0w8CFq&m+mH`Jni>a2Wz zI>yf`jfGKs_H3AG-l87* zF82sxrK~!4WJ|Zq)M}IxbdORGvUL-Z_!8v+jr&kk6Fr^#NfhOvh`&yyjqY$n>|P=Q zY71VCj8C)tgTKuwZJ7Gb=6pmY#PW)#QCpcCKR+}$h=g)4_4@LPz1I3dPC_Pkc8tm4 zqSXVz(bjom_i%jiPyUaK%#*1%j{_qMU~dd3VT2oj@PwkL5dV0S0!l#h1dK+aL=pd8 z)5H^YcGDv^DRcaIK58;wRJDF~+2LzK!kk;H%dJ(V7}+n710o#PjE>_R(jTbz#Pe|^ z+v=Z-tH`0Z!nxLHjPLhgPJe8ZgzQ~ciWAN+ci7zIaF;+dm7R7Vjd!S=%SF2mEFfG z9nx*sR9!>17c?r7&mV&D^84hAQqBpLO#-al zFQ>}bZg;^|8A|)Zq3hC zyy0@I>~emq7RprV`@_O=DY$IevPtWwT+{2tm2H)mt`&vNJFgFQ?Q|*Rd!Vs_YyZAK z;0Y&K_212v+RYV*&DP=%@3Fu5!Ahj`F?_B6=wQ#L5Cdtw(6WF_7@M67hCv$ zLzR=aP5~*{{SM|vU9H$ZBKjybgcN?0C1CFdaZsl6|20F){b;2C6{>+U@rA55JYhan zu`Lt|MaMLog*O81lNmb_IAKTddf0+Bt_QzU#V?>H zFmpDt{BFb(s#31f#3uhn-xe`pOKf~0Cw3N_d$?dRnG#pLK-%EBa2`Cm?vI{J$Yrd1 zo6WN~=|qW3WlFDzoxXXK4lG{nC>%BxHyP3=ujGEy?}r=mATH-^othh)GwQ}si8mRof%BnHRD&P)uB$Ke>4 z?ros2Ldz%j>L7t7f zdiPDSypIsv3iE}ry-D{YlHq`4c$o|uU;Pnj@bEMV@^Oy)5u0}Z;p(Bul)e074^FBQ z3*4FRxnp|foZCIu(>HfC?!pRjQPpIJ6eQ2@WK&?xsk|~%ga7(akqw&PZzALuGBP0)xywmf zrb+l)6hdR3-KA~XV%gCsE+i2|CS%zASlCwF#_RRv=5IF);dP&kh51cnXegB5>J`C? zOiR64Erbt|r%U7;F6NyqRG6i4H&tjZZp7GG>dj{~j~L0mVhgQ^rboFIb(S584;{6)lzGqryQzucr` zzu6GDphaP`nhLCyumJqc$0kLNWO7wIT!Q_`dV9XGI&Y%tR&~evo|2ibvX0R)hs`1? zQdRW#p)O%_4}Icz88Hf`#!XljhisunR(v9{$DE3fp`;Fb^duq!uamr7<@bw|f~)AJ z2o$oJdV$$t*9CE`5lK|+%fQUxbGD;qMU)Ko?TVbJFVc>5V%0;MUJY9{rLrYF`B^Ou z8eM}7Nd6M6J%Y=aEeiv3p(~LsK|1&Y0VMX93^U!KWl3|>F?D9(L9f_6UT=(*)4Fha?8&yN z41W(`Vo&xI$w%F%w@TcO zH*(U)+C_>O1z|Fx)wTzb6|^rpU3LY<^nztAF|yX;jC>px@bE6NR)A+pfuAt?u#)=H zvc3il!x|lhNp}?C=CZZ2toJ1rB77_?%!K=0;{ZKb>oLzz2K_PwKUrTYv`a4s&$(;} z7mpGpl>WAu{X3ONRS3JdpG}(#zQVu}6dgd<+TlMd_D}wPIu2ibLo(KzcRrL)ObuI5 zIr6z15nFbn0=!V|0}ukf%3~E!>Dnk&4)Ci>hroeqBnDS)niC5o%(U3KINuYs_Q@PD z9LxU+2{SI{!P~X&)iSe;BBo6LL zT19Lk+VO^%JL*eMjTWRQN5`ZmM!iJj^0G>{Hn2`(Cpn3|MTnHq)Ix;ZM;F27_S$kqH@F-i*_~~p_hYO9d3q1t^#q{V& z`Fs4^GW8|t@y&LkmoM#apl0%5YDT&O2m$;gYIDDAEFmtmFZ9GvF$sSuu!W=HskcZfAdx+(g z7@Ud0WHCq`of$UIK{o(M#O0cRtK$Sce2TCZH(_y?SywFko;O}nsF11=uLuj>n{G~& zcIV8&0byWl`+Fj)BkCjzyYU0-iXDUdKzD51QbbX|Y5a*(y=3wCjOw1Zq(bud%=PeC z+0)p7Xx|-dz{}k*A5mnr@acxzZuYVD9u-PH4c+V9G#ZgnHn;WJ%`LY@I24CG(1ml6 z__?{)V&*d`BX37Lj0j%wKH|Yv=^mY~4wj%ZS;R;{2Fpz!JX7W-Sds~5Jr(}k+HO#X z?&m_U(Ja;fn6jc-DiF<5Jtys=TVhF7JZ&y-8(K|AOC~a4;>i0DT>0t2hJU-C5wNu%PMY~8 z{kD}ReKiY4Wm5y1QA?TV0hyn2y*Ldt)m`0G7W1J#x4-XwPkNGl!#7FBu1{oiM(@;N z;DOYyp*(bTA%_X>)ZnhKx5F3lf)6Mvwo;p}RQBCsiPnTa!#Pbb^pa?55 zhzBTaGx$qZ3@t%MPT0dm9(-I`M$sBV8xFt&LHG0-5l?k{DCplG}PoUC-*AA*6j$Yt8ZJvy^A)qnv? zw6xj0%QjlCocSgE!?1z`(kKcNtDtj3L(aNSrPf38gSiz?bk5F){)t9)x!eeCHeFId zL24brC?%2yAvlGRHDGi8%<#O=940b?wMxp9@yDF$MlWL3vZ2@!pQdEJI#|paR+`>y z-cyR<2wCr?LD(_&xc`!g&)`hGH99M74)Q@K!JK{#@EowoblB_!X++FvHWNr^93CSC z75XAPAfrgQS|$=ZT+~}5hWJWQ>D{$Qse-8?uguph zb~)2iM7BWn#);RwgoeK;Q925TGE2Qo)7w%BT=tqB@*!E6tGw2kBc=#;%N)J-eVrrU zZ^^r(22xd9zRLH9-l6jhQQ@XfIMR5o5WJ5+$6yA6=Mtx6)63cIKL$3Un}_R#&F5u= zI4LmfoWrIFM#j#l?meEpJhN5pmA~6q*CN^YMY0j?Jf>D8A@IXc7&1ss=f?l!#HWPK zI7Dr6WhKJnWvm;Pe41;rqXwh~P2ET;=J)pcVpTM^hlzqFm4cQ5=ZhwAiUUjgOTmwr~$1KH5COghK)w z%n=@tKhTV&16hfG?P{sgE5-2wN`7RQA=N`--=7jYjz{evr4r9%(r(<`_O&o(6QK#p zU%e_VZwU`!kBs<(YE2UK2YNTHiCvK}H^!RIyoIF|y8wyzM6STbKa0k`UyP4(5)Et+ z8i?k8i6=+z%o3+fyNmfGGtpxH`KCZvIPegewG?}*ob(R;wuJ@II`F;h7x+2>W9J?!b zW5l%eAS{3ZChS?XU^#c-N5lq$cl)glcis`%L=PV0`@#Ub7zX3}+&CxMrfP-Q& z>PgMdISxpFe6S|xDgOVN|F`r1E&l(J|31EbkKZ%+&B-0jzfmst=sbS&3;5&7S{AAM zm+E>syy7MgSP62i=UCy;Lp%=oulsl*0*|kP#+&@7w8l>1=gio#FqiT4jMGH?=S_s@ zTrE13*7PYA?{kfK)T*$#n~I~HM;o{zMhYT1d?PAzr8_1sV`Uvc55eWl#1UVt(f5Em z-T>VX)cM!V37g9)Z5KKoyCC^EpO3ns4x46b6fX1zEN7pD(KNy_pNwoX-TrrZkuV6EVAav1^|}*tzJ_bx`<|P*27_|&2LgHwdg2u5A({2_EWOH zp}#()BK&%P*ld^N5?sw+(jSaSs@Vrg4x7~3@^{jMbS4`av5&eEkIfX9HdgJtHh^4} ztgW$`IwOK%{xuR3GP7>FboYll=umJ*G{-*3E0MI@NgEmyDZ&^K4ET0BCZB)9AVP^Z z=u70NysE3-z(;^;l$sAjK>~9nqelO^8pcR4Q{U#ts&7EYOgI{=evyQds1GHBp`QAx z?qHR*1Sc^wt$Gfr=Mhdb+2~w16xotcCf4fJ3Q#2g$$q(q+Jy=dmyknAjb+H$7N?z@ ze?`9XVGSqg!Af)OEs_gwj`3G;sR$xunFCb2MW;x>Zq6(N>8Xuq;#-7@Cr-`5`^|>F zpxYDft1mo>__=wXF}Np!FM5bao2uWyZjB&6U+5++X~*JPzdG4glWg-O+uX?$A-vJi zt<^A>Z1a*>a zi#CfZubfEEp_-g`e}YRDOZrmSW@~Jl*x0sO?+!(VVrg;&FF`K3S(j`oNjA+$Hmy!J z&Fi0Y9<}5qn`S1PYLZQnWK;Q-$vKtXP5BbgwP|{?sVLbLOE!fR{aeD3ZDg0r3alBk zsgLQ5dm_6)FHO-99ff&6Bu{^^TsR{76tkRcj6UdtOQQbZj5A43b+4vSvauN6CULi! zoDH5tkf6GpEH`@EBm{V&6UWh$S0i1X$SPo!N7-WPGSq!_sxOj_3mk7gud-R%1xoeI*8pzBF2Z%c1aVs|aoq}%3y{(M zZB~s$$r{DuDjt)Ba&wN1)ykXQW#@)T1dG3q@f1O$#fiUq!TPB(4x_ZauzC40+LOkT z%MPcJb;k-g67={m(KC{998z_RwRYmr_rrI8qN+m9eybZj9kZ>ChdKIlcN3zSVL2tW z7Qr1=f_2O(&sgp+CSA$mFIunQdOvM(s)-D>c#!?qV4%^{zJq$0QlTIi9-J38%LE4T z&P(7i4^E9&v^kQt5LE4rXrc%Iu@1w8YslN03QlmqpUSHoC1LyTO|JE(Y6$>em_%K% z#2>PUudmka?p~PJSDT+>)?Xp3&<_{wn}^DXOcIvzlmk|-<-RZJ0;7uW%YbilJk@jD z>n6@((O
EJ4Fz$U*?v1Z3|Sj=11|5blhd) z`>~2w{S3)qZs%L-uCpPv?6B>XH1On_W3$dR?*>debE}=L$6eK(Yd_lQbCOydI{L8f z-L#>}acLxo35+1U4f<+A10VC z7Ovt{6$hy`w1+e_J7nT}7;YO%pw;g+LECL;K&x1@-3GSFye6eY$LByf>wkyxGcuCH z1DiPie}OVI4CN*GH~1f+to{g;Uz`8`JC-V@95w&ICHC2S%vzpC0KEG=L8jvL6x(I2 zJ&sEV4ch!E#WG?({Vh98TQ3+ICfn$B(l7+sg9f<zJ=FVW`(0YSyI|Dn>b3SH6@uy1d8JgnwYw;zC>gYIvE%32L2f3GgNybc-&a3knveF@E2})3FY9}>%rCzuosA~SUJDqx z0gCeyxo^rGSkFVJ2ykNaf`@B?*MupE%3IfOlX-DgwsBpppBir3zPLG4O>dM5iTLa> zWy1*yj2bmAW~g9wkiJ@Z;!CPODQ*_$-%_K+R^w#ONXOc8}r?AIo!^!vaF9Lr3T=da}E`2 z4Nc}}RF}D5Kx;OaoJMZvrXx3%SzqSxyb(BC$=Sz<8PpRK=I{{fX>X{O4e;?YW(OEG zk{GqFJ^ZG+#AXYwe2xxq;efzq9;o*$II=cKv}s}g|8CFEaon}fn?Eq&=3FBcpn zIXECYS2T{b;hFQ6gK6DA-(m>9g;_r;r;iP&cogLSde#9+Os=a)McZG)F_N<4UZ zjQD8Z#9f8&3!kGR0KN;Ds1I{}Mn~WXnMWIRLI$>oEQaEp9A@FKIkF<(P^NEbro!~4 zl4yPj$0q9%$y!3z^!W+%`PlK|knRArjBrdhn!9tpuD40lf*b6DlcnJCA8n&r+bGJy zRZu4|*g$4SX#+$;kmr6GXx~luf;y>`j8v2PBTQFz?aZfrT5xWyJ(H=fr^q1p13wQM zu;_RL7#y9M!k~0rjt5Tw*Ga#8%_QZF4A>=uwWoUH;dSdF zNZ!twCr*p|>yZdf44Lmlgu85u7pHFaGKAInYY@&K6CQ#yI`@gJiv0Aad6-!qpCtDK z^wuRi)CODNN}qs{lkbsj2%`yejxrWGok0zz-wW#l8xOC!byZuIz1Z50s0)*g7V{3k zVIYou#GtDk%R_g^)#8a_A$~XFCx|duJPlb*EN0A4JnZYmiNnrf>2HZ#zZeS#(?hR1 zxv>&}jFxQ_-}69sVyWaROv?rgEnu_xjg+UxXHS$5Md?4t2ZZNl)6YyZ@##-Vl`oVy zRh}SkOM#Cjua%#Oo8iVxh=~?$6q}>(oJroS0yN7ee>zuI5{kT^l0CH~<9ep4=U5X2 z8(Leti)VA6H&={hm0O4zf_*~}NMIY_1GA%Zm3l-cW427j`NEB?{Jxx%nTiz+Y*Bh# zcx$eiIVR?vIWBhg%#&m8nHR?=R%>oydYb#%eL{%m>C1E^`*Z(`hUQp}*iI(A5n%&T1EI_MRW3D@l0&aAQ5iCqwCpZm*F>(-oqNT zSz7X8A1qB(7NkoFS3uH78QMZ%a2Q&=7=cHMt$RTt$T&pi>De}svHHup zr1`*+I36jRP5w#Qc8v=x=k(uUk=>}fL~4E5-fVNLfe40mp7b)=bq5ZvQ-QIux$H>n z0yH1V#f5AM!+3DSNc*c#(H8DXyqGec%yzrBWlmGGIsR8lauAFrS&IwHb|zOAx<0UW z#{2QJp-Sf3O#1!o&OWjYJG4lQYfYHUB5OoeBTC64>OGE^h`bzWL#1^+Fw5Zt&T*#7 z7(4EJKF7l2YZ;y6MjKD}OW(L)BNTw^Jkn{2^TG4^_LqfxK~&KlpR=4qlL~T*CZuAw z?~#JCM}1o^hNvY>sc%bw0dGJP4cFKrS4oJ3c`iS4mj_FV6by_051dW*o+Y_Gy)!Sx z(Lc;(=9Mw*7n!DxKg@=LIVee{T~cUF$LV zLMmFhIQ1~Xfwc>09KS98JW)B&dGprcpWQS~j_I;u9hfr^VY@)40Rh-BUlR3Wuow6qp~`#>)0R zQje_@3a^?pbbEYYu|J;Q5FZ^(^LQ8x3kKG@*Su-HBOTgGsYJhf;FqAE+`jZ}5%Vm% z@jeWVPOu+e>t1`j?}725aFe=IR3(ccY+#&*5E+BtQ5H+G?hI**Swp}DUxCj##bmxO{opJ1ILrf%ym|s zPSmm^srpeehY@5dq2!;7X43y+cXTB;sQsbz@d70Dfy>k~itnqGC_1fI8QCD^j)s?pXTr5jXtL zw$#bN-~&6r!NlO&O&7)*5=3*V<^Zv*0k>fW=i_p8xkwnv$)5qlHS+Y2OL&UN(^h#} zB~J-?x=@}L$?WqbfMyusq(1EG0}by2;}u8Zo<=@gKcJqFbZ#J~T~A!^p5t1K`iUr5^KB;< zwxnaNC5%)&zdG)P@kQEMoh?ET>U5ju)or=N)PZlYtk~!aw$;m?Tu#@9&LUKC8tCx0 z&%jO6lwmPUFsc&z3S{XEhj8ppD(6(Kw%NM)NMYiTpxjcEm3UuaQ8dS+5?7e#DR!{yo+Ah$|Gi5Xr%u9bm4-=qz4axtG}7RPCOz6XeEdLD*ik~fzr=Jm9p*UrFM@e}RDi9?B+$944Udn@f4 zuzPo^|2b@yBS|LeU1)yU0URdw6-;Xi)rZaZkwXHJWqIrFSo z>C6=p^y1O7I*)pPAe2s>zagX{u@>4y!3ogEDYJb7w;BTOY{OZSV!nYwyQCeEoDn`w zL1IdgMwqDNyd#AWs;1carT44_fb_6c^hVD ziyZ7cH4)X(hcH59S%S@e>s8V+_e*woE6Sr4wn!H;7t;*_5H_;Sm9N)hB$3VwyzQ%h zTP_*n-44U8?I%c|5R=Z1?4TFT=KRN6~?~l32et4Ve2?eRcA_iNPOa_ z;7zs;*Iv$$7gMUz3{*iUxv2?hjCc~R_-UANV9GIJ@!+582KTP?;1{IMBaT6EJkqO< zM^tcg;e~ssPLC5C)}3m|YzBuiuJw}}LgtyRGQW$xL{ZPwAmI%kEsGRH@L*2csV4n- zdbWX5upJK13AgA8)D~}YHKdnu_G})8bFz*OaP9+#eByTwKAf58C>k3dr=hC}Y3~md z#HL#7iXmt(Mh2ZF`1eaBGu(D;*;;?^kv!L02{5+8_QENbBK&n;x|P>?Z7@f>Mt}=B zbs|DmILe(TFUkOX47{*N1R4(}JavOXmxe4vcv({L{F{x5XL1j|-}~1*SD3?}N3-{p zg&|@uz6UU3#S(XTTg~(P9IsD(Lo_RvmEEm*oKx*4 zeCvwY9yiKOPV8#&%`>4PJ}Yy+)F!ebByu?&Dq!kxa4S*b1SV%q^%Z@P{i^-71F3oT2vv*v7;5?$Bu zmV^wtzQ`5#@jl0&McEtu)_eetzoESmw58!e=}R4_taE!B3ZDNfc)H+}CXdWz$S*OU z<9XE!{IT)YQi4s#|EVP@XCLSh0l5V80S(Al_9HO&Kr~P1OfFk8#+;eS>y`b%{7g=A z74OCULG80S150sMq2}JqGYZuqZmMU3vNn*pn40J9b&O9qBED}t&RXcT`Z(do>6EJ% z_XqLnAlF#=)c#J-!h(w3WowJ)KL6LiI~%833;mKa564P&&N<{P3B1$(t|J*h?)vLJ z%)Gcpw%z``>1HqsTx+>Jn5^ee*PX1#I+@dPB5c;ApXhxv&zg6RKUu)vLX4F8DZ)g_ zYW7=R44M7bqI2D1&%Eed50=j9or)+>7iX$t`8^8?SOkbq{-x5lzpA8b-^XUT$cdsC zYBv#<&{v-iM$Qt%i_c1QKqU$yB`LH5$MWKeAJ5e0Gqc+Sqi^&@D)}Te{^w#rT{|A< zC>8_^*5P1@{j50=W6WK`ah$48ljY%0)g2Io3ia0w%5acLaO!|mD2BvT;!rs%5jdGQ zuX%JtHO17xnJgXiQU|n3f!S=A%@PD>VrxAOv8IOD*$uJ6hWP0qc@3lT{61@J!zptx znxkVb2eYO9<+fOtuCQs3EBG-)ht*U_Jxr0-)NqQ6X?M*XvZ2R{UGu zC~<=}9}@Z$zl4+R>c+x#g^6I{KF;#lI3+d|<;2DXde`rlG|UfF_d>Waf$COzW1_*!r;b-Fp(373U-_^QecbjsEm^K()>L zZv(7hwTY1NWpeE@*@c;ACbjW4gr|!4lU>4Q2;6)eow}o40OkaJZgH)(GkJfESL;Y} zac8n_3kFut1r~4VD6p1o3B0mtG5;$qb!uqsgc_ASZ`E*xe9+Z9*Y!$rzS9@ZUQj2e++NO~C${_MhU3aGL7KTVcLEL_xT2ghlg);X`$IR!+ zgJI;0u8rN6=1WP#HLeem%eG))(U(j~wq#u=?ot9$b2TweT<2KZ)$@1YKu3OZ<(6c; zGjK5GckNB|y0c&U+h4G>tX0>^kd*CcTAI0E#xHZfjITQT))NK4eQV@rGlnD%){RQ& zLe{V|F0fV=C+iNtusLVMN$xlsm&9V2wT57#m=LjF1BZO6E+*FMPj(bT`NRg|J=7Cc zkW)%;qTTPwiJh%V62Z)gkV{ZSmMk2^I{}Lneiuz}CkYg(Po89hr#|th*i{4$wLb_> z)XV~W-}))s1MhuZZa*n|T&_0N5(R~7Q@r3lfl3dsKYacz6sO6>Tua|jW33z%hYfj& z?H<=2oW^2tQR|T#qaJfAYu8%H{dMBhSql#f1Y>C&_0@bPSIb}jG~&c8l@TT)1iY0O zhK6{|t9Pqpz~0ayJR(fU;L%ST*yF8!vGYQ9u+6;`r-g*&FEiH7w(x8ho)j> z?{|ppZDCSdk%)E%pMMi>B9IlVpQ2nx%|N6p6Gp-m6qu+VgC>r9j}K$rbOhr=*Q1F? zOA?o`qAw}pzb9u&e4q_xw3y@=e)H{9`fAeTETg0z*U&X`gnHPk+QE0n z@*}x_J5EZ;mmj!!Li_l|V`Qqu`RZgjB$zzGKJ-4+vCU!A!g+_3m=&35UOiO&*&?S^ z^EgkK``Yb1F8X*__XVhb-;BEpyG+w!cK@q;HF~U=g=9=|>}#N%!uXsF*C_ul2;@ZTgoDdy$Q7 zk*Bpu6{D#_H+m1j2J7rd3T`4P%%KUE5krw>rUHeaSKkzAT%K|$tBdDmW<)$zWqkrV zIqxE8*| z#U8Qr@u~H>+WxF;`_J&8ja%`<=y6_aHoF)(>zM$yoSnj&`K|(a^}c4(#Je4kX0K ztDx0=#lRvQtOEq(^S2U;k$i&Iii1|=Gf>bbf3%P@ zWLVGO_&|+>h)w_#up=wmNw)1v$h0Yn72$2YeRq7qaQ+1O8P2Sggn)OHiN?57hxpTo zbsc941+-r>*LK&Y3*euUWIDOeCmp~Rqhp+|dQ-;-T9I!TvAA`WU!*PLoZTU<5Gc&w zJW}lgjJ2!_Z=MvB<5!~lsC)UFshNI%R~zu--0)U8&ikxZ?-zk{7fZ)6b;WLntBfKp z3wJyVwY%K|8dydH8Zt$eWAuo$)vRjyQdC>x5G@ z=4_sBw|$1Tjkg)BA+P2GIb?{wqHS=S2;{(wPA%a!ktWYt$!EyRix?f(9j+I+@B(Nq z9%}V6ZA9+bpD)5iB%~vGieQ*Jef||_uEossp)zM{86b`0f^CrX$z?aY?`{Q0#!CDd zuxP2bRtzUMB0p@**8aGns|>@1)V@Rq>q$DmU{UfpMF_2(c}rrUsci)M+x_BEL*!O) zNP{0o8KntSk?naGm1u44@*t@#<`VLYlbM|yy*x6RJR4A-9Rknw)Zqv(|KvruA3Ci69ARj-O8k@;aS|O=-x3b26^!a_r;J{ARPcEcW`FkI40`_FY-+z5_fX z%6)M2IClUTWyTAexG%~w{{?%-*6iYr=BDawR`#d}rjOJyFe7%L*^_+Pt6F<1-ft!B ztt#An>kD;f($vzDIp5rQrwpE-(~tAbGdxSkZA{4Ib9Bv4YF#5T)1dKNbfxl~RBYcfa>c`eQu;$T}U zyAH9LW$8`y`mt7C=2#m5v;HzI79)YKL&W?cJdM3HN5(DZA z>AOE!**zz)v;E)Mo_7>yR+nHgL*r0J_6&@T`WK^j^gg<5}aCX3tq>6`CKR&#$3i=G^D%2{02;K6shNEadF4U-E~6M|yYc>Lx=@^o!$ z@y%lu)R&kCdF6J`Oe1Q29%@V{d&idU`UbP!ewc4QXFn`3SJ)2=O+p$%mtQiBa+77z zjcSr>0)TeX9oZ>5&xCmf>9_-8ZBH)o>RjWwmTe(K|7@z{=hr&zx$~t|$qM+-xQNlt z-bG6SJ<3?9MUCeG10vVH)PXA!IYyDTZH~u$Q#P8alJ44e{lv&aloxU?V#xo;+q(cp zRa}q%yV*^$z`_PdAPPv7plGB<5laZvB(MQgFc3*Vd04BMrdAPlK`bGO8(Nm@Dy_BE zueNHnt+xHPv{j3Ung=EUQG@Wsgog%A?W`LiJcIy|{eRBf-8^ilef;`IH}~F|Gp{o< zXU?2+<_y7PV#@tz3%Mpuv7Nbirzu$Sm2_CNWAHRcR0sAjP>T=G*K096W}(WGtY>!2 zm@On_{5w+1)J4%_0Mc2Sq&bT=NSW8D*k_>UHeT{?7SPDGnF5C~k7a|9Is-)n6re8G zKHFloivw`G!(!4~U+d-F#5Z#jg~=&c-XeNE`TdZPzUj0Cr60IEa62EA8*pr+a2i_gs!&9E?By2lyo}ZvFBg z_MFBB4B#75MJ>aPcB6Bp4GH27eo<9KUZ|d)&TVuvxh!!1ciSLW2a-dn5*^~|!1?u| zfk#~nst6_nOQ;|I-DN%C!@)h>&}<8)e%@s1POmrtlK~_qKqiyQ6+zL!i*u!Cozab0 zbiK721r83qWZV5Ya!vywhd;+b7-u(fIL$$Dq7F{<=OjmhQ*>~u4o>#xjEMx}ZWidP z3HIkqK){j4A*A*!rZn4xjC|=OP8@OnI0sdHm6fnVM`xQg(kuxD%Wx}XnMYGn;7D7O z(xY0G-&?gP|L?0Mb&8aqPWkL)UYQdqKV8bV`YIbJe;MT)vr<3`6>F1L#Q5zQMvmP- zHN$S?ICyXlS=TO#VTYQ1Qde5DF~fAQM)|~U!D!1C)|kAI3?=E2VT@AY&=OIjK?8J4 zbriN2;S*Ftq)lB)HMiMq+;&YIlYYZfQmC9AIT+!w(4})AP}b#H;)((1u1I~^XJ=$P zyzz(zz4}HaU6_`>6yoIcF02laIU4Wo`nY0IoOhwAFAKl8Nf;$a+Km)OHCQhw2ve?v z%p=@`zRh!W#r!z$6+NM6s|!tNKob#ifzHs+H6Jjna{egHSGt)o;FlTw*L}{wQW!U9 zU|s?O*w6^B8{+V{UfUg+^`6wscRjUfkbQLKF&W>m~G5*waUwUtq5i5qMgsYD7iG29A=rQ^4do1NU32 z{${s*kA0=T27wd6-z>amDqgsaSpyw#P1Yk8PnQ~=}!O54s>F}qySgN3Hr|jEYukXKO%s0A*eR6 z4_Q+{9PW)6W2Fwe2mO>v*I-abaC;S}`oF;G;=Hsf|J>HhhGm0`Zh(~Lt8Y&M1#?@4 zm+G)>_Z?2irBGS+HyI^3Ds6*tp6#o=n(8gV_akm|yjo;Bvja)*+*ag-Q(;p-WVDn=Bde7$QsukxFvxi7<9I?SckUb)Bb&UcTTwJz1DF`o^905K~$Qb z)~@cZL;f9)?n}5+(g-rdG$t#pma~VIS^k+$-YA?-v$`H$N*8u9DCR6c!kFtwt8-O| zWO};CGu?CVbWZ^+F}}=*D>5>?X{dJGP=2lLGWF9oq?AbI22#Szgfy#pEr{Fz65VY0 zTBmFC3Cmc=>^6k;G5$Cqi1_J)9_Bhi4^(|RKt~-9eQZ?21u!TrRca_;-j0*i z;#$>Jv5=V-%yAGA&Bw=qOW&snnR}Lh9-Kt1;cg|)n7ScR4vAVx&DudBtR1pciyX(Qb zg-}T2Ph`bn0D2prxO!ny&f_g_4m)YE!C(#k0XI7g>~mMU>d;V`Ro)KUp3dsek&fF= zJG&a!6=}HDoPaZ(9>0WM4(cP(bnk%Cf{i>?_Y-R7ZJVD*#;kyRiFeI2&4fl2Ey{1l z0bP6D)PWA&@@dJzOK?Zl*=q%KKRa;g|3uyY$r_yl1nKpn2s%>!iIjzfT-7Ibo43QP zWwiEf1vLLA`jUwgY=+gB!8g=Qc}mil9R|HD=`hn6@A^yHV0K;Z`Usw8()@FEFG?MH znEME1(HoMHmRp`FbwNz1qBt%%=)YX# zAStrH+1UBk-~eV+aeVMe8QcHGhIW=pLl?yaA7g+QJAxPgw@QVlv3bAYyPv-kXlfs` zxdUf5Q+mB-o!yP}volrQI$8KK3FV!44Rdf4P?CTTF!7oCOVy<3lpMy!p?9kZG87705u1u2j(LIA6yBlQd({Bzl zO-?^fKc*Rj;1hAQ#oa-2+k3v%4e)#e7)jWTtTI;l+w3}(Jhu>!zo-}M~N`2 z1+y4yUQgb1&(+1^bOAGsIfWFWl0WZcJ+6^Lylq9d%Z3W!Po_z~MiLypO8sRYbR=qc zqO1C#P{0^ybOiCy$nKX>X}~E|#t@3l+>Xp?HFQoIdg1*!P>MgoP;!D?7WV9^nb}#( zIGz3`+fh8jATQi+%vH8+hW#DC{VHSXJMidNVaM_gO!`%h;P)Y4WEyb(FgS(2mlaZa zTu4-X)g5hitGBI_-b5&k(2jpZWFlnNq%rgy9mnd#MPJ^bbD+4tG5LM?0kPi*a-e}5 zNrS(YvG$K>JWDFn*sKD?9`ZHE*aLYtwwvo#PXgKiR{k2#4ln=zai*2Wb)T|;Wx89? z99vm60?zXCD!Zi}2g^l|R69hh0|Ss0qBk*%xVL0`(& zh;ezJPBeiN1lsRzJbpH1NOmU8nA_sH(14@e-;>4YSsD18U_Wq85#0vObqr-ey_8vt zB@=T*2Js+si>t4upAM7AW_&y&1LU#0Unj+eE_+n-tP{wS`ZrkH8bf9z5 z*}Ub}*wNYaF|#?uBRGO2Ow?H^@})gk5uVRM+2J5p1V?yv^KmxfbG(!xf0$dQ_R-T~+_>E=&Xx(aj zk%t)>$|O#Rjwf->95t#d@=1xrwp@XEnAj(+4 zJtpmuc`I{N#@Qi;!XM4r1vfwzjNFs}o-wmijNDXzHb<()__NtxCtwU9CP&#=nkDawlMp+u!Y&})U@Z=fE7jv!vOh=;j9M;bKWeIW+ReILPDNaD_8sR-P!e61UfVW z4eJc9V_j>1BkAGF6$Haq_0D&yuY&EuIN05S72}!#!o>v}0huc&S;3y!s}C(FnK=U8 z`{kTi3vkyp0XG1!@XQwA#j%PuVVNc7E0ktmeZCPVtfFU-u!$ZgXH;C*aC~|Fl)PY$nFT(daWk+!J2CQ;W*HBY+-i><=)iQm?HF4xME8_JOZ8rmjm24HB%g!J=`Lpex{yK@Z}qwo`dhdFw@q zfXaKOGD`OQCn1?})J?TISO_fsmC4%MN0d+xpBp9AvdakG!h!|DsSGKZW`^auH0%yb z9CU_IWxb)ncjSyq<)e)zxA{(oy_sws;Vt#;k<3c2!pN*HMY)e{s7q`g%abHN7z_8H-L#-tE4FGLA6HWZ1MIqgZ~G(S~>E5|zVrLDobb zB5TTZ7$+&uKlMt7v7IX#q-Ty~u5>K>KZAnYN+?t{GXt`LkKr)0F4yUsmZ*X|kLdLs zD%KB}KyI4`lNyTuKa>YQyiTLd-41DE?tKd^hG#nQm^f1C(vL{qIWK}0@`X0JvIEeF z*>HHoglx+MepC3}!EX(}r}}<<+ceds0Jb%pZvYc#7 z1HWHS$+o=2@3yJg7LlaS_TAATzIe>Tc`}#1*Z?IM-zZeye~-54^Kk7DOAMU3I~7sZ zs1w<7R$Tl|NeFKgsaG*KcFg-2P!U~Vlb31Ch0Y|h@3G3Zhut3AUd%gh;ap+9nj3}G zk~&w>aN&^h)y}KT^`g`f$BM?7JekGFIt2%mS;sm@~+f1j0mjo z4W|)j%3IMemx8#}g1C{$-4?d1(YpnCV8lQ7xW?XCgdAY+Vz3v3FUDN3S1H)D&Q|At z1T2abRFB?U#HemAQXBCl)g@E*!*s5K@+~v0n(`H&m;&Xy9e0UaXK9?w=OhYF)YU9gCC~O|hvsVmv?iCERu%lE9b!eGWSV z!V;L-jsS~eTC3MfHDD+>h^|$f?W~2CsCXSwW~bgDwM)+t5k3l0w`s;`k}RoQQWBBs z7qxP#G!LsE(JAcDQz2LgG!y&Tga7hH1X*08nli4EwU=;<9ddXmk6zhHMmnH#;Bo7hh~&GR}3PoZyLlo9Ahll+>u%1aps9(%$pM< z!=~%Jn7G9kLrYFVATO6OLfT|~Qq41PhQGk>9}u|C>UufGKg+JKM(EV;BR}?CXz_j1 zj&CvKz_1T+1h5a0t8$S2!(n=pFe#!}Ig1$f+WT?%BODjzOk=Jc7x`vZQK4_<8pnz& zi&N%hw!3oLv9I*Cf3b3KA#(BXH6pN^y-MmHF$4WRAM+u@+>RF)nFLxJ&uxmHE1@6x z3IJWG=Tc)_=2=(nNlw$*QL3mPqg1t_!+#~rkOXR0Lp;z%o2`yUufJjlXxxX&})_@&yAbP&?{n+vLs_&G9lgrLJg1X%c(6cT7$6F$n6lfY9#SS&`D#`FFbX7W_kGfO>Xu>V>WGckx zsUh>orn4f}!fU2?KT|&rw=-Ac6tO>PLSGYKx>k`} zHe>v}0yCNU^J|Pd6n?sa7WPEoy@0BBuEcqryMrwxwx8-rsw(;s-{j_xrdeV^tL#jQk9+QNIQT4{w z(H5*vaM9A&sG9#{YGzNZ8EZ+P^_Eaj3M^sIGM0kS1&@DvFRe^k00Kca-cT*fQ24u~ zEJE^BEMJTz2BTR}~LEQiqEBXdl=Tu;j?Gg*cyCo;c?Y)^)Fbg)^d+BvO7q5 zRYi*@6`^d{;u$Qm*{~-j^fsrAq?EL(nn3p1&bKc*-#Q|1!Yb&45FF_BoF7%bPx_jS zBYIdLV;jNBf+pRWr#=T$uuhmZwPs9iEGW?aoZ*)xZG7WcotxgnZISmU{dA@S+Nk3E+#EIz7I2?Qk(^gcmx1ic&j;iAPFJdksi z#3cQ~e00b|J@}BOqj8(=s-9-4@l`stPFA%D5t`|k?RTgY3g+0Nl7Z?^2iIy|G)GAp0|4`@og4`u3VYIjX{N@qqc6P@+*)zDlkkH0*EM zf!DfOJRIj2j@uU~X(>i&>0j8)nU)M(`X(uZbaj>?l&`)^=ZpGtCQB%kxdnvW)(&z@ZjhKo*bRP#Q($jI*ML6-<+=ly5{RDxl(=A*^nc!>I)1&R>=f) zM#>nm9-Wr8;yurjbweS0k%B%A#rpZQYuaE~Bh*3f-c9;&kQ7%}obX(N?n$V_AbD!? zP{)NHnj9`MUuvhtSQghrJ{H%ki^N6|Ak%~DM8>l>%ZQn1^y%&j=Z4 z*)iS`1P+||O`A6nD#h5XXz{*hpxv(t*~n~IrLW#vV}wsuGg&R3VL^bPQ znv5Z9ttc|by*Cbg`c83gyLGHCn`nh{RA9g0_Meby;F!~bj+(&|t-uQNO+OAtd* z?5kB7G4qQMwpFSGBBSo;QlTxyL)HBk$3Sa6vBfd--@f$Owms7<{{F4H_@~3EnvDeJ93vrz57A z+L+@DSyx%5EN1Q zze?pzra9{qW>kSOURZW*9QZBFe!Rmn#$Qf=hr3__CI9P82$+*B&G$}}Ie7!}XD*8j zT{2r`%PWVl&NJf5CQX>k*$e>k)U9SyXrc=#p)mmalU?Rf2_r7cRu}rb*jZF2sQuxO zXbI9d&?%QUnCMJYuRt6kE7}v!3kwsP^oCrk!XE!gnf??h2E?8{H0oCpAUhk5Juom> zio0Q>pNWloD5l25!9j30ziVvCAd}HmNMBX}g^;lLkS)!i4nx)o23F`%T}+T$I)Y2E^AHlHSi)VG?dxw7BMEo#ujuL^E+! zh|#(k>iyKr#A8O1?8GgL%%w@MZE*OWr_2Q|F1o-C0mJ43x1R;>au&D?A`6_;T;RN0 z{X;{^2)o%6@h18-3gpd>UnE&Eu{I-LtYiG_BJ6q>K_gsiSAx!HtxbXiPPE1nA85@yK-195C8Mfq`Y7o?qGwPhYP&e?31E-P> z^#%2jKy`@>P`Mz%nI{8DwL~LEG75}#N-)n^0E>q&Jqwbf17CM_ovNTNH3opB=vh+D zGq<0Z#-&h+ix}APn}phi;xcPX;*Hv8B?g;WC6}l?dn+HB-M;{PME`h+Qw=>LVvEn8 zx2R^^d#@~o!}B|1d{rrdIrj2XZCj^VO5&f93_=QeQllQe1h#bYBw4@MT75>9!{a4d z{Rs)GZ?wEJ95+cmnzLjY=u9(@EW4=uYyGrTsfs;Zq+{>Yy;{l#J_pXIcak&G*k za#PqT+{LPH)Se6u9E}#5)UHOct{12ZP>u_3{=%OkdWaItoP7}27pkRf(|t>Z zGx?V|IXW!D8U|#u49JOa;6UpU*^RIyt6RGY&!KRmPTd4(4Hd{j`A9yYR<1N*ag9)v zYc)lYffpTS4u(~zj!YS2TqP%KI1k3X@VP$|On3NG4|8SHtKGl`7T&E1Jfp#vG(;Rw<`Cwe-1e^J~OZ9^c3ptc) zar&#=jj<3N6KyD$Ht2=8 zH9?%c!AmfkkTYU7iESe{9V|@54I-BddCyb7gHl9>)I-k*LLyoK!<)>yEY_~$-aF|{ z$ZTAZic_Jm2PK$OlogMDrZEr9mM)l;8|8fdAI_Z2bvmKH#)n9FXTj>fA`&icm1m?? zd<@zn@!0l??n*T1<6Z08iO6hR9?MaOg<>R_S>^fxQx$2&HTbLQP%l@KBkH-W376*D z(;BUDp>Jk3Eh|EHe9^e~%Fi$oN>>cnB#)Cv^dl5!)SNuR(ZuSIO9z-hD-k$ny4zRY zflYKrf%={h6{O--dQ6Hks+>X*`+ABg#NBbJ8%50V&?N;b@eLp)Vy@_2w|P=AUt1TG zb3Kj#W}w2QrP9QGuC*hm1R`baRZt@h%WRM)YtABcB@9?K9`aIWxrma270hzqEN4pE z8BVj;f$QKZGbAC@Dm#~m;k>h$BXkZe2tZe(M|_ry_}}^IH6cbBB?+D@GfEOYm+QXs zj@3nT7LH>qvf627j8nbJF||mC|LiI0aP)l!b8Qwi{=^MyR#DisdWSWTl$lYzaAB5?sUg&Jo9G}9U3ay|RH zakB3c)ms!%#C>Cf|>5~8{D?`UC+vFk(*6mxVJP?hiCNZ_gt#@U=E!HhB9 zI*|>v9I9fw8t}I#W%OxVWqK>~L(fslMkB)CGS4Ucjp4Q{?eK@=!f#4z;<%67+sC)Y zZt>RA%~!jNQvzjvzIkI?=BoF75uYY+Ce=L1rDw39%OF1)mvY3O+do%y4^05^0i+l-w&f z=BcDl30hbqS|(XwCA0m1BZzT2Ge4A0m^GJi-Az&8fqgO>Swa>?1iD!T>aza<0npbG zh>KUeg*sBc)nA|mQTeUh45xlYK$m9Ww|t>l2z{g;>Jk2k8E({sws&rycW_6dhrRO+ zsflc7@Z@FewAJFE-hn#ofg&3=xl#2){ITbPLkTE9RmSLdJ@AgK5Ow2fUts^tC9hpN zh`sjCR2G--*MOP(7uVWN@~*c|uC+hq-L?5|F{m}-d_j9)h20+?D76!0TR!lZYs~;( z&=FXH=ea$uK&j)TC}b?2Z@D&qVsrWa#QPH0J+qDnRvhQKD8Uy@=b4+rpS#loE7ErIymd{rCyeB14nqKmkzzS@uBN?{`RrZVZ~5w$w|sWy?Fbc++pLbjiVh?EmHFCxO6J#G*7hFilPXfMjPgqtJV%p`WID29Be9Lp{{_RDbUqY}HpsM}2ETj;7~ zaI7|4JtqE6NtPOLe4BWl4+`9H{!Hh07r#+ExwRmEeXe!gH0%1@81h{=4a*>#uTJN_ zqceAK-3hCqtl~knE=KC`t4^p5CJcLQsUa#YV%K%XlE-1n+|O?}za9Lx^4r4iMSj2I zXH0#l#Xt4o7M_Zy;@Qd*Gp2{yc((C8&ht3WcAgmcJk-InBVdP(wOG(Lv`5fkgr(65 z{D$*O=f_^y$p(Idw7C1b+4@Kbb?ku=&=G|Vc&_XStj_3&bV6!ZC-nc~*e^RM_s7MQ z5k>Xxjt@@8yqd5g^@lxWb+-Co^?G?zzW+f1$j@WLV5h=@w=5@5x=GU z?&D|3Z|XzC!S!&SoIZXynJ2zUAI1ypLkT<+cshAHd1Bx6u!AQyPY>IB_Kk_7=g9AO z{I>9G(0NDtCf5cAy7h(rrnTMrfwlo-hV|?_Q_Wat)cTrT@R@7hV@QaMytL}mc+VXX zg~tZwHje4;nI$x)KnT*Jbmn9{zZ+$877|ptn4o)>Wbxl|O5i5^*PCrS}k4 zx{|Qc(xriy%P1xAhWx$G`z&1RjV#8WigtKt;0XdPB&EDkZc|QR-e2cE>^tc4T!u?3 z^t%0xA^xFIR_N*sPWIP}KG-D#Fn8Jv?@2TLJO*u~?pp7#Ss1FchP3UojcPMRpY=o5 z4b>^QoUMDF)4j*$2T!v?!br$29-qX0)V1*Zndz(J)it~r>FQE>*f*LYhypc#IiHx( zB4bg=szVo}whF(>T58Eylj2r3_Whgm?xszG?y;d55AyK*UJNqy;UDM`t5R61T{9h z6rHFvD~%)YMy{97L+QenYsASaz1STB=>ySpHlXF)%Yq&Ih6Iat^HmWkk7_JG)5de< zCD3J7fgYOC?WAc!1VeW3xwh={sRUb%>*ZuVi+z>WiMza8j0U^}>x|aE>&+OoHY@ zg)#S>G)|dyZi*BWM={pWMXySWTrXcq8WzfwTGtZS@|GR#m(T;%gp7r-yk#Ww&3KE89eLT?< zt35*zv-a0QS3deyRC$Bwwi{$l7!f4Va zZuNW^`mILV`s(0x?91>7CKxp8_(@Gk!l5zKi&L;4_W}RQ}VJdgImgOb+ZubM1>WV_S9*nZz+%Gd_DM9OM61$yBftXk;&a zK8C}e^)deJrIce0_3d`tt#;AEYP*;+`dVd^6PVeiJ&)Q`@Yqy4LL9Q=1V#z>A? z=gcwYt8%OobylR8%vRHX4t1#JT&+&{svPh6yBzho(IniD%vt3-MUU*J6{r|Mg_80%8%+|Hm3+a+U_CgcG4|!}eeyBc? z+RJvoP}MyntVtm*%mnCmbfsA2BeGpXcsG%)=ui=MEo?!_HCt^C~uk1{k=h*t0a=n$Y@VK0r54+Y)p+x)$x+*5>msHoo zSMq|OX++tnbnGVjE)J!055|2S$E0~4d$A*QxHw^PsIM=a;7ta?i)xcd`=oi^KI>=6 z9*&~uY(&r}Idaa?2{}W8Wqg@bmDA|1m$yl6h0nBm`e;vADm`@mXqk_msnjlz{nVuM zT(4KHwE0wuuX0;v-SV-BbA8(#$c_hlV>8`tD>xF7;M!2<4ebjJ=-%Pu_LY?=Sy4Hu z^v$JT=!gVBWPV0OlQ_Ayz#N3gTSX+dTA>!9S~4z)&#I!Rrl>@?AGT! zA!V0ON-Y&D-Kz=1gwN7e3W%fpl)Skrka!}*Bg$2QKF;!j8G3@(^*5xt$_|#f*5)%{ znvSRs_tAN$tlpTWX5Zvi%xNsHwcjRTpnp5c(#<1C5`2#D&d3{jMyhX+!Oc=XoQCjv zVruFThu4-J8uvk!2(K)mHwhZx@V4GFd)j9SS!TA;(V_9PVeKqG;P1cpnc7M{s!iJ<4MR9EF^@h7LI4lU!;Gi2Xb zOG~b$;%yHIlFOPb!J{N@I#lsCY&B&xH3uj1IMCRnf>+9SmAU|^rsj%muC-6>Q8oW*H&> z^mufwO(a3cf@qMz%aO^XtRLpG=NiKvXiPi1lZLC=kyI{m-4wf2N1SGZv}^kd7Id=S z8zzSlUac-!;ySi{@>O<=rL-S{MQnZ#h>0_PJ5E)PZA>RIam_rbCoeJ>phk z*gO^J7eGhk>FVxbTI%eJlZ%>VA${Z?h7c~y8@2QqdNdqeC!+Mt|JS)nJh@huqTuIr zk*Nv&StSJSfT&;auWT->@9Mimn*82o6w)8EKFMMy0(9@v0`(21*1xdS{?qEwB$e1z z%M%q;Pc&RCch_=g%=7h5t%W0^wOIA6)T7=%b3bl!yw1#p4(}kHBs5@MzbXEOjb*3$ zdbc7uj%s}(>g5;j)~jy|2LkX5DXlYC-#l!==GD13UJN;?bH50FNQt6Q9$U;ur91-e zjYgWWJo)y_@$%^z#GMY7yrK_Q%T2d)aB;fm7T;NmCp|C)`;!+w#2}&4GsKvZGCBRK zDFw*>r=$`vC_mf~>W`B<%ufFNfs7M``o@$*RX_s#E8c83?spi=kX)rMz{9Gq!D8H& z;M-~s-=@`15bfFde?XMI1IGVA$UGk1=t`l5c-Fc9z<8b zn+-Qi<0|v=v~JNKhZ%3u24XlhM&!WgNrOAdQ}hnmw#qzp7a9bAoahujLw{i-@Tr-QUHF?C`QY;# zkx$v{PHsekWbvF=Bv17$sF!B-)!9qXd^O{H7}K|uPq7qj<&R=LGi`UK6kYpiwnaVI z%z&_`Rtmrt0f3ofZAcgMH;y(AE0Q;LT_j^vz{M=g;Uo!-T$w|V<`#b~3B{0s@e@v3 zP22FEr+!o>YQopcF1?96XwW5&!DE@E*vdw$QETBC^qF_FZ^@X4YM={hA{_bMw9w~E^z zAi%;O^&1@!B^W7GT#PBam)6DVcJfW@+obg}Fy$I5eY2h%YTBzYrszv;cO?T)V5-Sc zcXD?Tqu6`8$N@h`-K5d>hQq*V|=ILR>Zy%M?S}X zxp+}pWld1>EOr#7NO#`O^pX8kr#aAIwSlQ?s_mD<>LDe)vrxUn-pB0jJDu+qs>wf< z-p3{)ojaS&_p_cX@rXU3^4OUSDMkG`^9xhPJQynjGCLP`8r>W+7+EgD%Ug z^SVe=QMt$cvH>T*SY&H8z9pO{93a=($;maTmRJR330|F$C%qJD0k^kL zp~}?tPeuRVNZZ8XLP>Y_swEyw>Q1fu^8#N}%uzqd_|JDnGQO~6Kc#j&OsN-^ba0)* zLI#KGPq0Af6@JXIWt^V<4w(4XnAU|lS^p|xT`Q6dYfE;M}g#%M`3(LNpmASD=e zY^B`$RXXj!c1cpvjHjSwy1&KzNv4fhTonD}?i@=G)0cIEfj=$*17bSeT@HY$NaK!9 z4rlXf%0gNoOL#X6ll_INbq|fh);enGwH(p^e}k17qGoheCrzsJPM9jA!Qfg%tWw0* zrZ7Ag-)$U|>1JH4Ih0y8h&F2ZfVZVk{aWfAoeYe}=Y5!bfBIImwKOzzxA;88jDay) zDBYM)pjdv`SD7>;`r9!A{Na%nM!Lr5T!Rn_(=qSUkZ>zV|o!m?u+r&hoijoXL&RsTy0@QlUw z5}SXq9c>bq53q}*D*m;{C8J8$y5E5o?)G^{Y+5u;fv$@0lD6!$ z)up{L8m~!dhW$06qlSjaSYS&fRoG*_ zoZlF3Lm*ULM8_It?b{6978#6oZG)2um}~7TWSx1F^>WwRKk$|xcvLDor%25D0*`jc zd-!T=IO#D~blGmMmzo3wU-`2mWsJeR4=f7)?$Y&?9Iz4j!6Lo_0m*|QF%IjIh8r`t zFTZTPby9!tV9w5E)-O*EWOJWtU-*;y5SBTWuGh;p)=jh6>a*$Boq@HIn5!}ER|c0! z)_;$m6H~4BG7}q~^9)`2P<+|MzFzCJoXkDTK3}hU1P&<0s0)6LJWTwTd0W+k_0d72 zX^0sClM|LnrI9VD?r&ypoC*F-!ha^nPFw1*Ew#hR+kYlFzeQKVRq;9iT)h!N%fOR> zYfQ;1Q{}zZcvLzNZx6Owqdx6AarC5*4p;ft%J5o|M3mWAf)Q(O))WL;$4}-sEh+47 zZ|cPNrV|l-W7*7HUuG3)E`#J z5&+vPHPq)#4pK6r#Uq)GEAf1OZoAbEv0L3G9b4cm|48!-*J#UCd@6Y^cde@=Dr&jf zp2y5)+w)fu`=JEe>Wo;cTDhovA$_$7)%&0VwUnbd@Gh>kKahO!RVq#p`b%xZU=?v{ z$c6_D$*e#%Vm$)=UDl+#R2F0!+|K1J08-b%(Ai>BQ?7o1LiT02l#t zcQuFhx(qa-tarZIiRBp-3j=oxiLfdC!Mo8u?G!w!>ophL)7m>?QCd%~+8*w8Nhd3d z<2Te`+ahY9S53<8vMWtihqrc?yVC<2Yt_eq))emVC83mREi5 zxxf08PTU9dhoSrwB5V)nhz-DY7$LiHmf^KY{INjH$vNQOlBeF*t#!&OU-5M* zE?id9ywcn6LagdfzzOaS^$~v1IC&xaC~pJpa|nhZpPV4uqGlPgAFouoJaHE_9V`H{ z5K)sD*859d^p|W5C#A|3x52pKr7c|N!ELne`n)+9okW+^$m$cm+A$}b^mJrdS;cA{ z_Pn%3h1Sy_x4*QJWX*Yodw1Bgc8hxJxO9?J<5Sf9*8|Ol-_L^+aleyw=_1~Xo=3(o z?nFVry(3Spgu3fh$urp3KE&JCJk$2*GA$;-Z~P8%DsjT=czq*@crTv1h)rE!?M*Z) zPyO(9o!bABeAcU4Ch?w&Ybkdl7uWuJowPAgGl~f4DrP~>!(J3c z+)Hlwbo+&->?T*8T1WUgm)^Cn5+U?S;P2`RevaVc1f0S=mZ!G7#zgoDau#u$(I_1F zes%BXx>$NRRbV3R7r{|pVV>Mow?CZpoOEb=@P2UMyTf5AEVA>aPRPhNXWYekY8ZKQ z$0L8k-W{qxPmr%JmBMB=`W?Yik`$%Obx`yy^R5x~lldBY4r*8=Y!RqyHb=GJ{k)8XN?Obj6<5VycqvrxQ$h}1ZR}EU;Ts((EX{0B%}GZ zwoL+=*|vNhkvQ71bfuU;0)}6q8qRptHcFRv!zlxhYlD!L zg3Rh=pYzLvvl|cF$GY~l$+T$X@=MTH7N}AvjX5HQni)MS*#zwM*?yg`>F-@@lcx)z zeQ2K0FQ}EQSe<7;lhsR1-3Vd*DKcOffuPI{**l#U ziFo*!`~_N)m0NrVx#v@M+T6JX88J8VkHe#J^OPR-`~hGj_qGOA*F&XY@x-46eR=r})* z7$ocQ>JBpYr^A5UR-k*K8H(;JX+&>ZsE$CzxEMrj6|9N)rbzna_+04|;UF%RN>|P*^-5Pn{Hb zhwS=1e~i#>HQ_s;Y@a8kP~El(ykxqa%Myz(k^(h@>8Frd27Nh)mPGUudR2%X60R6k zAB zylC9px2V1^({*eo^5vgeNUW$N%3EcR(7j^fT4n7iRxQ06<>wj4*8QxVPc=(2;9ME68JL9a`fF4l_<0+@{=IMJKuwQtZBn0Kzv4o_MDYj=gmUTa;DO&40d zo?5FXwCr`6dT+>_3k@h9@VbnV#X9;=fwfX(o?gVlNzcoAO?P`PEk7fR)yfMa@mx$a zn*&hMBE5xySz+5-SUecrTb}Te+45TJ@@$-MR28S#YV&5R-yRXkYx=P$E(aG?ux`?X zvkW_PBy7>hk~=i?Vy-Sh)+<=+I%$Ouf-{1aZZ{8!gi#zO70E$;x%Qg5sp^(g&ed6Q$%$^z84rzd_f#bzn5s`>fF7XS3b zs$fF-LCM*ul2N7h=gN3@5W`L}YX86p|5xJARLqDPsEI z6zc%HDj{T!7(RHu+iCicL6{f_T)TX1lWMHu_t6L2noc)uYdY3xt?+YTHj5EOp#QBP zBlra0D<_EZz^Lyu2=HDa78I7U1C1VgQz%s3bSU(?I=Y$hH0%YjLhv5-;X(}OaXmEo zon=hyS%#WI%A2FcBq%=q1ALx>i~4u*`AgxzOng2-je6p9EFoXUXOIQT#AjL$e0E_o zk5rpw@eTA}D%hOK_sWT1#^$9_Z0;Ld+>&eYR5mq-wy0l_`#*=zPl4=KiaNIG{~10n z?fCcc`G03R@p%FYinE3yVR_{UbVCz5~e3gES5|2oAULy>IOQ2&%U=eXfDq z4>UD}LvO1qi2CP-+{&Jrso40z)i24c_uhHH?U+3q<0|$0{sJ=HGh@1E0)Yk7Jt+rbCzR&W zhY5{UVtWNtaZ<5Etzj|6+A}m=CoE1Faj-`iY<4fAT<6H(D!+{)6!? zwf%SL?dKpYGQJ1B^yTsW+KbWgeS?t8o@1O@-d5^aq$j&KyXi9;ccgQ??g1fSK^qbI%iE+Hnp_-cH%_Q2-t zPyZ}FQ+k%AE+yrEgwGe?a{t}&IhV8%d@iFNJ@Gk$klyL<*)P%G8-Uz=fu_HeLVsuQ z-3$Fa3;q4WAG;9sk6EabKz7SiS@`Z`p(2%jjj*1TE+Rx%TBej|b|+()Lb%zbBb!cl z4(>+?-HcR$-fxhKZ!n>-FTLEHMx)KSaY1N@`g|ibzo_S$Y|gHYJ!V&D=GXMhTwO~B z(FXhn^T+=m^!NC#{(gYk^hDiQLd^cYsV6OJI;#77B&9}KC%$$Klo9(c*t`CM1l{djQ+{*~d)F)-(aqk~gS{&v zyr{h^mL0v>yS5_nIhVa_B_n`E?;Q57ai%%#Xw#Hd+q))E2f0$To4xCS*gOKY&1*U@ zoKn4Mgn$7OwRP1iI?__CRlP%1^KUwZdet|xDjo__3oIBASn$xxy{e1TE3m=B%7Wi> zx&O9rK-gUF--wmver@HNt9m?Sxumh32!~5XHy!$5TdzP;b!mC=pwQ~cHpB-d7sp!S5LuYDef%8x zhiW1K_W@8?01i~CX7dt*?;ujq*cFV#j$tqeyLeWZH$pVRbPb26Qbe@$fht9|Pg_&J z#M|}Y*VgD2|6iEzGfB{WzAx2tMJ&vj>oo8+t+(I-n_?J zep;;On+~n!xB&BbOg%H4a`sSzn__R(gPdhs?6YR{E>3Npr3o~)CduP4d%vMY1D zYbx1zQl^=K-Vjx)CO1bu$`eNJY1T3uV-w^fH;qw(TqF?{s*9L&bS`(f*PxAP0W>Z}vu_3aTc zMqLL5Y5H^ZGo7|na=S@j=5Dz$Bted=9aFiuN+stQ|46Y5RNwk#C-HS7+78kcC_{Jq z?l+H|?dHf?oE$mJ_nYBzym6ge z&g427lZ6(+)3@mmfwTQ1Lo-jrlzURjzA^SdY#S@G9NsywaEE&DIgYiscbEsSLSs(G zh(-w8%)26r+v_y>(wQnCP z*LhOP+@b+d5D{%6-!$er#VFg z;6fXZX?(0Jj1k|brb}%dRh&}vyeL^7o`jM`+yNTL7_^L+?`aiLVZNz5Iv+C4l8br* z{m~F?5!)~F_BCbsKb#{f7f}g~`?e?(OrH}NU@T`8VfUNQL})1(hw}CR^ca;)F6_)C zzEJ*!ULg=;#?+sq=m^!2szBQ`zl?gB%dHDkKO$s!>*E>Esyospd_OX5|I7wsdXazT zZn2({o8YQh6I5+Rr9K zz6KdJZLTV-7j^L2WR?Ci+RPL>k+n3&d(nwlk;PlND`qON99{)7`T7WGtOn)Uvqrzw zPhK2_^)EP^5Y@3objf`(MX|Q9nX`D!ChlDnQBNR@%NJKhM8VL8E^9tEd4% z9~$MVV0R{heM9`#w{n7xy0|J(`C0IL{NQeFESB#IYA5WN5dGk+T z*tNbQwj_D8^Zheh)pX(M#qf7ARl;WL2|*ghV6|Gi^gBMgv`gU|BYG0f&R`gp^gB`D zK%6RV?)GONLtW(CiDi(#|C78%ILO)ELERnWwx#tKz+%g=Sb?(5Y%fZ%Zo zG*}~;cB}k+QPm{mhnG5|M~g6;>pEKGBw0{8K#mp-Tq5JY1HRki4SU``AzP0X7J z%LKUP4hm?Kn=NKg(WohPhqfLL4yP$F)5*;EG+hf*naIV8W=(z!?x=<VyMsPUNc#sYKHu(<(t{Q=rOGxC`a+ z3`XBopf|0YXb^)??zT5iG{Bn6G^Aq+XweY0kJ)7E@?W0`hH8UnAy~z!`e0O^`rvWt zLnlNBRPM?*lDV9Tser9lY2Vf7m7mEf42PEuERLU_S+#mtDW{p=y*T@k6$91UXs%@V zJ2J#%tEg<;K`R$I45N;|%G_5r5RBlm`1ZwtrD6Zfw%9ZY@T$j8o+{=Fd#<53Y9But zE_wt+aVK5ve;D)pulhxnt+?lO@No#2OGN zzYE@!uQt9iNIq4NEd zq2<*>>x5@##v*&^pw;$_3zj?8^k~Y^w&M7C8M*e=eM>(r|3>(ptVdRUq~_8Dp=jzI zD#zS+t^F$l2OWH#`Xm~%ONVrD9w!>2eH$zi@9+Co-_1ZhRd(EFS#di`wkbQN% zB&uKg-s&_VS9lAamTapkj(>d7$+OlQ8KueA8yU*MFTQ{!c(3$xX5qHwhXL|c#uFa} zl>-okEU%?IeU&5YrX|QKjtxzq{JUV#I&FCP#>CJoCaZPv40B=gwNCF^&3$)l!H#ef zmr-$sqhu>1@O))~ctwo1KAO(5RUhux5G-Ufz}1oYj~56=eU-2J7d9B&;kw(mGtA!X z2cwB!)^D;{Qn`MRtbsA|xDQ^h-mZ z)zYupJ0P$4b~?J9!Nk`{$+EmkC==6M=%=|7H<{5mZ{A^D(-$6}9DH-@8KS$5GqCK& zTg4XN2beDih3%H^heM_37$fej43OIkI1Bi=tsb|}I|lFN4qvO~Qfh}M`hvn?d)4g% z?A>8&8@v}Twn(}q?A;yqHmK9Y*xCvQ@3na=i<1X0tS>+GA>7`u4;dUENUDWf8Hw8q zizgXVWLyp2j=jj65<#djMaCy_I`j_`TF*eYQ4w%EQxM5bf1dBEMaeSlF*mmIqG#um z43U6)Q=U368nT`c!@cQotGhv%;DK{@2i%RAe4Q~TwD$&%QZvNjSK;@#ea+>Dd&`$W z(*WhJ9}?@7JK7^D2N`6^MKp84XmOwU$9@?~P|y4%I=uAA>O9pjnu`F1G15G}x)0kB zF;_9q?Yfx-5mz@om@QTncSm`ZrL5{QA@V56ZI5zqP;Yk!*5F-tg(YB5sExPup^oz?gH7^+kyg4+E(&|~hZ5a& zPuT3)CMiFr7Tlywn)`L0{I=$rtQj|GjgTYX1>s#{hzF_nER*f4FkToyg?F)}!zwHa zI49V)%~p3Wphw#00|j&6kyj{Sj(~F-`x6cogm(dYIjkGi(HRh|!#L;09k2&4Kzc?s z6RDNa$7X52tu zs~u~Yc}pVxM?AYUaEh;`4L1@ur6zyKO&cHHm72g|TO32CLI6CIJ_xRTZ`B1&!&41s zYWWdHT3NZAE-#hy^*5y^`rfmIE|0{T;f-eaUP~xZ08Y{u;ZpCWrUj|V@{4TR<*1b0 z&OKJKe+UhK1&X6T+vKzVEAS_svEpw+>wU2^DJymPZXoF~*Ugo_Q!(CTvPg0~BGT?F zs#o;YUKQ130Qi2><%66^HshPgm5^H318hThy9WXK@*^VQ+lhltt7oEQV==SYzVh-V zu0n2EyJEZNBGYe*_-uJ0!7|S~XtUH)?dQ+9O6*uHWytFaPXPOltKwH?okvts82OE^ z*M;6aHbv)qPuo#f^${tXOWXG0GdHwbibuA1&%q_GGk@i|7kT1-p1hwS?4Hxe7(14@ zmf%D8wrwP8TH^ZfT@rHh+u_fE2@>`WVV@I*v~f){!oW47$b>|P-zNZ}-SWK%4cENa z5088%a#E-zAtK`ZK%$P;Y)1HKQSv9+CayUXvnj+)slL+=kE7xN7j5v4X&Rf(6A@w^ z;zr}T zKWm!tR}sPYMx9g!-d+hI_gL^G&=>ZlG#NMBlTV z+^a;6;wq>U8H1dsHEIfcio*g2hFC1qT@`PdkB802H+UfCBm8$syvb!GXHr}bJj1xa z^LUJ_qD)%xayxg?rnP(U^}}hhr>X+ZnUosHS&QDbaoYImoR`J`Uq$`2q;96#mUG4^@Mmcs0#En?b3iUOMizPm` z_X;Uk7OITj_?)=jA!;?mtau>e&t0LXfs#w6a~#nMIeM$mDzmsgWY(wm;tEZOzCiTt z9b&v$p?*O0J9mX#xoyu1^_XjcZlaS#4HyOu5(vt z4N%;3otcvm9Z0;rye6TyTFo)b97v7>doMHHgct5FgL(g5ILS3Qc%$($Cg_3)#P`X*% zd1Q88@5K!;A0`UnDpY#s#l!ln~Hz59Y?z+pt zKOhEQ7s6TxLRi3dE^!m=K46IMh4Dy&7WHVIU5lyIR-lYPDtNv~{mmTklz3U>16WERXaK z(E)cs#g77Uw0DTT8e-NlAda28S}y`+Kre~By}agl?|FXTEcJ8p{JeJv zu~8uX2_R1N4v}Nl>m(2-&s{HsK-8-}GY7d)d334HY41JD!x~`CDY87(JH*>2#AzT- z_YUz}4KeEs5NFO^t+hZIxgD7~9UiYa4Litg(3jZ<@2%kA3$ps6*Vr6l&k}pK*Vt6( z7+$e+kjM7UGI(JejH<=|A8X$NA60ejJ(-y#Lo&%3AYg!iQG!x|77b|P5KID@M3gWg zFiGU4El$&sdJ)b5D&*nGXeP&L?QQkyty&e^(pFpA>jP0BLm&@8d1y5eqTwO#@gRnW zNqCs~{%h}@z!`*7TrW~w zg;VHwibBCD1_kR=NV}>{v*v1%U}*z`^*Ry*EVwSs9fVV;BE?lW#V>e@Lcu8p1#1=u z>l*37YWw6fx)U2 zm4Z>^4#Fswh!j_06y#+Yv?UnDpkR&QV0Bw_yZhJ^7)M3-z;JyD1u$YsgdT)b>=r4m z!YMZL6orCQ3<}ow?$09|TV+@eOEWi?6V^FY8 zLV7?ggjjG7speyRU%BWAG$2T|9kb>hi-i2xz;Hdo%d0q!GKsYe0GBLMgxSjgDdzJO zg@VNl3i)IX)=6vb$w;sUkxzmbc|QyJWI(XmMT)DinRj`LLcwMR1?y>~Qf*&ZbH9oN zYY_RQTvRILlL5i9i4<31Gid@W!Da>ptM>wdb=sPHIufiw(-Qe1`2bU||rS{H0)P_Palm1^s< z=EBkTpD>FrktMHW+o}y6jl|e!6f?=s@`@)+0MI;6XS@*OcF@V0x zy64X##Z~xkMd}UCumT|BG*4#iOSc9y4ZV{DUW!+;DDXzj-PJheCf#55Hg4K*v zs_koQ?$?oE4YKa}JukB2D(jwyMT)ENl@bA#;46cIbu$O+j5YU6A8~=tR1C84F^F2P zvhX>2p7--r7|RZxqEIlFLE(B1sZ`sy*4%F+(KX1rXN9QrD(jv(B86D@3^;%%3b2Iu zGALMj4%P*0?uAIO23hy~`y9u`Rn|TKBT`(2ul$CmC=`5UP_P~l46 z5|(wv7~)qhoB|mL!k!zHvbdyZQYQqCdYH#SV`}6YQ?eGIJt66>hlE~sz!dvM3Za)B zFa=q`N^jj6*2^YdDI%%<0C}2GVHDtoV>@UsV_)f$3 zFAnDD{_yV{ou@fR_vF4D-N6GnIupLXZpzVpyEjL7JHA(b1n1ZIUW)IH`*U<{ALi(G zH0J1<_T=cy_|C=mM0`)hxBG(}-QQ5hZ+7SC{(?F_Lmk8M&A0E8VS7dfitNZU0Cfpn zt$?Na>U&}F!*3-@g0-I=uJo9032=x&m=q_d)ggQ0Qe4M?ZN712!KGF|%lpBM`#$Lk zVSY1F;A4bcVc(@T@LNS-Fs7`7)$$az0TB3mOz>t=1h<2*UDk_ZA+UAKGr9D&d6#RK zeAk%Q3zs1y*iY9Hk>xb$ZbG@WOD;F&eTU6!=3CfR@n>Qg5 z!Osf$L4q{cZjH#ugKA_g?-veZ-~qhDW70IY#Eyv?c9WKYB|O6O>Q^X15a~=?;w0z zKmR_)amCuN77dK?sD6z9ZUBrQ8UW+l5jHr+j{*eR@E#E3oooJ|FkTXY?GE%H&Rz6j z`_Tw&|HuCl+sg(<_l^*{Q~S~VbO_zTQe6DfX}w}Ja6a~wP3ed07N?oJcS8M$+ZO8L zU^0HoUU`+mqU2jdsQW908CMM}Q7`$An{ams9u~vr+L&c97n2jiIkJ(%eRfd;rTs#U zOW(UA*xOrSxS*T~rtepF8mocVsA{=GopTxGkH1BeN5KaMUd?#bYt75xcym#MFSmCY z{eH)PFXI;=gRK4^?Az5KH*7bIU9!Q0|q8+ z65YetXm{8QgI!L;@wr?TcfS1VRbT{%yy9U&@hQ5!tW290TxD_vVW@y1Wud{;30sJ- z_z4~ug`l8=a=4%^=-En`x7YX54Nyx_*W4Z&MIl3rK8yZ)i2}%?A3$P+tGeoppRN97 zhkJsu&SHSMPIgjOYV>bM$z!svqeoKK_4G){(q%svCC9*}`%WXA!b9-iSN`-7_=o@J zbQmh*qBz1C`vb;3+1^Ptc^FozV=+u^>hx_Y%@x!c@iTdp81@Hl>|GZ9@6H{-VD)%euh)tSEu z2h>o&^6Jdrh66}XsTEge`oaOU?$L6qGk+Zppdz)Kt26&I96)A0aBr0fmAI${d3Bmp zo%sd@!1M5OfTatrS?)3g50!Z1KZYA*J5U+5LW_QIW1%yuNKn%^vM92o5oAk7wY%mmcG~`W}&2E z6ZaB847>JB%7!rXuuiR50VA97C9HWa03~HLKL=eVmR5 z;;|j5X|k0Q5|uw~Aufsb>C;gkw}G~mtNVdyQy8x^DS??Cy#)!#dKX5vwiVzFrk4qw zcn_{w!Pd<|Y&ZvY@OA7(K+^yK1o{%FSKy138Tkvl2iRgeP$w0{eufMf;pKOHhVee< z$2#3cLR}d-X+7=;%<(UdE{zjb2T-ngMedd`=qtIGwUZ)U-aDAcWsh8$69>CJ+o8QeD|4 z>;)p^F!++^cfga%qv?i+OK?)^>RqJf!N9M>dO`B-rs!MfaK<9*Ik}`x2WRI+n@sLR zM{q9WxV%(32u9h%}5*BwN0?5Re6_dytmgsdx&_(SAr`>5S0IzT%S zXcGA{{74BJBp*Kx)=h9u=luhJb2EeTu*`9CqO{#+G*2817a)j`fmiqvKH*KAfaAd+I=$P(OU%~d={zy6Swr!r z@8RP8Nnu=QqeW^Y@Q&k`0HDAa)wso5YRL~|p&cN?kX05-hrLAu$aB+FbSKS|MMX;1 z4D>SW=2-?tCeER7hh;l99fnu;!zIPFV_!Q0${#Uf(>(y>D zCZ`j~#jsb!$^iDNmvP9e8fxF*9jo(b>c0?No#jh<&U$dwt@GLMF_eW#BOGyxoIc|p z>^KjdN}2O0bAdN26?JS(!4ED=F<@wfOcc4KkOU*4WV%SDL>Z6srIscVoRJj*jSO#r zoUG01EpQiEHS>D+P`(KKUA;|h-h4g6wCOlMKwzeK;V{>=%7$djH$EF2uwRdxk~_78 z^!oHenR&^$z0jMNd@!KJ--+6djj@aK0cVT*OkE5B%*mXa>^7nDj2T1ZX*lwFle8U8 zVi&1NL3ubKhPT%iH1MYeyLZ`uub{yeAW2*lszafHDfk4;z&HvrDAxnCfS6pfg()Sm z0A~5>^b@$T4c9@FGXP><&!DX{vgD|I2kXLhwze{I?~3btHD7*|3n$~sz!PEgnl%x< z3RWDH_lP2jfhl>WgZYuEYGziMuQW=&o^F<&hz#CX&b6~0I5L(j7yu@M0;S1lAPkek zRRY7oe(Z!WIf34F@D80`-xQqkUOd?0~B((A3}-K`{@#APcz z*Q`22mkz1M-A53_W!&mhBy02=-E$Jgg+qjIGV+yb9!7jIYJLt+0z;~i@P>d|3IHex zAyA|ml0&O|2_D=9huHFUFW7YDNtmR|w*c7iJqqcENw;z{CBVv8SN;{^ik%zJenx}( z00#B;OK2rZ9#j4r3UKfQ?B)xcy--%FM+B}zU^bwdv1Ymm_&EYxhVmoG2g^4ZC9D1^ zRKdZXNu7=&b}uglM>QqSW-8@G7{cuYp@S{!>8phV-5bg+sZqM}P6CV_ z5HM*auJeMI@{=`p(kwF@mr3lcGtR56Nibn%cV?!{lZWI~UpS8zullms=P@F*cF85U zQl@x2bKDQ6Puxsg6-9m?JVO^41~^T#PTV79!S$lf%oul1=-F9} zkvFN^ez4c(fLH6_2s$K)rlH2$?94p{yrQ)X{xWZ(AqsnT%uL}_pjJ3t&4N`gL@dE@|y_I;qQe3OdPJN$bZedn3FvS9=4*^dm~JcfG$a83{!EAOn! z>MS#$K^aob*NF8Qvwhh{_3kH(KI0Q=vNx|2Yo15dkC}p*^`B9=Mim3DwgM?D%28}+ zO+8V`nn?4!d~G>BS=vK1_HdF(PUJ@`f+Hvxcj1vvbFP^=X`0T18d={3*HlO?<-F2stwAeUdgi*%&ewG&eyI(>j8sKOI98=_sX}1O4z#<12D)^ zshIEKWis>uYDpL`x3CoY&O0BP`TNipO~Dc9`*s<==XR_vktk)G6scp(}2`&`Ysh=vxD*allJk=>x!koD>$3j1;^g}^8+4N(l zpL^+NrO#W1_T1$@fNAkI^;5rQlqJmozXFJj^4)2<%iX0MljIaceTCD=FyBU~aSlR{ z8|0YUsGPgp7fTm4bRSk4ER#N*uYRZ;6XA2*7yVj)4wQ3GfbdAPHGv;=%jx#C5yVAC&K3;pNHWo1qb$o`3&??)c{JHdoZ>meO9oOIIen5?R9BQ{K5f z@+_1zq4SQQlvNM+IP&xEIVEc)Jw{~NV+=Yu5rKHbBG90NZ{;io@Ra>gUjCxR zgaS64k6l0A6`S7TbDt*;rN@bz7~QG-Ty!|~G1&G(n1$ffbDxY;{yDNp zB8m;C+7OE|GL`4OGGZJm)2!FKNaWsfdl7Ga8 zRbpt+z10uv)l0(d0d#Q|h>b(o_24?Lk($MqA74Ap+PpRi*T*2_`eoh>cFqFT)Cr)J zn1LQQVuBZ`4St3jYXL3yJL7&B;!eHWjTz0_EF0jb-37P*zNPx&`K+FDl5-`ne-kXT z1J5C#$~whuY>b&=Q#at8I3TW#flS-moYrAnS8I#%+4Qdaah~@bFm;l`heI6toxiuG z>S#xme`T$yG1~woMPMFEO0Taz4%}45H$V$KXuyVWu`d6F- zQ5tScuaEcRjH>%aN^H|BjRrN>sK!meUYahx*$pSj;ItZiA@uYDk6LV_O@A{b`oIR) z1hlGagwG#|4hB|{W=_@B>1v};1TO$=d+iYAcAfP2PNI|hlXUMuIHM(v7x#w*D~f^2 z`Nr{iN#hSf)DU+JOFLTFFTR5qHMPuX@JnykD?OQO^ajj0V7;c=7+j2`vrmDTx*7Z4C88DBOdsbazpI=B89IMut50U02+ukue-dukIwB;X_$uCJN`z!V}uww>k1wIJOy0t^tE^O~%bfu^ZE$KmZCr<9d(T`F;1a|^sr2UCv z`A8$FBJep;^M(|ioC>e+5Sk8QW7H16L77v4TSJ4L9RuS*Nl?J#VcmO9$(Z7fX*5jn zt&Z{>su)pci{gaR89|iLj%VbhQ1+dMPK;5yi#Qfn9W@&A%;hs)ghvbqYXOIW-U+ki z8egG4b3>ABXpG5I7bdH@=BkXT@)S_(oQGq&Kc$XxEnj5)aJ@<0`CRuU9mZ;0u!Jb=9+*3MCA1OHzYowP_He`r&u5@Yrfm-8)a3r95d;_C z+!GM1k{r<5n{g1{f$hAHio}uqZG43Q_JS>qdlu7G01Q`a;gGy03+HrkY!-r%OFIGA zKh`s}->iP1Z1{*UI;j~rJ`5bw$#u9snguv<%N~#PhoTJo4Sx7T`j&vFV%0fsEAei#YpVvG=> zB(dLPWVuEz@+PwiD3JrtffOLqLaC9lBv~Gfk|DqkR~IIM9B$|2;CD6|r%t6cIgx?j zsM(O?DlNFlgprcA3}eYwrs)lx7{PNW^c93+$_M@Qvn|eA02jE(4OCZ*+SC2HX6kOW z_N>>dJj_9@6Zo;PMwO8*! zuO+iOsD2^2uL$`Rw6G^d_$m=zEIJQ5kciF$O;EQ%^M|1Uw4mRLh|rtt?D)AW`U@i$ zj$^35sIz_!!4uQwQnAci#I`Mr=&Y$fAianN90-w}4QpZ1fZ^?JOyLBXtU=J|3nvzas5>ul zTJ3B8_62>-7n{?h`|jJE#vPe%Gx!_ZkKI3#0ycEk#i8mE>J#CK@AhUnHkSh3-`44K zp@f=-TdwYx<7ebixE=tWaj0kKRNX#Rz8K17s?*;d<&K(D7n5tNr4(IDi7_k#XTSo7 z%ml>XA%Md2k&wCIu!WYEMRa+FV9D|*?!{s@P)3twUXM45X4)L1Hk4mx^iDVemuP^? zMU-sfYX1JFip^0oJAs{03{Gi#jGeu36xiyyUo+caBw}ZKFm65dW$rUPvI>#VgRFBK zFnz(mp)T=?aRmiiVqG&di)i4vr5Smbzu!DCx3<$qlF5T(&Lh#-d05gwsv&2 zXwnSf9tw0(w%uo(iJjJ6Uvbi2x(s(#m&1P#ZcT=A3r?!oot#M#8`oofPXE{}%)0yF z!7BLH0;~qa1=rrk=^a;1E4>p}om@(_i5onyRr)1*Ug?RJ9wTCv-i~qoWSML6BB!-M zj&|ZM-gVO3Hhtfh3tvY1x({*kVxKKWdfVj08nno1-78z1*8MUZYUo9vTT4IW%(TeU z5kk)?p|q(3rlk}y&P==BO;{0ojxdk`s372n9O#}5=E6fzhms0XFtJ}_YNrnFWb+V3 ztd;I8(bANR>$!)$G3+*k6G@rjFd~C3LwejSge=~uWx}aX^{I8B{(csKdB=nvib#)MkTmSR@ovg=5;8!27-rI?qVpsZ5^-+`Ap zl}GiryT?jz8z8k&J0CU5|F*M{$Ogf$OkwwG(d->Shk4QKsl|Bpd47jqdkL<4*Hu9~ z;(3w=F)oVpoJL2L-%Ztew>^bGb`;E&SFZRx6d>*$@pV>nF(#$vgVxos0Fa9c9By+%)yw<6VB zk*2Noc}@UXsDz@Szd8j>5oRwyNDRfz|czw{$Bv}qcA`dy^hIYvYiO5LE(Zae50)URzQor%rh&8!?|oHqC6<1;>S8s*i?Wdi|ZqWAxhM z)FyADx6q=^2z$BQ&3(6 zM^tetktNVq@Bj_KE2afZ7#k@aXNA?LCW4vtGb>Dq!4eomfEC`1t8kkkArJx(ZkR-Oma{RxFu?{X!6WN1Ov0_xDQvms}HY)vGZY! z1*g+%ACe7Af*s#q*$qY4?+b4RwV-y$c@3^>ST}ZLs5e~EW$*t8cLi?0#;B`z;C8Il z#5&j_pUVY$bvA=z!mn`CxhQAU6(&wPGB$c zxa$$89fN)kJAm?V?^0uz-y(0oIeohFE{5)*vzh6N@ufzkS&zHJ|6)b~?wigSAs!#j1ugXN2*Qg{=Vx%4Q9SquSLIOf=tnsV@#EC z&0cYF3A0sVy}uJEp3h%f{xDXuQSR}OKT5r60J+D!Gr}qV-Kg>w{ zm(#U}XxZy|mH)oMe~FM8gY74i3MpOXY{v-d+Ggx$%cGPF zy>gOzkWg~$TK};agSd!Xi{8pnlABg3d8&~&BC(&YFYk%=AC37w{?e-8LzI?0Pg6t4 z?97I*I=Gd<03~pmaoahbUJng0uV*uUYoGpx|8Bu=4RV~O;`sR_Zn?OxpflQ#k^`;rO@8G3f8Xn5Af?Csul5ElLTdd14ZC zp?gz&gvcOpeFYW;jyZNiSSP#&W*{qSR#>WRQ*)I8AD%}OcWQTbyn}KqH)5My?X6<)y13KKf(S%lQG=mR6qL5yO*aocDF`$e&9`n zKH#G?aRiP30E}KZSp=R&4567C75EiCsOU+eXddXP7hsBtB=1V9K;R&-oEJ-3BZ}oE z&EzHBDd$yPESEDx2~`(YKsj=8t>l@E*y_DxaubLJE~_$DxXsxJAhj!bxKs980;l;i zHE~>^9nuxL#p|2db*{W|D@+STqaM?QXJU8lMq8xK;+`}P=`XK!mnL=Yp0#P@ezgt z%~ctgTPQTh=ZoN*Tkyf&L4MUouwu@si^{3>P_UPW)L>vONBltfsu7MHfy?9>1AW!A zT>ACT%n6y3zrp;lbCgwO7Vspucc^CAL7Uxg^rhX4`jHb+LNeCB)y z{aLA8grjaaXS=PBK)V|(R9a0jUozk}>*-bCoJ}031ghGI1(cM2=uTP!;bK-jboA#Y zH6H8<%s>u>p+hzUC^U7Y!eXVG8&M6lIgMh&qd-(5PnF)z$JKp0q!;o(WYi;QsGZ%w zmrA71y0*zf+>ff(hS%j^t_{u5l#T)gx&pgNanwJ|kTG3dX~ddmva|z8APSa~o}09@ zzafCQ4wL%s1C=m+JsRs^9XRVz7I*+5O3zB^*T+$jRI?wySZPa7mhtePOTXeb zmiPQZUas^!C{Iv&{zLL)A+Cux0yXMt4nMpIm`D3WmuPeQP@q;`W@90H5$o229#l@5 zT@9&N$$DcoY>D+ZJn#r1Z*R6JL=+FUOYN${1Z&0<15&E_6(Ce4rO6~%0B&}zyd3yD z0^LKaF83z;W) z0DcU2rBr{3X6fKIdmE+KK{qA_GfX7cMWih&tJ+3W%g@~@5Vax_$yX+tIYshJ1XDpz zpnbN`=%9n`qAAosA_HwvWF%CS^T+BJtI?|P&$ z7BzZ40uWPcJ^#j|iO<{Ua9v<2mC$6wU9NnlqOKj<)sQX6NIUW`Yl%ernu!A9fG{`( zRWEAM>b%QpFNLL8dn=RI8?7I#9)@j)-tNx@@B()9_fzOpz1k~=GW^v=!%j;}QjT|T z>8Cd9feJkw2>Roj+%euaX~y%p=-2+0|W*cgg~O&_OMc-Ji{VjpAWtN5pSU#LNH zFBh8Ci^_$}iWr0MH3W$YYZD7q&XTmvDxXAsdEfyyAp-z9v|qjzR1;k_6WaO0Ag=(* zUhO)VV1cF3D8nKo_D*ck$kEwgA>U4F6uFgcG~B_?9tYkD6Vp*1UgaSI&r`I~T@6v; zuSpv2P}a*sMI1OPwGyd8nly(_+D2?b<)M>$x*E*xaaVBi5?3nI16Yv)!eH>tO~~Nm zGE9+}CBT!Ye<9Z+A%yL&29rD%7dI2ZMNB5pKeM+{Gf9-&h@qfRs$AZH+hQJ8KPV8$ zl-`a~{Cc}92DUO@A0Zj#zr@SGg)gAN^W1l2@*w%P5IB+bAYbrJkwHdvD(@nIrSig% z`5A2@l+i*A)5Z_fAP2jh1ByJ<08h)h@pKXO4^i_a5Q+APWGWZt(5WNr?14K-iY*iD zAskIJXr1MI(w--|3mxFE+V(tR`TYrTs_$96^F1tD4YAK=A_df`EOQnuKt`fcHU>jXTjH@K*SOh>o~;(8z(%;WRNs>eBZfZ98{Z0B(E3JLiKq} zUzp^TJ7CHP-_l=(WPYSgMmmcc16#+geAI4HDslH|9Q1V%_oJe;LgfY>8QXXcS7c)g z#10tlD#39&J0oYqv?;R79K;2`8Bh`{GjDPahtJGA7`JHTah}7Kdli@sMq?+<8^hNc z%7nC&`7rVUtS`TWEv)otvfYtm`BHpI!;;_xWA!Y;$jh(x#Uf5N1`jN02#&BIk~j3_ zO#?-n`l4Tph^DKS!v&c8Qf%%^A>%ITkOG|7!tZ7dj_ARPn5&BcO%ZH8YB1nET@O!q zISDOD?-G#XqHsCEL1*8B2b7*i+)*|!Pl%nZuBF)ZBBFqm?=A@a!4XNQT%DfESK8$r z2r3DFEQ%>VMPC~UBYi>Vc@UwYFG%+XN<~fY3o`Q{YHnZ9L>@%#>kG=@L2pC`-OYox zL;1YkwH)Kpk0wcn|V-kWKc5?+7}tr!GjJ&2A$_YA4LRhY{f^p>7_{I zq|w*LBN3^}XY-(r$e??9(22;PDjvilgMP<@PDKW7=0Tm2LA!WRPh`+XJm`F6&gD2Bq>KQ)JLI9%POTn$3eOkwK+AhzKxT{KjN_l$TR@IH3P!5a8hs&+F7*-d+Q;> z&}C$8BD)Q&P`FYbeHu)HJo%TXy^}7~Mb#RmoEjKH&vBny22b_zwIf*^CB9xe2)px7 zFwD~Hv30&if_ZOMCYVBY4&9)BKFrxn%8YMo8^Jx?!yOD3bzJRYat&pN;E%T~IE_aJ zN2~yds1(MNlvwR8R&j}se2s!zt>mFl?(af;F zf=f<;qFY5mrO{N~%cGCMAQMM=GU(h%(kIBAMJG}q*s)taL?gzrT0DfI@??Ui4Cmj{ z4`-%)M^LyIyOLQVBw%sZio}Ly(L4dAI#`UB3nQ~p;HYIdJP^*1T+Gj6IG7!}QSe#s zG+vMfHPb&ckTiU#2kUp@QmpagV6Xw@SP3uT#_#BXDvmd3Bz9wFDox3}(BQ4?A{?a9 zoY2?2w)FaCIFy4<0GAI=+f7ghN30BYH!;A;FyzK24GtS=KP77dXx2-hb0gCJK#N_R_loRlqAhn1KC7I${82<6M** zn(Kd}1%unsgO%DM{;)Q#8OEmc$4iLUvegae`3yS@#oOl>u{!`X$tIoJgy34EeZIY@ zJPP7@@a+^J(T5{Tth^kU_S*SI%Pb=R*b3c@iw2YFI9ZZ9gt=ef0 zVi1y;gB^qzugxUXeKdG+egw7HS*#{{L8_s%j3w-^@GZs_oo-`#P?Oky^KVjiI&o|) z7@Y(vKs8(~x6SkLGim(t$RW-`w_%4uaSqMYu&?rHYh7mu{WlaV6YjrVXB!& z)2=|GcWE+It<{BQJ(RnkYJb6~BAPpa<9Xj+0tsBV;j+uH)a&lBPG61# zG8>BdF&x#n8AzH9<)I>5>}(1xen~58#JQytN^jEoABB^lZQyRi1b9er+Gcj|bfo5A zO4??fO?xFgxL_m#T^(})h6a^`KpK301i?{~-o>{$!4}zNP%_!zVLRD9Av4+zl&l%+ zV9#TaRX37}Rhe}<4>VvK12**T-6~9~oktw!&JVw;D4pxdX zM1pWh-D|?7e1Or>5Z1A^?gX6rhGafT&@8@({c4oBo!e zG)rV@+b#^&yd8a|fti2`MzHI^vqEFVz(8PRny+TV(7J!&#Mlv1SXLZT=+-oX~2xH7dzOJtAs)rD;-5Kfp5NKv6SX%jJQ zKn4zW54JDM8d$P`Ft0BcyYWljHkN_bh@EuI0wpX51D%tjou3#~f>G`vUEDe-${nu+ z_3o6e?AcNBNNjbCL{e82c105?Pnoq*PAOqg^bV?h9#ji-#)*wD@C`iuF0RKwRh)N1wb(EHo4T;oA*bROUigD+Osw=4&N8|&%!p+W zzrcV6ouZSTrA|sYK%JzX^SWs`I(twmrYt&=*kR40(J1-e0;)noZ3{-^Jv1WSbGl(M zRs{x;9MRG=F7(@eVqy*Aqg20KfHs~3#T$S@%#h>&SpP@H!uW)68I7$(69 zmgHS(@{aQ^O;HLh!7v$ccF@Qc?I(7?QSd&sfI1qt5u_iwZXs^0A~uX6{VawwnYTvS zs%ECj7janaK6%QFRr2*S*2>qgVc`RzY1Md7kBQ+5C_h%yTIzzsL%D3U{S)9HZ?r zb_S~VS0O@^`AtCOeMP2fKULTQ*{-le*;*h^B3MQ#EJpG~V}^t#q#lHj30+V>(-o zbV_dZm%$yFAaBA)YHsy@9y2zSQbJ191Mep|a@YzZ6P)CWZxtBveA`QNG%j(FYF;2% zLfCd09ObV<2xgyP%oAI#mprvd>Pm%uaVb(|#K@O$`c3kzLMZ=oMVD_v4hQ=X9FJ$g z$ZU*8<*e)$z!EZ?r9(Fn z|7bEPj0;coiSo z=H4&f;4+al%HG}$< zdPenwM}0Gc#0tc=AU+s8iJ=N4xpec({{*xYvARW3q|u?Nctx6Neo;_POJZ+Q&jFB~ zCelPeM`FTUI)W{^xo^z9H~3IHOZ>gj>~Xv*9oLrmJ7NstP826RQq~~;xMENG!(w%PxvBUd9Q+_hZbE*`aXO!NG%JE8C5hog4nWqZ6_ct7B27}J+q_D`@iO)Y z5-fL_k?U4&EoNt9C-V>>dkE|N2^FB3k$m-Vc8tERCljyTpY!4ju92lQlVN7sLNn82 zFeA89e0UUDAUNMN_11SM6+&zo-niM-K|zXM5|>~Kn8oV zuYkb3SUQ&~)lS9J8cm{=q@>{;D*aYS70U5z{l{bYjgd#xBraam%^q#sgfp|f7~Rzv zFu}?-*RU8-a5^uT5E3|pXz~m5A)cG`l4mcTxFiWZ{3#f~bs5?r6qKf~KhRT~HypEB zfjqLn9j`WaAAlQ!!^+8CvRcXO)Ru>Wnh%~i;;{EE3PJK4kURy+lU;_049i?`Wk_Cock&Gc|2^&0O*j`)CO@PK zyO6Hgql{JZ&wP0u_TaQp!4b*O+raP{ve(^!Oz*b2u#`M?@HDiy=N-5Y6&HAsbQqkL z92}8?w7?zSvLRX8**zG5Ww^=}7zu0vV}KdE&x{jJr?d>@4&{WIy-Qa_Me#D8a?e+&bry-jF4lzbgdb|Ce{2BZ;;x|I4)O} z8Rs^djha5~QZ3Q}1ju=~$<>?zuNPLoq$Gkpx)LC$*Tv2lE!7-HzufMZhu}0ue&Bh0 zsCg#}tUaq!Z~(q4BhH=RiYrx`^tCa8zag?VQSGs5>o_j*?XDg{e8O&pQUV1Q_P|Xvq;5hX0Wy@##@#YT!&z-H5ko)q709=d zX9u($;mSq*`k|pn5juf~8el*vvWU!{;+6hF?mwI6y za7k?nJP2Y_=={;cs=HHddJbC2o$3b%dwfax){Cnbx)Pf5EbvUATZRoTCz!mPPBz!^ zPHzk>&#~jBjfK-NIEWdEcl4MuGYY0^38nUBA_WdnHPSv!c$`$-oopn7byU-%eAydV zj_wDo!(UyeEAAd==1}zV02JTKDZYa!ewm>75LjxD^mZ)4CDq)32&dIg+w3tK?qKbJ z_b-tCL|$%*o(fI`gxI-7g1{GOlhd0>jcC#bev2aRb$fiQQEeF>a`4c?Sv*E!5r?V<+_WmoN-h4xb@TH8upJ=$}9n zNJ=!>6}{Bjh&l21vDTl6e)Tucp zTG!w>C426Muw1qt3S+LFFIL-1puD;WW;N+E9H%g*+n}W%JI{SiOP+M5&N!}?4#3DU z^GI_~SIp{bVHUbj%aYzw?8qrm{cv5S zS68jW1cx1ae8i~#szL)JzE{!g+80@SMI`o`SEVefuIM@PKDh`}XF`$`Nn@1?++v)d zR?zK!;h;v#UG78bV>By7lY~cpvvNXWzBq_M$G#q~p*reiXO?L1#W_f^SiN_CI(fHu*TUE~|s^ zF(&g_Y9#@B77}9cEsA1wNLW_piYfKI_!7S1=+jq`Qe7!fI981X{5aL7b4zx%jEYZZ zr91}&N2L-t9SM^Tq%l((FjFfcIuIoeoZ}-(t*b(G#P^)_5g&CNMSY3U?Q_uW4~M%wllL)|6RSMTyZvF&?IOs} zyS*j%WL%9^6dSr8@#taYg(R50aLM94Ql?IVv?Sc05bK<9Io zG`#%YAIU+^%Ui_b4e{8_AIK>-KwLUDI~!n3SV78t>hzvo_pqjD`lMQ5f)B}2o2>MF zcV%!I1&_e_IhaG{b@pe#(Z?$`FG3p^{OC^!*C#-xUY(3{kfX+dmp%$my^oq`G5jv= zg}g%=2j8RSzy0lRJIFuYq)TF)NaqY$^1z^XH5!tcWzcXyZ`t52n(ANsAD z`4KFihk8RCf`r6Sj{zHT)i;CXbcLKwYX`MP8 zBd~`?;K%QP*>qvBF#F#T(-Rtk^H$0!nQK?b*QjgDLxb-EwlZk&1(2fw-Vo0p#WzeY zkOJAQ(@*RJ_nU@s6Y@eCLxV(i1R9p5kCGqB$t2`|-VUG#yBj$>6`WqUj2~2k! z8Azj${2HK~iaN0xXRo~u7+bwdI@k;_Np*EOi6Dn)yn+QPA6R-%RwCY1+JupVoCDE684d*pj_EQzBGrrN8ml= zPBX5E`A1W!c^%%;asj;oA6?liv6_aqQP&ks91JUfWB5Qb_ID6$K+-JZ#swI#P|j^E zrKdt9wy~UEKOl#M7os;m1Kd^j z(lFZq5*tSpQ~P**OFp*CzTilP!4_aQxrg(wEYS;gW+dR@GX>oc0{#lr-wfp%VD?bQ zHfZB<1j0S8`jW_8%$i0an0+ypBr&dD(c^r76Z`b-u$Ch=a&QB+(7m%;v3()7&=R4H z!3Vt52L1wgv3ITRhw1&?6PeF4z$4W#U@S+epNgO?8vCnGR$pRRY)CbY_^Q4{3lFH> z@h-jHfc=+LunXc7n-D?{u3=3;9E5@%Ll9?VK3g)BTk`=qogsmK>#%n_A%0fo61HYu z)rQl^d0M`%t{@60d~p@uL3k?M1+W5k zJq8%u2zei;@~}an8N3y6gyyRt#u#`Hi8Hf^MboC%i%83te^3hr{(2>FZzfYY<_Gdl zOr;feRS%7kyc4KaZI-9k<$#09lccwX5|6v4`XY7u>fv=ybRd~&_Q+9KH0DHE53KnD z+zxgc?hml?&R6Q;O8kTpG_KlGjD|Pa=n#yK{l|p}kU_S-LNg}yr%s`$WlmM^o3D3z z;kZXX59gQJoq!pgwG5ntpYX4}20bYbDJ#N*Hf2K7!5?@wfB26y+#t$?J)8H`F1gT{cM*Enxb&9VB|VLK7j!u&h5Irm?}|mjSt0`L z>jB!2zw6j-5qbk~x0<8RBVPr;UEnv%+1PYawxx zhp;gSTE`-H;VOh{BYj5N)rZjqdkI5|JGyflmqEWxju(&v`)IdeK?ea64KiX+^LY3i z*7`lcAa{UGids65PUSxX${`x=LTa1ui7FKB%$zEZw3ol}SDVhI4<&mHiXQA;R5wwF zI5TgPN0yXtiA;bs3?(3qtvEED7^)4Ic$0oT{c!mnWW|}LQ6m9196;|f<7l)K)xu3o z09~MS{kRNEUiTFEu5o52%M)CZ-S-65=fY0n&p4cDljLb87`;#6I&Ph|%e%x}{u%&r zu*+bTLYjyoi-60{-v3J&)8k!}OP$tMc!tFsg2Y(D z(`4d?p^HA4XAYE*{4KmAY7!~dA+v(;g;Ts^Q0nrY0bVfIjT7SIHP{*+UIMJBJ?H+Y zq+%7Jl4WRUd$;;jClJuj9kEK|{yngR4Sc6~JxLHryxT1JWqbaFdAs%vG~MeX8*^Xn z7XF^f-^nep*EbpObj%y`E&GD`xw^C^@M~ zp81G$Dl=!ff8xxv>D@n4-y{giH=%Pds2yxFPs(@dZbVXu$v8fMeS>Em?3al0dQyGj%BTvKt$2(OGs@XNN^Cya#YR^0t4rb1r$vX1}GZ6dI;e@ z50(tDwSMc*IvpL0=3hWcG24a^vBg&d%FqR75_r5iax9H=vqYXG4L~j9Axz_3xkZ=i zV2|RAA;%-^?0EzLHU~$LwC&~JIMNPmZ6h4zmpIA^yby4A@;dp**pJW>=xqT>dVlCU zJQVEYP3ki?sz2rrh~X_wKkS6--5XWj2atKkX4~OSTOd2|r!Z*4IJIi4RS)$7od7ne zoWRhRLtVz;n}pXAwh1~#U`u_0dQeaSgd$MDO}r#v8N|^LcmW+#%vK>R+fK|*K=&w5 zgH2L*bbz`p%)EB&7DEK*I^nV>bBU$mdbnk?_$rOT+d@2Q8D;_AiSBrt)oodSoUanZ z2&3b<`F_B5Q>ZVc_!7a$$j5%nAU-9~_qC0_@7xYBYR z{7*-f)7W8$4-;9Dgm;7#mgqk6)jjM#E1)3is6*0E8k) zohOK@Xp5d)9m!+MN&kUl?3=Gbb6Q+e^h!+mR3vKn5K0}Y0R_;~6w`nQz?HCNogC?1 zwv~GKjF7-io%UmO z!x7cpfl8aHSq!45Rc5PkV+aKl&r5)y_Ilx_@>f!M5ty4Hu?=UanQ+4HEPyH))Fm0+W5Onvu( z>)_$-X81W#>dm|^6d0V^SCtP{QRJ@FtS3?7`sWmQ>~YUg=cPdP0iDKBq401I39e0H z-z5SGYJM_!hy`Nu)IthQW+y2ae05ohZx);fCVS^uy!lB>g~ukyzdYA~1->?LflrIz zX(&^aHZR3HFG+hny`_l$v1fLQfl_^HA4ulP`GM~i0=3?4(=apv>9v?`EDnea3j{$1 zeuSr?UY*^P6rj6x$r(j9k`nQOSYMby*|67l2?5~scf&y?f2+{Xl03!Y)woRZkdK7& zXZf`JU!Z%hCj%J;A1BSW+C1rlgBQHM*$9NX00m-@KBcCfW{ zhHLCYBixLy+g`y4vG&MAgB(Vs!CZZb`l;eW^-Zb|GcTtL2Rr&Dl}c?^c@w-Is)5f6 zW?plmeU(iK(xI(M6o(CqOvr^2Yq$jEq7C^aUxAS_UrL{wIRz4iU`L{IgkgN)Q!PmK z`lj-iv=Epdpl2!IX&#?%wCY8HQqB9QO7-zz5%~h;RCqxqHA=nH=pC)Tz=0Jl<^|<= z?=h+zu_2g++Y25CZNvI!R1dGuN8Ly+kUTq34^~}ksaFtYcU6`rgD;R9HK~J;gmstL zf>#^gAwJ{0?#Vn#nXU9xuKK#17RbTrX6Qdc1+W`7ph2R7U1qXb$tEvV^T3ZE6Tu-E zc!N}S&eIc<_xsyo9E(k4>nAla05Ahd`Jj zys&LNg23>_Pw&@8?gk_Q@02`gVRK+!4eDI~^AGJAsOBJv5o!AX+gWJ8ElL4aP6 zzgI+vdieTH2e48CXE21tQtfB&k#H2p%5II)ZLESUjn!?)=1eGzl@pz9E$u03?Tgb) zG?;ZQ4Nhs;>@;{-9+qW4ZiCC-VI%J4&jmeaB}<1>T76)$Q`%jR7{^7t>@$e#80w=Y z^rDl`N@X)V^`HY}oXlSg^-gKxG|z#*+jLvMsl=YhK_y^`-&1`7wN**VVbn@;tZr*1 zT#3RD@gApi$xqdo&PgYccUZzE)Q`G9N8l$npvopBTQGs3`+n6ao{qODZP{f0LRFVe z^K@<{a2yk;9cQJ|PUPN)@;*d)A6|>{8W1}yTGX=mDBiO+^Rm$#g_;HFgcqr6;AJOy z=!Ht&ZP`mN3wEEC7BiH55v8^wsULsue&=tKN9 zB4{jCkHC*9@FS!@iL_04N12PyI;EuGgiatYi+&{K8tBHWkJ0;-&O0!#@7(%QHZXf& z%E}o!UHk#)eK>AUm@ ztD}I{(5!S+Do3lT{Z&YW)E%e}MU{@t<_$0V1XYILgLrQ+)QkV9>e*~mh3E7tk#;|j ze-PE~LA7O{pcTQbRX_tuL+GxHXh|t=?@K)SgDpt4AkB>5CUp6kDahF}8^vWkg?Oa> z$|;q#;kg@VuE%>PQg+~p_tIWGLHK1D7Aw1{Iux-|qnC5|J5ITE%=@1%i)4CkhSjz4_o94Hba1^u}CV3Je1h_b#p?37O9c@AZ-0cC%(Q{4m4xyV+1 zQP9jr6vf9&RuC0Qo+mNz`k(`P-Z|e_>VgCT?COK4k4hgl_5g>zzjR*GJ5lhWY-p}N zN`|(|Ma!zC7{u{Z>R3BZFL+1b1~4Q9uIFrJIsI0Eb*%m*6)!z%^91m?$}EnA$Nn2o zNl$!6Ll5nL;@@})JdQBVBGA+ITRLZPdaXC0#4A`7`ePRgJcg1#qLMpMf;=Cn4l%9; z(Vn%9%!8U@j|+%J>OZuK!5ZdBXN{-X)w{adLi@dwaD z7-6eB0n4xi;yrjSNHd<5mhRa)206I5XP$xj-191e+D;R}gpgO#0@#nBJJA83w7^9$ z5>#Ihowp6|;1>G~z7plc^bm-aKF?tWM~HLA_=ptfPr zL}*(_hpM@q5^f!ful_N%cETz+eAcZU2fpGN2N7l>3pQg+gdN!gi< zhf(>;M;$HGm3Bf?H@>!IaX2DFstV5`O=`Pa$9v7toj8n3f(q1PaBd zKeq=ELZ~Q(WUyc_p5SXGU5Gn@xMP?cF2l%=T%MeN-JMq_`^y(@LmlCk} z+m8?uNHD8WdAsWc*=yq0pbeD9OQ904Qc@%2)&7#U;Ohq^ z5s@-JeEGy&kG9j4?C|q^ff_1-U;~ZHBPemJ7~|0$cn0oAGzHQJK&cNb-~lU-&IVPS z>>pJF3Pg{6(*@BRWmm0At!@#d^dQNaol^z^lk<_?9~p$CRs9vwE^6=_(s%apGY`y_ zgMqv7<9js?CF??ury4yg5!f5|wcFa((sl-3+(p1GU$=erSqtNAkbUOw^4y1HQL?d( zeRig6XSQ44_Fdf9vc9b&@Gp!aX2TGPhqccwU*irPS>N?<6n0emM_>~o12e%p+gsaP z4^PpZu<1kv`i8czk>6@u!1SYnfrn`P)1XSTNM3ofrH6j9S9oiS|t$_sO#e;w={XPg$ZYd1u6-5~X zpP-H3Z-#+@`$srKLtA~@(Zf^50qD=#zTdK^5qY`4wz{VYqirp%NBhFsj<$7D5tm!r ze*byLi|)~F^+ytJL17ez0ud%B@KY`gqypAyJ!CKQ*-C zc9h`w{}0u)wTOD!z6e#1fY*7&UmW=E!0x);sJN~p0*V7C4xEVF^+Q;gA4tP;vbBMc z=TfbUI+f-yda#DW+i($4@bK&O+vWKMezS4w1s@f$A+m}(5-k&9wzUkD7%q5dOvU25 z3H}oGnSa1SQ~%$y3$nWxbyjVovUQc=vg2Anfiw<7T}tcaT^-nSCx$22c`%(bPg9RUR*~s7nc&jHGhY_Xg<@RtizhUJ+H@Ey}?wemkn*dG_~~83N6Wr z6ptLm$n9wB1TXNneb?UB0O`^Q=@P?S*;DDn%p23TA}yu?J5V?^9Fk&Nm9$5{M?z+eAA%bf{~(I-E<^j9|E);?a^<1FX5}SJ(Nqwlk|KVpd^$a z+NCemhQN?UGqz)ABxD?cQXt)SR>BE++fgu|&S2);B=_x^Zi{<;8*4pCzR#l{k=olp zsnHV;=Ia6spi($h3(JRO$<5NQD=&!r13kKk#<#W* zC)V9dki%Vyv;uekLO1cOWo4}g)n;C`dw73qXor3@CXbP!(jG~o5EC`F0PvvjcHFdr z+dim<_EFsEnQN{l!pab??DH;Xf@8pJ#uA*q9{Mfe$4za9$JL(cxyYZi^-qA_VcjYH z`g!nY9Ld8k?Bb+f8~EpW#JcqNOxh{c(E1yPhMMqDShQ605AjVVsr*}^Z@ExmLnP05 z%nwbtSOGyzc}L!;fL~DD%k8$b<7sYR($Nk}q1N^#tUbowzN9mFkhX_C0o2Go?CHgp zARHrKvRVF$e+75uKPE${B7&!51iuoN`7dLrtLcBuG!}Mh^H**sIfR!Z5t+hW%lj4mi|zt$bbUSSj%VLO@86`m4x#ONo$ceemyajPa&=GP?>YRv zg1^7v?``~9@z;RAQTR*1-+ug^#os0T8PRmkg^C^9F)P77V!8#Ia`w`ED%nJKL+aeb zEnKtncFXu?KH8ktX7>X2`y#O=QZa>YT^vofE{;c*Vx9z7+IHUsdfsKx9&!}3B2gS} zt^XOZv*?UUILe18-0CNVFxZ$Y-euD6VChFuE%+L&iFa|XM+|M4qrI)Ty_K1HA@EwE zjf0axcYq3j6wp_Ih3N~Z$`oN3)31Mq{m?=>-B-SDQ7(4IO>E1xQMx4|`G+~B1E>UolV~Q#QF^cR z+FN_8S8eV6`mpr@(of9;11Qyi)ItFbiuQ~{F)HREnE8Hd@0pN@+V=l_eOj8K0Ds3LV<^Kz5*7v9b74rv48)eBP0>r(O; zIDH9v*)Cu3$4o{yoE!~MNJ`+_&H(qyH>E6EP*&w$;$Gb9uthpV5O=dkI+WhWT;W1r z5L%=&!xscdkZ2_7a;HsI12h#4MXYM{%S$rs^PQ!&DjO7ucA;TfDs)@{h&O6{3LDDo+KZU zE|;;`Q-Ol_W%%TL2?xkgUy*?uEQB2-(F){5Y}R^0e&uIzA?=zhahPV6FPNO?I@HM? zdzx98)nbk7fwK@7m9|48Yrx~A#;-4iLo~U_K7BsJK>Mlpxzg`&#>>a%oJi^^z~G%L zzpMxqjg;-lO!u1n1>$2EVK|Qi9fRLIC)YY=7rs%~qfPnfRD}5-y@DSktr9#LRZH9a z7c)NCpGsNLAp`foy)tmdNOdCL@P>H+p$0_N?z&5IpUUYC&Ug6x9IMtKOLBymu3)jlUC_0*(`YrfvN5H-y z`bC-pD3!v49unFuz3hX?_Z)B|o^hZAk#gsXE&+(&6ZWlR9gJ-l)ChhDiHL7KZ518$O$59 zEtmQ$>c|rO*?PWUrM4~bjO_i9eoJGs{uGenY!_%)M;x&5YTYs{1nQko1!7ofk!PgT z$TPCaTAPOE@P1!pPaoJyk08@bCHyN$Sv&Qig_;qX8d}24&y{e#lyIGt&=EM7GWPi< z=4J7$RglX$Ga5;H*elIsJG5wR-5hY&;&L&NKNZOMV1SUlbSW z*XDJr4h61UV09b-M-E7yL*B1prrzp!maAy(SNhy?0gvvs^-B_}qJ7&US6WBHloz;C zExA&g9396i=mpiUC%%k*n2^9s{G8ns%WKq6C1I0F(wG#wi5Goo`P-js@d9w@(e(^6 z#v`XR^3;1f8FKkjN6e%3@+03Ky_X+wDr<%7H{jHz0hwF-5qauBM+5}`F++6#0ZM;= z^2RcpGqro--8n7Zi}-W$w=}24S7L3c1>nei9l(mgXpz1C9;mXj=$KUXi2Q_?M%GqW zSPO_o^h-%io3?O;+3vRt?>DiN;+;_99c6W4k~}JPw(q}?zId84q>+!vPxyS^q}`9m zPdJ4)Y5ODcgXdD(immFS+t>JWo&__s#RSoFElkRv@HNH1m~enW(u~k2CMEg0YW>ZU?evXlBF+9K3Wf zN<5whL+~rD-%D|A)}sT-t%irdYAi+k_-^2Zu1lG8bPg{=t!K=TdNM4}K*rFs(qF7+ z{D<^c8Q(Jerd0RU2DnZ#vHh90u&2V>@F>hAusL8HgEF0M7KIe+VqoFVr9$LGAu{uqLQG&pBy;xc0&0ACT$gKE79jEF}#baXnR2-%tidGly^3y@hez7{&^{uLL0>R zdy|Vzm+Ay&IM_KMTE*9B=c3yG4U!>$ecV*#S2UuCuc8_ked389PrSHx4}S+6 z?kGpBRxt9;F&m_vrfO48s~v;NkjuLdQKcvTx?qDexD0cj0^by6jVD0?RoaZtR%vqh zVg#6;k%(CGcR9btlD`2&qFvk_&9GpBCn2DEEe*mHg{}ot-2$YjYj)Lg0r3_JKwV#f zae)Q_``?q%F^UeU8k9W{3dwN|jW-p}nn6Kq`-d71`3iN6d*{iZJumR{ep?q9RA0a( zv5RrCz361>R}}wu2FL%rjB1>5Zx3}v8l;xFrW)PQ-Fya}1*hYEc{rfMC0#*KMpX~L z@jxpYPJt~7vDBDzTc*LGQD7kRw-;DX{h49X1&i*74`IlBg%{q7#PC8uLCy*eO*Mb6 zZjq@<*T64WvlO&8SstXeH6(frtN5uY&B+q|de%FuI<(sXs1{!zGA`P;#eAURs ziqP;02qjYPzZO(YSX^)l19Akd$?pvi@e}U~VGgwuZN3a<7!Am;+n}k&-Rp@vGwuD4 z3u3aZR`6-@q++6Di$pEO@WpEIDIe@X@2sZadBG{4%dfBxEwVg+TRf;#6DZ)9@TO1u z1suC{w+B z#q7vbfT6Rfp65!>@lo^X6A;!_EqUF+;6vyHRaC54#Ljs^Fep{zwBSNyFGfFF$||Ck zRg?u^O)*qok+UVC^w>s^l>jjB9vj|Mw@8n#L#ss!wzotH(hx#Ch!~-(6f7izJt)~U z8A9fia6!w)--XYQPAwIK9Ti7D0ce7(WX;ay~TM7mJv4-k+9Qhu+i&D0Y%;JyR;p#C%L7%tM(@&Mrm) z9Qhop?N`u2br@15e*@mMmbpX&fs@gdR#Okf???$73#^s@BWEwsI{I@UY461xWuv-X zM;vODUTAas68)WT;j0EkKiSEaIwSKO>xP?Xsk;gTap7SLtUTFGvn#<8 z0x32f7bTQ4TOjV6^7gR-5-V*f=3fKB44@kfuID$*zq`!5`_$XHYqa5)3+Mp|v#9 zA8Eq8GisneyImUo%u4mUTz-S1>LqqJPZ}`&pKUI)es8CtH2vypgAFRlGxBRbBM`!E z>_?sfg2B5jV9D5B{*j-qq9~Wi$mPcl{ynj>hM2B4p~pe8wwb4VmXf)c}|@jFE*+ zsrhMLLVq)2?t(RD*=bgH0wGaKVonqE_g-t<|6`_)_s9O;OYkPuJJ)-!zjq!UCwkEm zkz+*v&d^#~)l?jZ->nJ3`Dhkj;k&@>izxjz|KoyaHvi*-X|_-)DRIALsAQM>lFYu) zG>KZywpN}$@U9bYTNg`sol$?GJga5&{>P=`w#%t!9RVCS zXJEIi+?(9qAV7{xzskogu>_fR`cTTyVbkc-}ikicl6!q9S?JeNR4b|*1ti; zH3GRc28AK1c2!ui81bL=BvN&#tpgtz8_#uw(nM>;WtFpJuj%ksne+cEeyv{ z3mY8I^6LZ7OLs!QU?s}y_Me(8vhmjtGq*7#oZvr|x(5N(qGIb?s_nX! zZrufZ6{?iNzwig8;UF~B?@{J1o5N^NJZ8&0*mQiQTT66ZY?*j(9FvI32mw~D^*rYC zl$P)ijl?pocOKQtJVY0Ci;As3ppI@Hkh*k3tjnavuxU0p$T!Ffb;Dxbr?nXkogng{ zcHnP|h*)vom*Nx|H4ikYEV|f6$B2K>zF+l$7^2H#)ECe`)%(Z$`y=SOs~!)k(!1-x2-Xt6q5-^sBn*4C{=59#_hsZMD`1^r6@3OxBf)7yXQ2FzETDl!_x-o;t1G3vji zXn=sb-H~|1Kj>#muk=$jrhHIiE>Vpcz~{-neBH-?Jg_Uz6c|r|8u4<^YNpRQzCkc2 zoIcQhGQxP(bt-sI`RxA*yx&ynNvex0VI#%rD^~RlLbdkOF?i1pDC`eBznm5_o4q4+ zrgF;-Gm7OVHXfOj7oChT2 zrUqFJHO&`NDR0H0)zATE;mFx%6z}xBZBm4+zIg*AO_rfJ6K@5QUutApszo&z>i`_! zn~5*C9y~4`h^AHaLaW%>yyt2Pr5t_fIc?M*fw1VD9C*%@Xe9&Eq2x}^SpLZOae^6H z?K{VRYS?;kG}X9Ij%fZ&*eZoc9_nl$_Mw~ZIBNjMB}WIpHZ zYS?Sk&n8h6s7Axr)r)ix^iSqR2CVUCnM7FEeqVDGCYlY>Hr>#|fO-k2&4Tqk1P9%X z4deM7R3v1lQm95C(8j4uXi8+ImxBobXww*ZSdYELv{K6{V^gCwuxkQx(1|XxKb9j^ zi{Dy_<~S{q+xB{qYZ3o%pYcsaoG()qhoG0SB`APSy$w~?L)^b>pm!t4Wd_su)!V<2 zn5jN%WQ*%&F7-K!+ZBv3s2waiEG($+?N|HOx|vIih5#MIRKA?r@D0TJPtx>Hr!5xicAcdE6C%&e*Fyw}7Ke?&x4bYEh>+BSG@bL)LU zNSQKF)qbwF6jfW4RE1X1#cKz*Mkt0+S#C{2qDtmkH4XRVF4mMSH0kWt|8MjPqpcaB zfL^RS2_*a$tQr;QjjN3{ab2j2O7%vsF$0vBNsqmAJ+DJwm-CmJ2Dpz?(^k!fA(ZTX zOEpgAEa<{`5<8Cpy!5ao&?JmGZPYL3weEQ}2{lG|G2())flvt_+Sz~<%2UxdQSDRX z^!bsU8XZ}5R-3(!C*#7C^?V(=kNN{g+yM}1$Z^vo1wNm}FEn61;FHbp44yKKH~RZl zdcUXEiv$jGrlM(z7uzQT9l6by&LK&kojPTCwQrwr5jQAyv#&k{54?Pb5OU*-+_j{ZcoUF!d@6j0{fW)`)LX?0o(!% zg8eU|uuoI4e^5c|3JP!Ow1@L=YLMXYaIVUEF`_qez+2h!xI{b3t$Vv-z2IV8rHxKC z4L5&thrMk8P?Bc=P|LmuP)jN9^MLx6Odkb+F9*~uFwh18N;M=3BCk_mB31i8nA`K# z1(6AAM_|+oga&3&7CQ_*i}b9~eTibScZ{CCQM_UZl6gf}S= z7dlKdhMPkBc}{EbC30}}#T9>1hi}0KgzO}filZC2f;L}`g7uhcy<%qSY}QqzbaNIe zMgl4Ob0o}{TsL!~Zx)W2Z}k;4CCQw-2}p)7)n{+9h%ZO(;$s3OAuXdT!pTEO%q0Ek z`AKvo4&y1$TtmRB`4bsTb zcc{y&9v;yv#{Z93Zks3p7L{DNB{u@&f9}d{<3PPup)TO7uiPq3if*&gNXwPmtTVYM zAci3~j91V@;WZ)2C3msdcAVA(q3U&ukvGFK2680t4BuXEj-Af65q7tbT98|<%E;@? z!!%em$YKObyB3IVSBdi(nrm2Q{rWW^i{f=HmO#f!9^8aQe7&Ty{=g@Fbta6u^SNAq zPkDoXFEM&!Qnx(U@foJP@D?Fb51xz_n`KlAen8vhM)g3zcQ|_Mn_=Zr8UVGbRjKOs z_j#y@-2VQIXYBU(M?zWj?eDR(Z+|l!a{Ifo(pr0WY)p!d4P41(aQ&MhEI{QYpwhaL zvSc^jW&SmsZoLDSA$lG97(v=%zA0^7ayWMJ%K_hJz8=nzi{B;KWjRo3J;OeX0a~py zSU*2g#-+BduEoa+o{Kzl7(|ykKeX0^LOJ zsAa$gkH;9&yTh-4DX@FZMOcJT;KrMZfP2r4OoC?RPtDlUOj)T`FEdh)T=?qf0Hj*) zs7L{Pcv?eb-NhQ>t1nd0#`BT7WsSm;i*~1g+IODy>w#?6eYH}P^)MB-&90j!9(m&N z$m7H#&$-jr`Wij)i}ACQ*FD~sk~~&?_k?Q@qzH$x8*roe*s(Ibi~KmUnd!H#^o}g? zULr3U%B#&h2&-7<@qG>7Jdj*lI0w0{^XG1QC-NdB>Ox*p=i&vXkS`fV=!4-n-KxCisqx%z%6VLF6|B2{%ur!vNc`HltV58Toi>r32wf_*BD$ds zOXI#Mu)!5W29CP$V?rZH0w=n{y5=tojEu8lOf$ZioxNE+z1)c1l=MP9jOXBqri|A) z>ecHo`@?y9rulmlVdP$~>XxFdOcmq|fmfc`WE@8vp8HZLLA+GKOJ@3F>pUgk3@2L` z4P+kh47Na|3Qxl$$7wGn_e8d9L@#9stsG{3_zJ|;79NW%8lqAuWmfn(PJbA+?6Ur_ zQyn#;?(aO4R&)p|ZMs3|DXyrN925CGCw>SNMvb#pQ4|#F=RBgf>6`9U6e?6054O*9 zMEt`d<|3EbI&tGn5iY}Kgm<^u`hu<5W}DECmHBq8%!3He>@3;=tIQTyva7qb&SR^R zi|C>~G%r|$9B9#g{thrNNEks!^Y*iWHh4XlVXLN*c@T-P`qt}o0bA0oa3cu%PpG}vEhEg~QUvgla2m~r=YR#-cTa<}nP z+rc{!@g`MTzf+Og%-2tDXF_~IYa6Sbh-wysuwS#*?T$?`>NBSZE+Zc4+aGzu`Z}Mu z#B7x#W05nE8(2p!tMvPthKg_?=3FuRRLUlT9fQxsoXu!Y;?9VS+9hg|YJuG2n1898 zX7f5`W5Sr!2Xo$N!t(m%VtL&@&-XFAkmN?g3hyH0Md6naw0ni=tF{@Ho`Uaix|%xL zdkXnPvwt3Rp^bFd&Cx0W;cE$or%!A37EZs&I~wcl*h!vj_FF$ARWwPC@}K3+Btfc5 z;C)M6M)*F!Kb3d5`sjNhT1!rUyWOVtmHje#7pn^-S!|VsNRHAX?4Tl9=~<~U4TsgB zZY!R>y_$%=QPxGOY9%G(SO-zO7FU@0Yn`m+zLW*l6y#Dx{xy7MdDD5uv#)p0nRcGT zJ9bvndE7;eYI$33yOKNd7o~7-^TS9uxuH93m$!KH!&&u*e6K7XVp}@q7&9{Q0E34t zSFg~A?;=siO(rsErJWE6Uj{eVia?xWH9%1G@T^!X2okRLk6^KYG^rHcH}gx!c(F2? zHgO{)S<{D?u^k!ljTu<6Azu~Zg2m#T20X^dq#TgSN+%A?uv0A7Ysb(M#Bnq5B&fVm zOytvf<_NQeX%r&^1Cyv9U#-VC30})GEd*Z^OP6)@qbRXVkS$IQR5DXFNbe$OoEn~< zTVrJN&Q>9tnPfXVyYMIn@ zKEPTyftZ}SsKj-6o08ZCM`D)wCa%$fn1V3T-Hj9nsoUWs+a{G zNGK?EtvbpN&0BR2m08QE1Hrqws%_ykruFdDJ1{lhpSu<>hcHcM&UPQyVi97)znPj`Xh^ zgK6I~>zE`_eoa9KOLj$b!%u)s5IvOvm!=igzo~NCVygw$2I9%|VcHb+WIC#mCJj{| z6eFVu+hflMt&NY34UQ}ej_+V|J`*!PQzj_J=Bj7q-%+DZ&nAp6IeO&Etj`A2$6bBT z*)o;Zdnlf_mThQU+9tG4wwhw^$*Bo~`D)T9ine8>|sGp<*O$E3;pdg+84 zBQ>H=UvZqcx1@IBu?JI)0iXqkCe{I>7AvWTH`gg3BgTe4jz@d^b!5%`EdEC9HPtj1i>!m%q=N2&l-HJz&x1^5_fXpz{c(I5kSe{9w=}{ILLo0=B7wAe-sEHl~w3rBgK~4HU~x?^5CHi8h6TVqhnGvW$`% zI5$b$UHmmuvy3N$T!rZ)H+^)20#zqz%}baga=%L86Pi^m#(4HhfbLXdmZ185{`&d* zK)N!teUneqK7X&n@Qco$HmI<0Ms!k4oT^%sohq%Ry3?`NQWhZ19;;it^7I@=_ZUsU z1kuV5KtRkX@WzOU>7h}n533Hk!H3rNILi5q4-EavX7Y)pB5cs;6kvc@ZdiGk57kF$ z?I99is7RnJy>Y+8M*)2`__Z{w093^kb4zJ&%aC-O87ZkiqOcoP^=JMtDkTGZ)(D-( zw8jiE$YI?bG=5W!6N*sE9*MiF~bH*>#wsjESh;4#q9u z{Fk_v_l+JbuH5=5-HlNGsPsAo%TWo9>bX$h%bc!i{et{hB8nzn!ShQJRWcK*+P8N> z1438R$H59t|CqkI(u<{4V>8uL)7K2bVDrC7Uta}cf)4+PzSgBkjZykqr%Cx>`YOgc zK2KknwWDV2N>fdpyp1O)_Eq$?StU{QwNJrS(^pDnidG3Bgla1zV2ZAOA`s;7i|OiB zhF`o7NZ+V7x@!DIOsaInF?chcRP9>b!PF6$jhZO1Buq6zcQwr={5Qf-tS_RhLkOw> zh@=pa?6GyaBLjq$Mh+pUQCM^muL!EZC`M2P2&^b${u~AJs}$O;X?&-X@INA~QF)Y8V#eJKrH?Gc-Aw0la2syhvWlbznF!r2qM)YGkvs|YP}vrY(fhR{ucQj~5g zk|~t3N=(aO@?`Mrn%u7_W4z&?f7alj z-B>x3L2`+aG@)$GyhG)x1wH=G6Y7?ou6wrf2+gV%utVBco{~074tc&ygcx00D#=rA zYF=eejshVe1_Y{klioA?-93sXl3PF`L>bd=*>CWU@f>$A;exWj^=Rli9DxF?&Hs?)qvyxK{yp{bT3{oJyX z(0_*L_9!B`K`hR6WI}yg|1XK|{a;RWXZ$ln_YW9yWe~rV=#EN_(zJ8|8z~bS75HRv zSpNR)F+n3QsOgZyb%Ekfysit{DDB{MHzkrmrJz^{o=B zldpGmgYcjQHYeTuWwXC@^Orq2m#>7`MpjME929m93fl6NaZqO)u3QZ3fCbq)%53<@_73O_k0{P3Xg=0V{)!v9R`fFHD@ z5)yf7`U+aPdOk4;%LwtDT&SOH(5u+5u6}(I_5kVZh66!IA>SvxwCz5XkObB2Jlsb} zIJSyROj>44+P?64+r)ecX&&Ua#F$#V?5J7Vz3n~bfEaI_>b8$z%LGdS3F>QHH${uH)ZyP4CWL%hfK zNX*YoJH*m(`A-i%U(MH_`e$sz`aHJotjew2`zw>gc1v!&7w^Wx&yg*6kc#J`yTV7# z${ovmf%Ozhs5*08-Q!}`_%o|qm!V4-oH@!l`3*x9{+ALjEywygr76T%U z(dG90XB_`C4ff4HvPC-5r1W)%H+#>Y9`P=o-tV1BIFB2JK*K8O{U6ATSda9zM$FY@ z#LU$gJ5>TXe}{tn9Q=)f2X=EBzDcQ`TUO;|x`3vuMgWU4Ys7yE zuv4yIdZef+G?M&h#xgn_7t%0>BQ^lhK|J)d+>yYxVgQ7=?>JdrV-}65F%KvJ0SMoP zHJe_EVE1T=c#_6%knz}od^V>uW#wobiQ%hHg(Qm12iNZY-s)wN~}G|tuOe|x5^ z7iD}5CS(HNmbVk^ERSx=sBVKIrZ(y_*usoj=OlA8+2oM-Hi;}Yl_budlB)}s{R9?2 zC0=crIPPS#TFM4}R8-G9**yF!safYWlH`)14Vh%fP=b+`o^WkQPO^+1cXg_a7+{v&)9OT^i! z4`-@XaY~(jra!5lW1H1eY&nXF-|hDoD(lairw_LNoUQge!c~zm%;ubGE3574BR$lnM!kr zlhbWeR;uuR!mE3}K?bp8NyZ9xXX%ouWmD@0h}EV~*P|7?_B@J`6FVPh!r(2R|M;k% z9$cS=>?B5ZYc{=Xv)8~Xz_6Ivv9~ve=G``>OjY&PM+q=Bz;|J)X(P!WZ)%N3G0#?? zyn>P)vXJxiV%diE`S9*H*)w0Y0cQ{2 zwQ==jvYC+WeBJ7EXktR3v1GP{EG+UBP^%ix{*xW z=|!Tu2e9p*;Wb3@j44utg?ziLP=n-jsr`UarD%#u?7QVh6#Ujx1qmS6ElV znT%EkT#C&J*Ki!XP+Nk^S3%W>Q`l96uvj~X1<40_;sF@^);12XD zn+_aV@}%z8IMuCex)qK;azMJ)D3?5XSLOCyLf0-nLKV@jQNic>*2M*1v~Sx84)kp? zI}tV z>vH|+R7jjybhn`k^`CT_oVb1wZm`E`86-hb-u;}7oA?PW-i0%4Y;e3+&cK+V&3isd z!r$Y{=HQ9L=V9C~VJha|FnxFDHQ|zhSb3kMKZP$J9DB!w(bx-z#A4A(Oy-57&x4&u z&hx+5AKE7^Yl}DPck3qNmHkCRSPkfIb9v*6DdR}o!4qP-4?977v{5^; zumtWioC!7*L+~YfWYNikCuY2g^lEKVUgXW)#o@{~59aOD=I-KVn?;{KXGfOS--&}b zR}EVjvKO?}em!TGS#+|y=p>T`jk~zwy;gToZybgty-rz0>Im(u}k#dI`#uD+`5+{?4W$o$>?hI@l9lN ze55Eva0lWVofgaq94E(FTfsaK2e&U1|EE8KppjferfIuo)>uj=r`xUX8g+R@9Be*P zly{ z#^_5(j9Bf)ci2^kJOb6q7Xe&zpu|ZSb6s4scel8_$-KSDFg6dXa&))EEV~+K90V1jRbKTjOwAS?a{MysNv_Ar35^ z{*n}DcdIiNc67J8V&VAi)=YeL7deMj(X`k!o1&-LMS7a0FwMR$#W?Ml8lk;${?2~? zu6Q&}k@!)spCz}2yCg~GX(@#fW3pyqvYS~d*)LMx-a#}^r~jLOmm{1UeF^(_Iq}OP zm$YduM-%d#n~rv3Tl+E@k6~4r{@P5NcajvaER#FmjDs?f4pt!x!q;W|noGv`mo4Ly z^Pv4;RatnNlshX^5zP3&tW2>8DS3_zSa8WLML9tw9dpdNK1a&ONH2#^EwOty<^vZ7)~-%UYbJ$q7(?OEP@zgsg|4f z0H)(&R&7O^5VB(yWk%R5(P$+svA+qhh>EihC+BX~TsP)zmxeEh z<+46uA&|*BgKtNQGD$1qgLH;t8}tU#sg+_yaM+Q9D0lyk##hvsz7fXO_Fu_<%eNjX zj2*9@7NnzaoC@{xym|0Np`x0q%63Bp7P^Z@xP0U6h=-&!?7KbDzdMeQecybE_gYe9 zV3G5=41 zz_ICXT!q^(-wp5@u6_4)pXu$#z=P8n+@a7l{$oGjx9XG&79oPj<-c=JQ?FT$j z0Bl0ykewZP>PrHC>1>;HRKz!ocQF);kWuM~?Y#h^X2Q!g1uy#?9(y>XFV=V)*;AAv z*YdyxB?ylr{HmC%!=hTDNH1LWE!idlahY6NTQZ{WDgu65718L=MIH?II^m z0L5USHW%nk-ZXVAw9RYBK@fly!)yMDDu@3D=Zlg)Dvb~x?(ck&6;Gn9R4P*aKT>=8 z(A&m~7+=#vyx^w?YsEh+mG|t$&2G(v($h7`*cfaHuu$BRGB_!)gTTglhaZ|;MiUP$ zasE{3tqT^1jsb!{%$`lc!TmC49kasCeWx@J^%2$wVh$+ z{E%(Q0i6TGpE)qR^e2f)tz)Omb8M%6PTZ`YQyx~&g`6QOI3-nel3|B!ViyQP>W7zI zta5Ezd5Jt1c5#-1`{?EhyIL*|I2(zxl5BESwR4}!#@!!^+tl|N*^PUCOJg>L1dHrk z*Ie=;jp}YWI^5p!oQp52tzxNvjvVDub9GIee6rIERajl5b5|^@jXR7= zI5f^5!Tn>+Beo2A0_HC+|Yh|ybtS!{vx0*K1P`+*u5Ej001qW`3=cE;ww(^(GETa6CS?u z3;3f>T?S0aui%e9sSE$V@pRWF&#B|GvQ6Gwy!6VGkfAno$H|g?Qyg)WRk>vbT15HVH3Jj z>-cQn+j2DCF8p&=(ro6i2-uC9@*tmD-gX2N3RSNbPJUPwm3=^1<<%q;zTi@pCGk_2 zy-rx(+%n1)aJ96rB$pQuFnifD0-co3-J(z)#fzNEc$Q@96jjU0c!SlMU9wWgs3f#8 zyQDChM83jM7f{Pbccbtpcp~E_0)}#_??P5VJpH7LJcB%C=|fzlnlx2=_zqKX)Ha6V z{cdE~lrW)Oyb8}>#Qu;Oson`T5d_|YSKo~@2as$-ijPZl<%Mu;vjXy{vDfPkFDP~xrUuw{Sr>K7R{krkkB|LC*KGJmg2s``gn@6KK=}P0AFf-{61C4msuZAfk9G_=`^?n1Ez=V&WJed zkdsYXUQvo-edftQ==7OwdW~$sl_;CpQZu2orl8d~nda)F!jzNJ&k}E1i8rMr%KL=x z8Sinhci$#QJ6NAv@!Do%RBPDPU#Pb19A!A>;k?7>Ie5Wt|Iy9CO)~0B+9-H{<8()g z{@59u#nC{*n~pl!(*!q(S)S;L1B=F@H)q{PL;fg(WOkKDrg&wT?j)+ltQ`TIqZZ;{ z6AST}hu6-Kv1@fixqCP=j%wxLb0cS5HEP$0&G>O*Gk!LoD(LIEeD{p8TK=xaE%a}+ zmM`ENU*PO;k|RF{P)sZzS>m+hv@G9uWQ09ZxH2h!pamvnG=nM*Zs3WK8kO)INktJI zcL`I?<#r!&9liyITJMoXgCH`>*t%28I}TIFEC#O}L1|SPV;{8{ul8J7XPC*1GKUZ# zg)Lp|JmZ_Czf5clFW>~r85lNeSDDog+efS(X9`1e*o!iB=n-5IEVA_X*1O@ax#1W3 z5;%MN5?+)6gTpDW@xH`OugRd8KF-m{%|(aJ(#{+d6{ZEQc8IcRV3iB*abR|u*}B;B z{+r^n>I1#Ye>_d;o#0A1;!PE;v6&Qv>*zp|I8tsngyPmOemtP~)HoM-VUt+o=U~PK zOOIRCPg7peak1a`N7|JCq!1ThwldB44ngBt$Ay$uH~m6%v4!b57h7}%KG&-Elxqf5 zJ!1+(l?`WAsR!ElQO0~c8BjeIoh^`FPVqdM3b|AxA$woqT^-8vn^%T z;c-Bwp&2NM z^yG5uU0@{YBo|jy(74aF4IRS=2C|0j!IhCZ{I&~=?4+poV$n_OC@MT~0Y#TUSe!L; z`-W-aabX8wO|#}xh}IgpyojU1f5#d^PdHD<2Xf&#q77r6!hMO^mcJxpEouiD!pPXT zW|%cnCusVnuwT7))3bfZYl`K-No+fp%xa7}SP2V&SWu;k9tjfrt_g)2IgB|4184d{OypEP6+ zC_tdTfL9`4mjEPCdz^kI0}C;Wn+e7%dv8-|Q&i3Fd>SR#=HQ#cw*A(L-_xB^(0o7o zdG%t`B06eUh+)JBdEX8d!an}F#ytypYf5T#TvlTyfpgG5qkau>a|=FR^=9a1LjHB$ zINx^}lZ#MLeZ3|>*Nf+Y_^W-N!01Wg72910a^Bi|>MH6BF6z3jqI^L)8jpfT%H&~} zIII6V_4#YNY(|5iyIEw_tkX0v7ml z1*cYR3VW@KIZ&UgV|^c>4w@@Xhw=>p1UZJ& zYhMC((a(BuXei88Y(^AJ_p=T zp=iTeK2I$${CG-{-AgGZIuS-vj!cASK&H9^oA(`75#V;%gj9Op5Sq8Q!c$#tJ+=$0 zAD5P+U% zZfoHV{bn>+6bwWm$VwgTGhS(+*`+SRDy>xvslPb^Y!(LZVXlKs?r2lA;<8gZFIY|1 z(-qbW3kDkTD~YtGklEhuW+!F!-Ot!8g6-Fvv)BFx>#YqxY1``Akpvc!X#qy5E-1UlP=5>So6&2QXGqp;aD9wvrcDi*HMMuYaA|L5)xI|Rf zyzinEc$zIhieMwvT1&Vv!o-;vtT?F56`(K;fBPHkA7}t0zrRUZdD_~;6Iy&mUC)0! zMcA>aHhzP;3uK5UuvN-%{0vE{E1C6r*i|@X2N(W0?Ssu@iXcO!^Jc^l_xO>WLRX zpS12EB=ctcRNPBAt&5pGXe@KZ$HIita>IXvG}a;#`B77Ecz`$X)tdif+1&`92}NI# zd(Tx|N7HZXpGLP25*0f)t!&I<9Y zCFo)~UQ*Zsa>xQOo@o538ujV&-8u!7puP4kcT?i@YTpD;b)~hG3~oG>c&}LTSp8Kt zReq)Q9Wa@ugvU+4)O*3;53}?KY5ggMjlZN!nS(#)Y46%5&6DB`|9|tjysXkP&g78G zU#kf8)x+!0ls2<3MLwgVHeShzP-=a-c=(C zNVnFxDye|9YvQiO`da(a#jP$a2a;0Gy+=i*zt?TtpAT9Z^?iUIE1|~y{eao1-@z|a z+oRwfeuPK!#Dz^dV-9cUg(|GG&xuBTP2gs~55vGaA3ko4@y5Nre zKG@|ARZx+nD{-+QY0Q^&eaXiCxhh=-4jN@(pKcUur=jmsn@XD@X;W3&MB{$P=hAl2 zme3SQTbkyX_#HE@CcwQ;hCePo;tWlUmZ96Vo7z>onhAJPe&X)w!%a)Pxpz<-uM7P_ zz+jac^{-R1ClQ|`M*SlM@wHaQ-{cabVKey$-*J251r1*i@F=^H4Zoq0A&+zxLkF|P zlw{*Iyos6RIXs!6V}m6T96$|xo*ex=Q8Mi^XOEFQ8NAH6h4vWtiNxPah@un{gkJaZ~eh8BT}2HFOC%{k^vs z_cxGClNi7C9n9!U$ySn4k9RO}(05Yc=Uq|a%~R-7My#q(_40DwqP-OBQ`A$ax5^%S zXt$6=`n4_v^Y_E}N@$#MzwBBE_iSZ~RSJM0^@=3Rtt{i?EmV5noY`0ll>W7KzMzq| z?fD92_`gM~TM3v+o=i7cux#c}ijRz_s0?KZRBAI=6qlq`LZ4k~={TsWQN}@=$d`X5 z`)bK=Ka<}cAHf7%=$LN0!u53v!1ImQ(5(R+KTfVtZ4?kC;P|@9{=USG*@M$OEot1* zG==O>)(!LbCL8yq&@^iX)FiqbjhiawR+O;-Wty;yd+^fUj5nZ}mZ$Q7GeLnBv4^2)5`SY=ZH^Nx}=MY*=|_gthmslvH%Q_A0f+Xnje&ws6!V}+lWBP zn8=gY08Q~uIVSRZ!9wd0t%Ra;fg1YXB-dMxx4x;j9C#0wICz!^@Z z2uhq&;&qzORSR%On9o(oGtqob&XoS=**!A#!tvW~=31}k&;^HT?6cF5t2*BE zdW|{TRZ~d#1fdk3AMteZJjv6+vzrd)?YoXpuN=jTK82rgU1iyIk>4+q8s0$|$ok|K z(5(Y+n&IKr4|TW*77-Q+jQSL+4yKF>&O(A~O-V3q5m=8bs||v^Hp6VSCLm{Ip2WQ! zTez&9iF@|Y{L$B0M-X+F{g}lmz?-r&@yHP>Yj*}<98pbzSQj9NQNMvkTvt&^qZ$rj z4hn->k*B0#xW2ufmV^_m&YPuiwB~BFpA+N+bBUM_@E6$Dq??G(s{?l+PDUd6>V|zA z--9<*f%$W`BOqq*h8@Bz6QM@y7&bBFEC=E89-{%ZIQBEPzKs2=w)n!D{A6zrZ-FHi z%@AAC_n$*EjQV#4Yx}-t)ZeQDv%Kf{`%1j?{e7k0i~N1xH5z_TvK0=_F_qS%q>ny$AACzdBk+DL%atTWk4>9PD^@W)W9b*wvG!d={2?< zlF5?dz13XOTjM`9QNkQ3du6ndMngob=rpc4N&=(2Pkc@}@xeUdAatzEx_}D?-P=qGl1v;_X<4)HE+7Cq&-|D=a(t)5cx^ttM$09c z?d;rfpA#5Kx>5OyPOnY14%W#Udl3x}yR1FxWi&6=Z*x&_f{P`5%;Umt5i2+`PM>B; zj`Ca1O;d$r`0z+@^Ln=aIH0tSVJ6Ik6NX8?^^~`CrwFCNQ*HBam}B!9o-C`P82Qw> zbA%~y*ql3OX-r#Y&mQ=`c_Q|mmfT#MRxGE0o&!C9^&ha|7F;yb<|G=@r6+KoQ;J}eVf$Gq)H@x?tI#wLwSn;7cR*X@{ij09{MOyS&;ffwB=KGR* z{@QcE|5w}MrPp)9klW5Zd#K7u!LH<8{HFMZ@$Tq3%IET)qv}-S$`Dap| z#yN~mDMg=^vTMUYnfnG(O7%J=Vp?^-<(j4`1H_u~7zXQXzDb@$YhNLUU*QwPB9pE8 zY!D*}k|bsf2H{({VB~Bg$MD(Y5YM8^q`YZT9-c+{a8Il#*=Af!mM?TJdaV@0Fcc={WX4aqC!zk9k^H9V5B^>+Z_y9R@!o?^yuT3jR-Aw;|DFvwy@x*C^ zq#OGL%{A)xP^+;uq2k?QTN%%V(dTv1XPG=}^2e6&^o=FKH3Z5f<+afiSIcujWtf>b zB(A)?3|6pJ!+Oh?cvAUyOp=VYXvTi(m-o~1q>7U~edCDeC7`PKG@7DYo{=?amEqBR z9vn+0#Fv*_qC`UZ;cHQ5z)yJIAy4vE*Zcc!s!idh&olh`ynUX8*^A{Sj{lWUx9iFl zAaELIy+lrBNr|&+S$QSJiBr$tsq(7K^ap^LIpbmZy>?8&Uuw@UczONB1+Q(qsNm%d z7Zto#dm%I_daYIP+WL z1|}Cwc~FMpXU*Z(f+>$l@S*1L)^hpJS~Q8Bs3&*D!-6d-qoLGd#1u=EfhE|Z>!R_31KtBQG2#T+P8=~av@Y}VXs zq1r{e0tzSm3QHn>g*1^BRjlRK4eW~CVsL8*yU2A@$9cVvJj|fTnHFwMV=Z4P&6^#I zh5FsPmdHx$f3NG06qZ>&nF8)dbtWI)H1~spw$NC9D=V%4Ad$NeOZoEMikl&nT$rl$ z!UNSnDa?sPVx^~+SuQC(Fjms9c#@w;dKqHFEQGNAjQTdR1*Qms-;^p}r*n%~Wt)iA}gMz zEDngf^gRdpT>jb)*g_jxnjslSs*IdeI?Y0dG>N+7l%K<(jP1N9pg)JqB=scWs`;u2CH&wOA-Or~&HuLD-rCX|zo}*7 zL-^&|X>Z4gS80auZf@P^+RzKWG+mWiQG_5F(CK-wYvzj|$lejFt5yO7$KU>LnYpfTEJt4|^4@@y+=G zw-gj$rb+@(L_KdGv8lyYeQ`o0RY+=h*3w_h7B zXLlkT*chL6=!R&Q@L7MPwk~IwtRobY)tSTY>H>CKJX3hO=HDLM7+oLbyt2GU(idu* zrG&)atjNTJ+^DvdtbvqqpCJbQ)J{e?i*j)iNw4~nEa{)mg4hOm-v|MKy&MEI7jp5Nntjl?~k+#tWLfI#2G=oP})2 zKJo9yeqc^ZNkMyXW!!o$w#?Eju1t3OQmNduLCG+sUz8;#9Bw+_ShEqtIt=sW z!kq|BvQs~&%$l&8o*xiL-otj<)v>(|v69|!0zY{iJiPyjoRlyzG0K=VAVyIyXGPQ3 zs~RaBKe?pJUP>JIeHn7Zd5r`;MJ{0+cqf^qnR00TBXI?N#{Ez6#9H&1idfDQOJpCC z5&BO)G(MB8|@v@J}FUs^e%F4otidQ;@gsQvhHomvQDnk~>U2e^jhq*I& z!|?$&L+x$UsIQC_I&mQEvL5D&HR8)g0!K<2V*gSgRc5^mZAJ&}i%`q|Ir#>7O69^z zDoRmYq@w*G75C?Gl?25u!npjm*++}I<6-crHWT=rSGmCYrjG0lkK~@G_HUqyix@x7 zGtqv@lD*LZ4sUAx6ublLfDxB5M#GO~lzLE=i?0gF>f(>YZEy;pOq7|>DVQ~uuN$r# zG!5p)rol9}1)rM+*RMZ&8jM-_Mbn^D4&yD1rJ4gUqOv@3*pZIP64eXSS&)b0|6$Lt z=m%*N3P?u1!0}6`=CRMU>6JU9$b8`$WNsAHQph}P0GZhp$kc?P#c*A@KsXPFhfGc3 z3>-r)sWXP^`>m_1)a+tcR$<*EZXU|5Iv!@x0eA3*T?3f>)Xf8!EbHVhxS-6Aj(dHeDgxiHi{h#s$;P~-~7o2@j6&VL$+J(bw_S6XLQ25C6d?(4WKj$N3 zD;yWc=-_oC^dt6=Pp};A_H~Fr)B#2dTd&(WRLdcf;WvDz4rhm^FzU;R zUf~taYa~6B8s+>XqKaja*H_d@^rKRV{w8soRot>6agV9E<%8oMy_q*)p0=X-Oxfgr zbOrJ76(i}(Rb^_rk}e5aT1~U_eaCSb;iw8dPp9O?6;Zv^j?f^y?>?eR7+Xj7f}P&^ z`7@*@Z!u*@rbrOL0H@O9u#Oq}ODQ`ivX|fUgQbTTc$CKqj_Q#`7IBH9kj)}c>q5}e zmT3#;J|aD%+$_o?!PrQLu;9Guk4W!`M_xqY6Ds5b`J%0pxD;p+T-0NHl5N_6P}SlU znyzS0m;SA+FugG0Waqd(D+TAod+qH7{cD^By~bTD*!~*ZCt9N6w>s(L$3o{4v8k^P zfEahZ%FviQ^V)xfPMHmb%Iy=&+DT{J^;hAT=v>$$`z6VGc9IYD*d}2tg-iH92pik~ z7QZ@`JDrdtxW*Ak3T|`|WnVoKYgZ%uA!l%n6H8h~aHF%Qox_dI`+Z~kdv?SB2f|q8 zxb|dl4T^5JEp_=rNbcV@H{oH?BW#*eAA5M z@_JilaAR)m*5H~Hr5?(+YkPxhda*r0@T<$Fd4tN9w?Spg>pXMHik7R9;uba)CTA3~6Se1gBvVB)1t6|T3WXlTflJ7jPk>_FJ zoCGHEH(@Up>yw)1#x)klHO-Bu+{WT~f4iMyyp+CYpmbn8kXEINgVw_WmsGzPNVQp~ z@wdlG3wOrJ*T#Xbl<_<2e2BjX_`8?Cdj8h)=jHEa{>(Y|9Sh94{}@k;r^WL)&*MD1 zcy{qT$@3)7UY@-?PxCw-bOgaTo4w248wH0MkxqB;cbLB}Rd!a&((BtcOOH9<4TN>? zqcOB`2t)u!EN#$x+3Dc={L|4v$T~9!sh@q!m2%dWg&}$n={X3b9Rw*WHYR;~OgMsO zHB?z@OeFE<6@>5Q?`i&;=GrIhZ7jC?PblbpprH4MK}Ad3qG`^I)rolj2Wk`=2c_IR zC>=eZhtV-RVfTL^BWv7+a9?Cl4$5}YAQyib{AKc&&0j8mQ~CQgf53VSupYxRgC{Ki z`_g#A+`rGolWT(eQh28Dbn2-4DA=429o z^JQ{Y61DMqqHg2z7;06<&zfcYY*@*M8;RR^8*v-g61Q>V%HXqg)DnDAey{dD$}%$F z+}rZLsHvYDl)EV#S*dSh5_6wRaA};`MvC*B&c!KYqVdAM;;`M5`<+Ht;k<_38$;CA&iQorx>0pAr>sv6+uLoq zC3REty*>T5x^G?PONh*Ngh#~j?BVt`J?b+quY6m5)_H7EhtJMZE|l-;7(_h38?nHLxe~~aKyz{_b`?{ zD!HNUD6{0I-|ZHw6fj#x!mADXI?U6LKVI{=HR)CW?K>Q}%4xh%^mf3GL4eM{RSvn% z;2c(AoyZb^WznzE()_mT1FKU|*ja4tTB?UT<_qIIIKhLrXWXAI%8TUv9ik;SsOiN; z|N0ebHoLpCuTZ-gbX}q^UF-S^SEtwYWv)&%e$(;#`)$I`tUEgHbqj7{-oB_T5F;kW z;CmtyT7=5{s`GT$X|J$JQibhh<*bz%$LZ|N@5ZQ3%SfR*;qzf`L6E$|tR&K4IVHjB zUn7b0nxO-pcxyW+UamH8UePR_2eQnfApDZ(&-kcbcn9mLc>T01EcYM7(WGLC%wm6$^Rx{@hqGW zD0)Dg3y*6#niwd3aIlLq+>K2@kn?F$mRsM?pxpuAX_MqB%K3{~^kDa#I7gf>%X*i2 zs=dDvB@`zj!7fEmCtb!>&uH^R3E@;&zv!GA-YVWms&w-RZqt9tb z_)?-f{Cgo@Nzd6bnVAXC*;pHD0`n6C;{w++FDtQ##H%BK%+1>Dz}iWHwV8pnS%I|~ zfwf~KqcgBJE%ZZX4diLFXv9g|9lGtM4@RzlL%f2MmblBAP_#KvoEDlxVpKhf9f9KE zQbDljVK^wC&$L{zx{UfVQX5-~IXgWZ=S@n@;z$urNPU(^O`sV4$EP`$_-nV>q#@F> z@OOemjm+hzIpQsgmazv?ru}j?5_>#)?tGS54`nmg`FT@*4$IG9<>w9g*znu$}E5FuWdMrseO_ zolnH-%vN5ts-ab64>RkAqWG2aw1SU#=C@3Qz3I-7kDq%nq!URMEgE0va10p1b5duwS^HB)| zCn&>q)b`j`Pi?DJ`_NN+%Bc!gttLQ|fRrz(qEefx)g6b{s2GAG^ZxF&_fCS^^Yr=S zx!!ZV*UQDuUUTpD{c*2*-Ru51ZhUwbh*3MP-?J4Ucykaem1zDeRpIXQR=cXg&0dVW z#4dcnLEVB0Rk#pOf6IC&*Utd*9}|-+t&?jQ)=^bi-gprpbbQYU|7Mf zKNo#7-yu8zwjk&@7}+JQ6K0WThs4y5S)#9MH)9#$6)T94Tr-jFB5m zZnu$p|FY0?Um~+^-I;AooX-j8II>NC-bYgeHxpXc7!RcII?nKy8n zU>bYwYm%@lDi?A3l&D;zWw&yny@U?JeVyFb`C96))pwnBC5PXpM^4rsk=@z8YUDB< zd8}F~<5Z$&(^3hXxxn+lZ3LVrxp>u%^vxo;WTAOq87GqgeKXPek->UbAqkAaKg&F2 zw|(UpeJpggh+0Se`vS%rHpHtSZ2{wB8!|v3>RMA>jLO|6no#O)GhY48tIcrJ$Ig~V zpc6i&RbitL86c)Y0=YS%4vqvJ13!Jl*&WOp_$j#kqqmvZoor$L%txG^Y&cT}?wwm| zyw(HG>I;5Jz(vDI;BV8v;7g?9xZXTaLgdWNt?q9c_{B_kLW;)ZHX#dPVr=mpixeFV zolh9vt=y=G28*je;~uk#NUHkxk;lL<5czro!LHBneQfSRtf#btJv^S(rDqcqZKF8)iO?Zp+IlN~CQWhYygJ$7 zA?y^Pzwx*zB=jyY7HjmmPH+{X0WFcA948d078TC;pp`UV@7@>}b)NP9=!^s;Y9gBq zVi@DSG6#?m5gE8O<#h3vrVw20g4Ruv2T8au3te6se@+tEnE@JB6<(h>Sp#1shr1Wm zy2IVyU5%ADZ**Q0UN6Uymzh)e_mU4(c&YK(%{0rT6V;jE34Zt7Y-Je_48cuvQ>d)^ zC=AW&-9Ig)?wQmbY%hFMy$$@LJiZkpJsz$+;Sgajb*S;8DW|4gw;W}lOk4d1WJND! zA3qd>MLjId@5Nkb-x~yVog0!40uKf=tJ{TKn(!&}>EsW{l;k7vYETm1)Rgf~}! zoABT_yin~jo=r|pFv!I#J8zrXu$R{C$irM(Qm(%}WYr*?IGkCOB+VqURM<{?8ziX6?ph?I24Jg<;h zUMxvo4RV(9XO#vs79cQ&40(1-=xUy^JYJrGVIxn|HnP7u8k85(6S4DK{U?O`8aoF~ z$rwURC!|p^XQbkU-u$v$5RaxXu>XcJp^KPd7ff{!A#|>3lX2Em2Z=Gu_-U+=<6~mb zxga)msw0UQIa3`742H%`mE@t(#t4v5PXn*#huOe!$%yXmApxd*cOmQaLIcH%WWK?? zq@_~W;KwJuIm2=C;7I?l#wLB&nq(+dB6Kz9g$4<0JOMAG4u>mTbizd<`hawhrJ;V< zZ6-ZY0eAr@%ciI_cXl;OvM7~4`CtKVRQ&~$m;kG z!v_Ws3Lh*v?SGS4?<_SQkmOtwnC^H%ZOnp>y9Zm-|Tk%lPZjA*s_x~Giv5$|5xT6cjIACgNv*!dc+D36Pm)@zKFoYwioD219~o&Xt2IU{#U{=j}s+o+(_ zY8bU*KFOLM!cFdI6>@LfsDP&%RKjvU@@8PDJG$6Pv|_eEz|nsq|NO(z!pnNm^5ef&%~sTHno*o zI*N|<4?6Lwd0Z+FC8jrLw{TKC?rf1}pIp-0?EA<?IbmJ|Y;(pV`p`hchNOIEG$o*EJ;wJtE zl-6Qj;=y=Tm7S#sG7zY;mZHCb$@Z%7252K^w2Tv1YWUQxp``Gs%e9oM@ON1-1GZ*p zsV_SkIAsb&^KxDu+`RUblLtow0!cA#g5%55I*rn0tPDkUhGta1FSoqL_vQ%^$R3tc z-KhW#Hef{cX$4qe1J0>-&yyqzEP#e`e;z)d4QIH0Qd*~kk}<8jFs;R89gB9xsWaG! zSvg0Lr7(D6li)><^7R+aPTerQS-3gfNicN!a&yA%SXnZBJ&BA9;zC|q>}W9~gJB2^ zT)eG}y9oc2>N~?;i1pY8`~%(C~o7xEXB4menC#Atq_y(uJFy zW_8{)cfr`9C_dcr(p*}G37%+C=G;C;P#{J_8M#`z_U~qzr*QWExGfCuU9GWib@Ht0 z43UN@^BP~^$fIKg?UiW-nQE#uDjswTSRuSSv!YkkMDa~U9bF_Ph3Tmj&U^Gm+@cqv zn4TFH(=$UcJw-h|1FkApOQYQ1qMn{f?CGej*y`!&=#dE1(`_<66{}M*J>4)p(=C;f zUXJGiQ+Zv9_I887-DPr!F>+8jwG=`I(MS$)u~4y|RZ4%{8k|~eTv)=K?u=+BL@EpVLmpDvH6NJw1NSlgqdd2J{nBjx3%(GyB3S2lz~UTp=GNL;2Xm zN3%-IiD!_e+!ZJ`UXY5YgVLI^Fmlm1#V77CM$V*xhzONygCoOsaN-MJ;^?@S38_!t z;flC+6o#klaA=>ZqvdsVH9=Qt{3@w}@gU_%bY&l+TvaP}ljP*tbdcnmy4EJ1d?#9% z=xg!ZhXzc1D$jMEFq>375Oi6Y+(0HO2Tvm>PL5m_X#&QSbBU@N65@u%c+UP%YK-V_<)ssdG-zkh62|QCPZhL>}`BMFpGL>Qi^l=$`h}-B8)}5lK zkl5JbN|`b>fVt4le~0;7YaQk0Mw{QHNm zw^xYqU8t6qnKT<>)#`;2oO6vez29%)d#p&W@6>WMcj% zc3FJ2JAk@Qbpi(!F6N3EgQL=!Uh2@_7~7Mbm>N5C7V4F5v{m%f^&~RNeS0v29_(Oh z_VnNsdQi3!v5M5M*yze(8`Qf&^*_6nRv%?fyd4_kJ5zT~_Q|MgcqylRWAAje3Fy1j z-6&dPV}8TZyZK!;6ZKX$GX12`FCQd3Cn~Zf=k44DZtaR-s&Ow1O4N7c_n_kH#}EaI z79v#p3I{#rO1#iE=WSdnt6lO6e>lAPLwjEgH>EpUIkYTeY`F+FMp3A6gvk1bX=J@C zm{skN+O+(YI?Bz)`{BxCj$oSc11VXCFs-FS+_wPw0#$pZ*QP{j!(ZlkqDuRA%1?w? z5lQHmsYIl{U3V!+k_f6H<`jMuaMuj(QRas?z<8QS>HjHtafw3C4h0+>bePSXLG#*^ z%;wE-3unc+T-xS8Ms3cI^4V;TWEi^0{JoOqgKHdIz=_-Xegs{Wh!)-oQle&2zq0Z)bk~w$u^xw-cbWI;X zzf28J84wR$6P~*57$RdxnC{7L4rPcgwl;8X@I=f#H~4wXIXBo9AGA(*PA-);oSqCi zj14wU`enuo;blBeP&TQyc#q`(MV$>YerkR(_Vz@MAv(L}Vv9<<@S9Xxm$aPAe3u(K zZK0K1?@4Yf|D3W!&jDNp$|x=)yIZa7ZsvOPRP&cT)7u=9DJgMnveimizS@Q14o4{C z`jGp2ZD6T1v{V{e+S^c-E;13}&;>MfKyTp(WR;o?op0Bm4|^cooNkSuvzZwvGfu_o zdutElMKF?rbLT>ZgeNPy3t#aC)j`wT#+dMa;}RaVGwPV9bsm5zn(dZMI(0sW%q{Ime#vnx1i<>z2?&iXH6P zo&u3n(b+XU>pYjXd`pUF`_b6-TbzwenoHjs%V_>9N+3;G^Dn@@DQP|1htH6#b_JP4 zDw|-XWo}8C%}(@r^Lu8Bqcc#{uJXO9ohx@T1N92T4LqCV9x|3Z^l>LR0vdtFf>KP)Ka9-()Nj_1y$tLQ-0Rm*Ra zewMDPmajO75z|YQrmStAZ%iua>8+b7J4`@6a2^=$XkTki8*p{4Zj zg`VxsNQnvXY&wUiTbxz;r}R%pyNrgvdy9j9OCnC}rsfaKZs;T@++0+NHHgzyzVD>h zrt^2e>f|ctcQRs^6F#?gEPpdsj|4I}b^%T-PLn{2%?Nos+Z)~IQIaH<58Isu1S9?o zN;PYCJvF1qjM(j4oO2j}6U~*URnh(Yrwbs6xO*ddl`mhFoUcmG&xlO|a&GOF{LNfF zuF9D|ICe27lnl>~gUpeVjIQaa=Vja?;^vfOR{`H_l2eNesfcsaq3B2a5&Fm;(gh7D zpqXRZsf1W;A#{`K@HBoT9aWg_OUnEM1ThMp(Wycf6=wL7vMk75CM46BG|Yl5H6dBP zB(DW2GaaW-E2Xg6NpXsh!_j>BGTJM-7u0OMV-0&)AWQk zBv;=iZ;yToi>oiYN!%fZhJv`UF|HF%G+PfglUZX*krg}CfjYRr_v zV(&mghF_F?;aROsR51&P$;6jE#DmR1;*}ZatJnv?V~`2DN)QMR5+fv{v;3pk}pt-@!G66RL<`|B#-=@q!aIhVFX)ib{5FBEIZvq_N9k3F>j6D$uX4xOF>P_Pf z1Oqns!d~#GKya=NPU!`Y2?Wo_4zh5;j85*GdP^7^2%e`BKuNvM7pve)1HqeZ@GHIG z>_D)@25;>Jj|&8qDHb8ezIRskmj!}lc7oNt3C0J4H`(CZdcl_mf*b?rP4cXeLKjB{W>?&_0X35Amqxaxk;}v#0cMBQuiIlIG4Axi18O zOpvHq6}s%OJ5uPj!xJKf6YTKBNZ~~4rVQgHMm+kzHIN&Mg9u+H>&=~>=0v?<4Tyah zK{oV3@n@ts<0={uuUlhW#v3w?3<}mD)AFLEj;JWTLT4s&3UNB~CCL&Ly!81bg$G=| zq;w1No(XaLk}@pFArmsemo&+OykbHo`jVztkY5YrY&?MNJV65f&fKtC{6liX3K@7< z(tZmGOb6$SL6$&iMC&%La>^4 z2iRI-INZuWDTS*{g+0o9zf1^_i8ZUI0q&iU69eR9eHo*^$b=j%wXR{*0Y?4oQrp=*8>)ySc&yNhfb> z&Lj}*Ggt0fch*EcGyPr)d|akXB>8cfC(P%^Wp*%^AD217Jbqke0&}=yF{Dq= z`P z_c&v!dQUK}QtyezW$Hc57_Huuj0@CziZO_Hcz0fWRi2TmK+}yx^`2>bah25QM&ncU z2Fa-R&BnXxJB%RtM_fjUiDsL>{Rb+;|29zX8cyYR~rAJ-XY`X>b=JJclBOp ze4n>eK}lB|jRM`S*6vyrlxqy5Mg{Ma;9?bgP=fPRP-KaWA{BgCg40w`&XLA7D)^WL z$El$31&oVSaH|9dtKjnzOjE&^Bn- zp9(UKCB5AKN94Eel}R^Mf}!*z`2fz5?I^NCKJ*HiiH{8tZ>IwKPo)BoZ_4KKK5p!b zja^*HDJhYArP|RMu0Ze#Ec9>^=@t^~tLX@3&4DtT8@y7$0ux!yv`%4g1aoZIpMcSQ z0?r5oC)(hbt*DT1SJ|-tuwc+`SKF{30xJt%Q%riZj7=&(h`Hzt=(S7jcr{j9h_!4R zc8djrQX6N(3N0Aq)nzv9%HAy5CdS*~p;lBjh|6tQiUk`M2%?f;*66bWv(=DP({0#a zfT=AeIsDahCOD%Afsm5u={$mgoe7_25VN-Kp;5FhApyS zHwS_@*svlCHZKso(T3&rW_fEMSY(4oSW#~a1ZUf@0TygYAUMZ{eSyrQj6-!GILU^6 z0L*ImvOqA`1|P7Zt_)z5-0HU%Em$ZJ^x3fgWx>`2f>UhRzx8IhF2FsAm3*BU_1*?% zZXkHA4X(7ng1ZC3>um6B3oLj%5X_THDWNQ{>B;+ELF<8FzMbF#E5U;bJk16tTi}Nj zxWEQ~oUgj#UP1kV;20Zx6tLMFYEcG)W9=_5TL~Uh35MHXu|!Xo-YYvqAn3Kh4_aW^ zEdtff4A!UB3#)odcv^kzXMcp=7fbY#o9rys>J_SfngxDIfz8zmINJg@DX_VK0cZA> zARCaqhWYXZroJS5*^KOE4EQ|@EW47uk^%3tz=CN4tY~Qf;O78aU35f!F_$!7erzR> z&C6cZfEz9FF$FdkHsIwJxLtwGwGFtmH+$LR?B&guDOLj6@9Y&0c(esRsKDkT2Ry(6 z%RXqYbHFFB>#2RG`eH71zWkXl(m5v-*j(*^TP(0_m-d1O{2L2=T7k_q5BTBU>>VIs zd)f14y_GoyNw$7vdBsxd%cYs0MD|(vhUld0dS5578*cMgEcXS^=6-^zL=f9A`moD1K|5C zu#gruY5=TRU?DU5pa#Cow@ScZf_-1q#}1BIE-4k40AI6jGptMu+<1lai;@dcB44*} zbCeq`VGxV)^z}Dv{1Rgo8&+coW@wKgmJ6>}PsYbPw*Vc+hUskNYt-W`o-FGVVvSkgu& zM5JgRcc9gR$Ku;%BP%v4(e+R&aGu6L0!DLE$)u22rI?FNC2e0d@^U>Xa(YvU?Ouf9 z6V_iL7H!7q^+%&cA8$#3>3o4_``kDz>6p94HQ77OvpvCgaQz94uukG2|7Us|u{|k6 zsuD0Oq#c#3{=;9X86e#wdRq!3$Fe-Yl6clv@#OhqPk_(=OO_Jz_M6+gpryQ$tyQ&d8bC7^0 zV}$UT>ln78W5EsRLrPue8jmZ;`+Xn}2;~0FB$J-p6Ds673Ti(=o2uUak|%PHfXy~U?s-s!CCiA)T~vQm zuBbLd?vea-bW;jLv5OI_G_I9wwRYob^*&~dSMT?Yi+Sg~O(_L&?WmE(uX3afV$;ob z$i2JOh{#W+a5!Z@m8P_+;@6bVASCpcV}XRE(1wMC3Pbb}Gw`$$kiUC04K`Olm-xa| z6N}-P7tcPR4a)vhOotD^WU+#!56(Ui`uoBKoA#_t;;9PH(m^)p8EYIjy-W z_neRjKlG^c4E5~Dz{R&H^^~}Ak=~=#^Ks5$F#?mDp}oE^yLrK(?)fS!%+v2gvw77y zu(Tz!etC>LG7q(%nk6X-ghX+`E&j(#XNCrOc3dg98~v+%S8z7lnMi!>IP9R>{m56P z+IV~$&F)1!($wfC(Sr+?1UAm*{*qBWa*muG(LG*&srfBn6k#AOFJS$O-VBbEpKCu1CLbuFwMBE8bA55SWx>Gb)xXSG?8!L?? zhwt?*fD(K29?_Hcv)N|eA0vT4-sI%!n-fF8b=@8flcwS>F@c^%ku1M9nZLKJ>KAVM znbIqBPK&*(@3cBvCSz|$%jwx2d2S8OH-7&d*f=lxu$-{<8#1Gt)&V{FW+{VpnS=XKmcd&|t|fq0$EIF97ND~Y6#lfGxuR`MfXm0dtDz)&)J zk*!H0H4-Q_{@u>Z(OA;Vo74ch}<1wajSsIcP- zYSC5TPSkwiyO4XjFkYA5o7u0>Y@h?Fa&q_vcVg@rVkT(kV0Von5`Kh$6Jn3*1@7O8 zI>5+g#Eu*9Ry~nnW^?k*>IQxf-81q)U}Lk?o~}5C$T&Dax+1*TEy5nL`MssSH{NQ( z6q+EDAt#=DS4QYY`9*E3a$l6p}Zc{U3HvN&_J2PaQt?no@{)mXZ*&WFj6}t0c z`DWAr)BTOhtiBjgX8iCuX>D2Wg8eA~v$Yq6S?zjZPm5{rLRrA8y>^4`CERkPcL{@D z%9Js!xz(99hSu{!+Nx}t$Vjc3UCcGj%&6;{N3wUK*N%0rFYkOc zJyMj89yE8!@d-xFIF|208jngN-^DLTlgru0M?_Rp*Cs{l;s|8jmEb$G{@MBHfgfIo z)TN1;V5XHB+y9G_<#N>=Z*!0$v+)7phV0-zB^s7>5V=3(Pbuzzg*h?sFwwim_M{ zq#Fm#bjMCHmaLrhsM3l?TQn?-hi;2AwN3t_r|~#d+IYr5;sYm1DlF0;o<=#3bNP?# zh*!xjS}takxiUOeieNJ04!!i+eLPk$?O6kxEV*f_%vH9t+ChQYit zCG=Ko97B>^P54j^DDGzs}q>eKl2!ojjzU9Z9;Hy z5W+ZuPWy#DuE=Wi(_MOsjl5(jE3)QS!_2kpl9f_w_@!!{@$D+JyZ(!wwzl11LSFT% z%h>_%*+w~>oaCIu>g;m35N7w9l0eK@7a5q76+APU48FFNUjj%t)b?)9MjpPy+2)SjKvm!=`Fh$U}y7 zaCie6RL*Z-G^@_*vuM7)O`ZP@r4LMzwYnI%yVA8^9{?Ei7 zrmuCidt>vPD15{Qh{XhcCvR*%#cu@Z!p#SpKX5rQ8L7Os>@(%GdoG7VH!14iMvFRl z`csQK@LANsk1c?9QO|a-2!r@aYa!KFKlWvqAW`vd5iF@3A)N-qj} z;B1UETi93%8t2on8KXq~FN~|(G2;2BD5;Ns=RWj_E{2Ze93aa3 zB71P$_^)3=@uiyxgdIj1fLc35v9pXH@C~u^$Y$9^kfTc9Skx&8G-XjTdQSthEs_(x z%Y_Pc3JYc0oOLr3K4vBQTF7Y!Yn5Ig+KnMV6B;1TY zKu~-Ymuo8@PbaHlvQiYp+g-9XgBwCcD*3q>Z|TWENQc;PmBDPuD^o=|MW+ld!+NU+2_)QDP}F@vJ?3e-Yz?Gq}MPhKGEldzBV zsWSY0g=kP@h-CQ&`AmQWQM6e=5g($yD`1m|(4_&KSaw|_YZ8-TiiqghlZOmaYfxB# zr8#?Eevi6FT?2&3EWrS5cIF=@$*di?=`aN3N+BR;CiwQRKL(G(+zAyQ_C+(9Pm*;W zIuk63PyBqIhL!`OC(rjFyMbh}K(bh$;QMm@L9)OpgX{BKi3m-j_SMHv{0)tur1mDt z9GbId?p&B?XK^~oF@CUO*EgYt+QuRG9;J^|UMxf^M?cJ0?A*}JQ$;{NR3u9VQDM!s zscS>vwQa#$U^u^$lG-^HbqH0)!*-&_-F;Gsu(6rDxgGi^*^%Cqu%%+3q^$eQ7txqt(eco}pD^XCU| zxh4CQND4a%d3LxwRlZ%d@VR!bMyS(^Y zG+OA!!;r=9i2C8dZn;yk%FWOrpWS zd|;|unpID#u`|w$QW}Ep;9z4Iy7T2-NiR`}#Lz{-AqLWwf`^lWDdm40%X%8*aO4G@ z#+jXJrz13Bnz@ksdS5URiVicUj2k8#?YfxU>=?!Y1X{#=qUgHH?UL|!ZC3-IVRk@V z4g8ZErpqn_ugeWnTkj3iN9WKJ$~yZqU%z4cxnLn#h{k-%G!LNOvNBx7Yp${~@i8FF zE*KZhux`^8j$AOszAqhpmUn^So*ul9HW~dVs+kGYya=$WMVR@Gt0hK> zltMs=MLIl}()`dlBDb2UXrNG%@=Ay1hq}R&OF1dyoe31|?+iaLry#}?>IkNJvS&M% zz!UI~r7Ua8)(a$HY!>F?A~DKJIu8r&`@-@In0rdTQ3sxRwILmsL0vdFviaT_D<%8MuEN{*LGmDMh<7-BeoJa^p2F&-Uig4Wva1 zcgyr^MkO|RUWSjBe&`GANU%MZPj&D$bp2Fic;ZrCiH^`nUYw8u4SO0klZ9?-h;5mE z#i&sm(-SuHj!pcQZ+E9B#@=oC>jp^^8)!!`wdER?Oc=#Am|I8>Mo0>+*md%)p@%oV z>u7=ox?5_ac9XMiM(G2H3eS%GRO~Hbee+ zV9N?3-$CHRhN|!U)vtaPOq}2G z3w}S|2Es!BGFtzBq~zFBGr5Bc;kL5XxfllirF4BKp){)0|WjSQ%|LaPal3u}{n z{7B=}IaII6eSCUKv0R@AkBqOy9?LA_+R)bS#c zN2euVx$PV}Mc*Hta&LB9PSa<azYPxWmm*S=ClD_7ro{XzYJ^jFY} zkBfQPtwAS^U!dRolTJ*+?tJZ|w37CbG`^^fUIBP*b${VzFtCXfEMy5wQ`5tpcZjE+^KXP5a_CvjB9~XA2Qs!_TIcqqB!{=o;Q!81M2kC!v zFu(t8gPEcRGqy^v->0_*hd-jml7Za#p*fOs!Ux)C$Z%vI0M)+sq5f*5{+N>?Y4@RR z-)@cJu{yazGzV}D19*`+fY#`>+oRVLY3gXdj(YSA+>f&N+5^{rALC@VRR(}= zJ*%xapV4={_96XU(SGs`Jm{8ZzbfVe0pGl-Q-4)Do$eahO3z1brYDPz$%JfYzUC3n z*Rt*nT{}wWqYB!dD4n14cDZl=I=9*JeR}<)j-#CBeEplKk>|djOK+;$F+*~iScG~r z$6KHcJ(Gp}&#Ehpjuf>8y{~;J<0JK@HEM29NmWB63SMhxcAPagw0-|l_4KQbP_vx3 zomWW>3p#!vHB_*Tl`yU18=Rk*6*gy1L~Bzs`s*F4dQ$(4p?-5-_AdHSYDs54XLY8` z;8wF2ffdq~;WJKVFcJxLqSl6zb8kN$__lu3>zibTCdtrZTEhR*Yahsk1`HubLt5wK z2LY#HDzZCI!@1cBm|T&R7#6C$T4LlHh(Fsj^Zm&^8_oZV7`L6;{{I;Fzr(o4WSGIY z(o_FG#?}7EW8C){r~g?P_b4^8F|O3^e-6eyPHhC^N)2s{E3iHoS1SBJ1>>r^d=uju zRPkTIxWf{9Fz&|V4WR2jTn16{Zh$P0_#ei8Z_e9!n2vu)h}KBOgJscrgaV(9R6G`` zcrsG)Xr$ulT>mHxja;78b22{%nG#OI?i4nyk_>%sq-figRLr7>JPoJ%g$SjFk{V9+ z*AP+pa7*g6%|*XbMt|>uR>SE>c5wlAfDcS@uTI8)@T}OMaV&=u!-huL109pyT2e4+ zUaZyga&y?oRveE3T=ID~-A{61aQ-@&_&WzJnina0F!mUtJJdQxLm&|zQ~9pi`t=-L z@J60)IF%jBZa8&W=;DS`6GG25oGS3#KbbrmPWjiTH*_p%IF;bJcPxNla)(|gRA#by1 zMl7L_ejx_5zxK3_PXJ88oYe{FAF8agJ&jTas_kin4HNJ1^T#Y8;(wCtIYbU2BaYhs zr?;feluSb>BmSrPBsCaq6+cLdcU#5JwhI5iE}X9MG|Ii4D#xh;1?#bN@AusQ4=8P` zckU4u->i;)s*REtRba$D_a>4e=<2wf&(UdTBK~dW_@)H=bqob&4dn=!R~c$34+9Lk zI|j=u1-Ukc@}UlqaSWz(FmKHUJ{ohH^$c}4bX?JZrr&rI|CapFk$zqKC+DUdMMO-_&`t!riH!P0RTan<@V3sr;j{;W9KickWO1 zysZ5y+;Y(Q$;LB*OowAbe{TomjPlTl7)i2qT>+3D1>bpMkQ%GQRN74>{y$sm+Z%{)=r!d zm}g~#iv5o%=NKao;wy3_di=+tt|6WsoeCJy9)dOM313eMRbtL2kHGU^y&}IlJ@;NI zzh1dc{X)*5VqeEK5)q-7QH3Y{`vGQvJsV8YcSTE%E{yo^i_||HseckzlJlb4?&!oqq{EpCSUgq>>QTK2 zw71!_!{PZ22jC~2vrhYFo#yac^kf|_>hc&v|K`Z7(=d0ORq@2|@<(wJl{jZpe8cbc zqHTKp!@B=I#3A$Mm?Z)KzOFr_mmJ&JlGsFV*B^HXuHCM{v`iqR|iVD}R=%Z_ZA_Yb&r}k5i1J%6`Se&dx5J zeTNbXIGkYYv3)p$b!hpve*SH`e_QikhadX*&_`*f;>_8SW0XSn+WsJxpeSH|=-euX zGCrd!t(6`6x-2Q>1{}GXnDH-b+ONg*7;kJ3KjGc{hi+l zKP*!57WVtFjTk9;r!ba|`=E&botTR?_ZG*ISPFsHrMpU=R?~vPuN{ssZABZl_1SoK zBfNwcU2t=fqk*CH?~6?)@7L3`$H<67{n3cOB~rAvDx&R=dBr}s|FLf*AvbcO2wi(t zY^n@o>2s3w>?i`*qHBAbjf~ipI5>V)-^J$lT6AGTY^18r%TjVMF_vL}ils@Epo`zH z&DZN2()vEGaz@F2XXhBDR<4#%DA`-S5!ExpAb7m+Zob7Pko zcYO5~Mu3fncv%*0Rv{Jw)4bz(aC9C{Q@@-vFCpY=ad_p&KT3ns7vSFFA;kDMQcupL->iQy~d;lnc0GwD_lP@OefdzWt9Jcpb&p1Beps3%-N_|BH3 zU;@Hu3``4?5RIW-SSI$j31ipOco8uW$a)Pa?C4Av7sLE@?iWYIUF-c~imY#WPSaJ$ zmV6h+H2dS4ECI8`5x8OErM-Lea4b+yqof(I+F)xj4ch3dNfq(X6;a&yyeblLE!Y8( zGj2+&-wpC}_=?>1>6_0dWxgbhwPQz`i$e5$PfOQ{)q46v-1}m!@dTpwgpSL#)}SYa z+uUigT5&xON7Hb3f5r7imzD;P**NnhKVoyj@x;*0`c;n#Yc{T35QP7CltkQ8m{!aA zF`XZJg%L}V0pxb;X#91sNX1cDyYbgip&eknMqZ{>G`V_Rs95U7Mg%nxF;N;mp4W$X z6X{?kfLP!xEqnR-U%;95zmL&O2<;#Y$>Vy$QxQBoRW!jkd`kF|CHr8FRS`TIDQSCB zKeZr&?;q_52yCeSPQchqqw;XN+Bo#CQ)DFslAV?_(&SNgIV{ zrZBwfQ9ac){`Q5`TzXVyT{lQrU99_$idBz4IMj^`C8dVD2WwRw9oBbaAccnOk2=bu zQ<6h;=TRd;`qtSkHvey6&gf44Gtv!%TZ&-jNvYy0sbaEK#bZ?Q@%R5E&N8*jj9veB zCjNx=Yq(zVK6QcDf&5Ns7psvH>}~N;j)G`n18*bVU2UDb`f(4L3L8mIS0edNzOKv z*4f-JX=_-J*ZyYzzBPF~MWP?$9ODImTd;Dd7ul$$otSJtv~Zp&PMH zi04@C80nk`f(dhtswT7ZkE_nVmUhvi#?R?Fnq&No1#He4<2HifSFLgA5Opr{z)Cy; zN9;Ls&=5*s7{iks`W2e9Fm^|L0!A=foVbQ5HNJYm8Yj;tC8gjukT_jQ*A+V9W*V|? zCIs{ZJNo|K=vh*hik?q&J9bO5w@^(Q%9H^DO%}>9=o4gk|D9UFmd(aXhS_HXN{vPR zIj_K+-JQ+{XJ-uO-^byk#&ak&f$!zLg+4(B^uz&rf(nLrxytEb?SLW*rb-W^rFE$t zU0YMVn>7Mx%;j>aCh}&LY#{WwMAWEp&?2ZOeXP zTPu@O16P7&_O0Z!V17B(d`B+#b0AatUha2OO$?DG3z%x|?5So8?uz4~$q=ofwpfm5 zhm&EO3u^iiN1h@YCW-KA4CpW*mNa{z@_ZbZwsnX8=-KfGpzMP<)5HI?vgg*v0k!Ey zZEp0&QSNLU8G4+}L-(XLmv;m|`6}K6O|hXj1C`6lROutP@e>YN_Jo=Xo#a+Dm=Ith zOE%?qA|%AGx&id>QqEg~ew$PA=U>1#Rwr0ZD1ryH{Z{+JZpOChAc4!&-BRlcbBfnbxNB8af^14V#1C(}|NJu2N z6eM02bMe>9O&c>nn=D4sS{bU`dbekjkYG7Yn{RL;wXMz}a%}iP)0wtW{axP81&Qc9 zwHCN0I6988_sH2{(|u}`(?#8+p(>KbT^@c6TujUXA_tzQrBt_p2Vyg@uiv2DL|~a- zOGD3QLiIuU(4Xj|`FT<{Rn^P#i)$tBuZmnt1T=##( zfeGS&gTrx!|4Cf3IUE;hPjcjTwnpk7o6Y3sbb-M;U4w*gj8x!8=tQJ~i>GHj;dvDH zJ4sQMTl5%*Wly*eNR_YtLC?KY%+S+v)H6E#@(b+jopF*KMNV+_&uYk~X&xcDRO3JD zxJ)92>mPGygJ;+WJ)-p-^tPJ^y%P%}N;4^ytMsH4S$1Td-OFc{zP9rG|KfVGuSMSAPWqdA8`HQfC9 zx6y~w^!moelFm@V3@k{Bp;iusO^22@_47B8w;T%paVVDdNxW#Aj2V|WTBmkEEMPWm zlG@G>d8+i5s;j|8p|I$S8P=ij3w0>$g3ecm!cXZ}>qOXHCn-NotACP$+vRe}|1L8+QuG0nx++rt zQS2gh@O$#>2R}m&a}i>yxv$qh#I2QRtaXVXPto(SWYWvh@EfnlO+KeUwkprvKVxR) zGXq#!a?$mc-ko!bscd=%l?h${(kn8LM{;-EJ@P=r-{?fbt6SBUUas=BncNKB--uAy zqn>b%OjR|{g{er$n&-mg5FY{m?L0q6MQYN{V=)}H7ejiu{)huzH*vW290%5-6G6lS zw#~!WdK27?`gV>p(J6@fJ{YNg8Vyj-jug*t4(j<#>_?+hGJM_4=`VV?7>`EQT3n0- zy&e|RgW|*5NF?cOa$q+99#(|z--=k?>2clX5zy094EldtPhZYJ#J8Dk#v`GpS$dozxLcW;8QWP3oa)Qor=Ale){F#fkVn z-T%B^pNT+K=<0{%djEN_z~zD;6U^?*^*8ixmHFbdcyqq9CEqn+b9qw7)6fDW2@aa) z>t2t*16di(A7^a*_zY_yrzz-;)IUnyKH|W-5R z+IxCEcLOh(i;N)z`&@JKFKiMF$NJ0kikI}ghyDS!fsxRTPuoKufnyHNAt@P9-@lTU zP0?JUnlvd7rnRV|)jz7UCo$DL_qUN3cw65at=~-*o^l?3>M1UyxwWQJ;z!Q^Qgk9h zFKM3u(Dypys-|fN@u%zGjUV0gNi1gH0cL$$-B9FB^jCd5P1zKYZrHR3(+Ne-hY|{f zIl=Pyf{5L9358CMea(8w^ZEhuGloY>I7+uiGp|cKNF}1KtEqJwr@O64);AkjbZbRS z8P$J!CiSKjEXFjj3p;VgwSw_!Uj3t6@O;>jIOVE!{#d_;++5Awk}%gt)auYd8X6z@}AANCP2OA^&Bg1_|QC7G~`auk|bpp&pxKZ|90e4CUhhU zQ~hnz^U2~Lo97WIq?{)4F_rXu104D_j+is?+wmSuW1AOJ8t6un%>`$$BTqsxYLy6 zTsU&?g83Uy5ljeWsdpj?_G;%SFP#;;!fkmKyZYxf^vGzX|M<;%4Ey>|=j_>(QxF7ct5VyS5dVv?5DZTvtWd_AcRklYd+>I+>2_IN=$Vbe~t92Z>Obe+Sa`{e5* zKS4I66IVu!qMuW0Tt|-*znmTW&E?%R15&9wr^{@`0@aHBW-Ioat=ONaTCv}3#eQi8 zBFFmPk*|g>rK_ak!+4t&e^nd2alfkg0NxZu75A&kzpE;rK@)+}05Vby;GM`I*tO>; zDsxlc$3+3YHY3vP4~IM5PS1aW4E-l^_B1xFmG1tfWTSfgm#PZV>AzH!kbb8lq~Cuj zCEH!EAAC_ppZ>m-N_bVZzi{O7mi``W^|$n_S#8zl!MxEj>hp0wH}tY5FlHz{dzXq# z^9Ep1C$a=T2+lC8!XyxU!Bb|%Rf3Y3Elz#cP zro5t>^2WC`<&A&V6xEP_tsTv2`*T`;Nw-^_hgWiZ&Fg)vcCd-!Szf`K{R0fj zK%~+@YMCW$8Rus~z^YmDgd~M`@_x=Ajiv8?74O*Ja7K={Ff;{?BF&p&-1>VNN>kTC z02Uji^MIVQvlJV}1mRu#Y~D(ys)JPEw3wC_aevtBqJWcKZ2UD1jW5m{ml)m;glmAw z0*7(E3W}Jlk^Mv5`TRvfx8DaAKg#j{0;rt@fs+b8^!u_{55z=izsQR&)-?^ z(pzJTjqQJv+Kr$%Ns|(r<7-(DFGW0pble>3Nom;s8^Z8^tUp1 zAw{><79gHXJiLqd>@gppHu7k9Xz zE2k>xxHZZ*O!;Ks3=L=jqT}$Xvp*TSlkWFy28B8{;8MlY_zIzFS!(!88T34ib3~iP zVG13_aGYoS0r$@qXY&Pux=8T~AEMgPz+^VozQ%Nrvj`4^HZ`p~XM>x=2f5WE=*vr1 z4zuJ$nek@?xH-H_Wd=QxE%8-V){NZH2FK}w*iPdTvJX#6($b8%s!SPo7^(d#xYz0| zGdxlf6Ym6U7Q3RiQqdU4)D1Q3&@rm377mQeNZLQBuOB#U(_uqA>!;*E^IW77=E}*V zGao_0eeD7sCazn0`u`1Jex;NVWL_rQOSEs=jd}HT7UIHHH2P;)xsYqR_RX94v!=y97zlBO7#7u_9ILppgy%TQ!Yqt ze&>f~@Bb5}Dp7Yj?zM=TD!n*;-AvT5WLAFaS=r>M@~k|Fkk{xq_0 ze7vX1qqXr6H^%#oW>(F_aC1iN5Nr!Hr*Qa*))zozh%u5xAI==v92=pS+B-v8vR*ty ziv8U45*(V-&bHVaEG<}-46UvCWKI1ti(iRn4!A6yMkVj9SpP$emcR5YhOH0Siy=h+ znM`)2p|&9V5PmmALv4Y{YgZa-E#Kg^n;L4Td$`!miE~6l&EmJ48fuE&9vj$q3CYgE z#9e65$>W*DV3smgp{MZ}-xSv!S1Ln$tNPXbR#lb4uxzsN&|lPQJ8rab*UHsOuIb`$ zE~_m2FTwmNe#DA?3DFPNXyWU2$CSD(!E{(WOE)AMvlVpp7BOfrz8jyn7 z(P{qXYGww%X$J&2OAzinmd*M!pX&zLe_gU5yD?^O`&Y5)E?dPX#13+XiyqVSth(00 zQzbT{=!m>(170xto$TUIh=!19ZEqOV20mAwpPk1U0DjEr$iPUkD^l#vElKy>Bjz00 zL`yOvMer56@f&~+2u!PHb=HRY3Ynts#|Ho>?oK&+b1R>AN{+buorj%WK+-xRxD|Py zGw`WMoy)m1au=3ea9<+FTc7`ZQ^McLe7R6>!Q{)z1mr7{!eDKYNG1Js;B&c)PSr!FLSAy zfs12^y=Z!(x9!}>U21iGDFf+Er0q&5PqJB55eT*ar?L0IYgrMJxvCY|$e+94ceLHTlXeA{|jz) zJhJx1!p|lQPoC{lM_sE~E_HgbS9T$zNw>Byl{_T8=>lsr$g*fk3`&m?17#Nzu7Z$DkDl(cbm*G<#rHJ zjKp<+4tI~*a1?sTvm-b99WM7DK4nEvqgL zo)?|*GbwG=h0IF&I{c5st!`~0JCT#l&w0D!5o!=TubgPy#u9P0GyIV^ymJpDL<--b zy1igs4%M^{+GDVmgFazR+oZ6f|UhtbOl=s(x0I>|W>}qjOiOY=-p0erLm+y|AZj z*vGWk%JF+P?Cl}-eJQ)^YQS`043HIhnsoRf1l8=FXVB06zEht=Gy{oA7Vl;v7vvEG1xgl zVh#*9PlzS`?tInPWb+^LNmSPN#Yb0z8sekZ@b~{a|7xB~am@Z*iX-@6DUQ2&?&tXh z&vu>`p0{|8^EiH+;z;MYkY_y4G@eqPWjyP6w(va4b2WKg%<~rE<2>Cwu4hslSv(VY zZsb|QQ_HiS=YF1_@;uG6ljjJ}M?8*aDUauTo^d?a^33L`=2^pYKhI-4+jw5)d7a1L zIl+@kn?~_W=DC^YZl3S+Jj=6>$KXlcPCIz6=9$G)%Cm&0mS-K$gFHXy`7O_Gp8Y&; z@qED3&6B<(#W9R$0#6~&tvo@Vbv!@hd6MV1JZ(Jh@;IJPab)m};+euTi)RT>i02-j zhk3U0yv%ci=L4Q@o>cnsd>;At0(~j}0x}+7``30$zGM7<C+q+ zEtuwb{aAia%rCE+=J;Hn=JZPH@)e>5=w6=0_ zXhrQ>?-LQ7Xw7O$-fRW852rfP-b&dS=F%4IoMN^P3=>&jNFzIr0{TueRB>dV^7MWM>CclPC^?H!Za%H=Co zeIt%^ym#>mZP^lU&5Drs_Db)f#gcpF5{E;5TfL}GeW_ZZ)d0({@p`qIJ8M>~uJKmZ z)~=}aj$Yzjv3RjoTe-x$x>|an7IrJY8#Wa<#+{$%xR3XUp?Qvnc_&_w z=XjF$7~T&z7C1&We3t=OT+OPKH!DyUpcJrhY2DQmZ(kac@X|W-J9#c%&6xAq526EQh0jy4NnfqdMdrQ!v(hMQ-lI%UFw(yfToW1KfoQgGQ# zOIJu93k&81%5S=U#>|@?j+y@a=?WIO$zPOzgMVSE|EA*W=gbz^*?E%PmE?Ivj(66g zx)n9vKxOTnOKa-9i)xm5L)Dd_i^}DT?v(e6ASh1CvGeRx-f9!~I2O(+b_DWoo~g>t zFSuEXT$q2;jABVQCm_Ei=rFNbougP`w>q!%(PZ!FI-#T&t~85WxO~yln(@mt!6|l; zcHWXlNe>>bqIB;!lKH$^U8VPOt0?aZg+*Dsl{Ma_HQptaw`+HJm#+<4 z)>c+k)(SS4b}w16SX*9M!#IH{y<@-tw`)s5A0%Z4s`!$a-yq{fOU90ONbN`0UH*-F zNL^KFyqTYiD{o)4_)fxSMU>xmBMasf73I%X;|JJLrPVA}l~_?XzH&|FV%9;`n8nMN zjPs5kKYnZ~A@5YLIbp_E2~G%^U$vT*6f|a0?Hyysrq))5wAvbPZx#|~Y%23Eb=iu= z>;yfftXx!ESGjQ6l7)3E7A{^C$oDY$@67dMlpvAmgJ@ zh;(<|n7$c|O_g3*IL^CDdS&U#Mf8Imp-%OTH3%TfWtHOvA&wEqIM&wJ26PqkK?c5Z zSzTqS8J$^NSsStvNJm&PBo>b~dwK_`&ykghnuzBoDJwz|EC4tHz9w^-rfu&3m(~UbOU1 z?OQXXPUKm)!pu?v-GMuc%w|-{ik!E#+86nBi|NWqSFI z(-(eQQC26JeekbV`RiGEnH?iWeA^`X#`ur8oE`7-WlQLf@v^u_j8*f;>TUHJKi)fH z@(2)&ca=F+m;`6d6sykWP>r=mN~JX`Vh@knZdtl&p?{-@%W3Y_6DMD;)zzB8%O|U# ztRE9FLY4om344-R0XvbI#9A&@%R_w@j8$#bUUTuuoTcZ_#-Ai&;h#OBYb)!tWrBH_ z$Wk41OCKZYWsQZZ6)v!1{XguzX?Gh(k~TV@`77$cRugO!AVtY)Tedtck(8~sweX6R z?LL%-2LwS976`BbP@=};|9+n*mRza|3*}|}-rJgK6IGQN85tQ{Mn-0`x3CkUbM2K# zoXJyAwtxlJZU+7AHnKgMa6n;BrTb}l2vHxJ+kiJ6!u`SMeQyjxL$UTx{7J=FdwO*5 z?xX(nyU}EN@Zqv2`g)~t{lz`JZ>+C0`W^k-xj4}umzQ1pLCF3v0Xy;i$}cN9RyNQS z93thORujyXDtT&Y>60Qww}@5HQ(_*}gVy2Kk^*@7VtaRg|HVsq72DrEIoRDkc=d8O zssx8)LR$tix|&{IO-HBAMz4E5Dn9-0#s0y;KldEKN^CHe#M}Zb(z;+H7(Xk*KTc(r zx^vKW3fd+NS0cJF9=a>F#d>+28B|KN?`ennnlF@ef&3>Uu^CNewDsbmGkLpSyzdOB zVn&27`R=|rrs8Kd8|@9(X#4~sYYCXAJ1Nr~!!d%|x9`?6;aq}4t&&>dTYpt+=pSX! zV|Dv=-^{O<3I;fqq%2V?(5%j>oAymQ;t#?tZR!|Cr;Xft!;tnEdVMLE#? zpQFi%nvudsUbyfhNe($BjU0L)wc@9=0F%z374;yiTZvrwXmS-bPB4w5YKUo--C>*2 zU#CMYd-7voVDAgyPMR`(&~BKGQWxGVr^rS0H93!<175FfL^in~>)#i<@F@2l%het9kHwoelJtq#|G%uag@qm*3{%P|v8$aky$n*w%KFBrc1*<52 z#^wDp3QAj2{cUbzLDUq5*VPHi!#@G*Yu17E(8Su^LLa00-=VcOj6m-vHa&0O~j zHPrM#j3WptuucUx(9n!`5GJ;NnKVxEsWuL^*)P5+Zi{-vPp}f(7Nc%0Ll$)o-v`wa zjL@Aq2#parkHqMn-xhQfQkaT&&$q(st@SOjVe$^<3-h7Wkp+a=t-)~jhK#Yt7R%kA zOMC5NxqoxIR~_n<)Q20dU%yfJtUSurlEx!aYF^IhU4POg7dzXkaEAWMv;wBSOWoL> z$bT<5>kj2eU-R%nEdv((X;(TD%D@zLib&w38HyvQf!``JIp+bn%gbfZQ~Xd^FzjAm z>)#A(_Tb`=M*bXX5#FUKg+MjsCa3PX;CXI;tP|(x#2-j#DCp7ExZ7KT+T-pYhu9e5 zRoIcs^|6Tem2;2D;pCUC|QP)e^~Ln%-P+Ot{h#1V3vQf-Tca`Ew* z2&`mhI>p+eM0WT?d%|D$&!{sS&BI%t-D=-z-TJRvpWbTTTDx`Y%^JD#)~#E&p=qps zrP{5(VxEX%uQxa67qdcJb6^P!3Mp;*tOikNi?Ha9v7Flw?Vsp4HLuaWS}dq{eK|mc zzbPS9p$5i%08ONGtCm8BV65I#=E_9oP{^jj?k2ABdz+E+x=yF;#?c z49AJax2+*GJ|ZfD0fEzt$R3Wz@#?$^=df9s)0#3`0v;rtMZisPRw5cvv1R@@x&I@CZU zhSr+z)+F*HnEFDEwfN55!TMe(`XnR&mo>YvPyGv31wz>49?*nY>6}6xt|OOV`o7nL zQc&&>ln%l88NIP^okT`7F5-}!0W98Z5HV}90@8udPWqXMNzk7E5&W!d5{TCBJNZ*u zktWUQF#==4h!YVmB|7;Rb0S7)Gs-hYZN-zIr&TX2s1ke|5E`!(iyVV6UL}Mly^G7~ zhb9z&4W7ixAWuLM6Wh8@ijAKib4Zs8Jl^4qr?6|fpN_8zbk-dm9}sC)f4zV?w27eW zm%6^Z?R}V#P|kIHhIai~Z3{xQ`UuLs zT^Guwnov7@`1TF@Vhc}X56Qr3Hu^&_R{un~6Q2kmrxB1yTk{k6RfW0jDWA8b}Vg{UTd>LQoRI<(r6=O^;H8a2=`7ID^f80 z72Q(6l`xW+AS`XbAyb<#zk-k4@q zrd){suEWy^6i(zP?Tb#}9AOr6h9Idfk=-*rZC*@}!6O<8qsz8P=Fz}H5lWG&%1e9TLmr6Ot>uM_h0+{eb+RKt!~n9r~f)-|4~C z1>tLG{l4Fy*WMhsKOT>+@7&v*-IZ%Ca5M0psv@K75iA$-lMw?SXi`{D#I=?SxMT=xBfY<&(VwodM~(lPx)^rZT;+)i5cvm2v*ETm(~+ofLp}kI^R@D2NHs+jRHZf%v>b}N?X{x(SeKap8 zz^E7wdC|<(wV0Ju6|17m4KB?CsLDT)ch!~gK|torg95{LfdLk8gYGu{kPEoE5(?Lc z_R$e5=Crw(+Ur0F>#&cbw?*41^Jmx4n87mTGFRLBvOPeKxqV4!A{nY}E7CbRX?8*v z`p9%2p7~18X}_l_oR&ZLb0bNxXOswOw`XUtlfQ(CIn8qCP%6H$9$qL=;32O=2Zb8? z%}Sc%YCU2A0g1JR}ACFC-b(Abh*m>7FA9rGW{g84vx7j(Pf~IT^r(7Vm~@pyQup)}iUU$Sy<5 zwgbG>^kJ!xfVS5WvX=v_Xx`ZXh}&sf`l}bWN9aA?NIfRb+y8*ksdQlkgwbnvhkrcmT5*xF4IJ%dzt>VMEP9*>dU$M*NRa8 z;-@p2>A9I2xR|Aai(Cf-~4JOT{3Mi)hlwEDd2reisdqG{1y~u+kzV*t4VY5U3~2&kE?5B(SJKA{3M`ButYz z&zJNDN`2=5ptiayGJ^*~6J6R%Y`B9% zB_9!I&$7yt>4DWRsi-sCehdK^RU5#rQ11;&XL zxHL|OE|E@_z*hi*uc7ZVC_XZ!1Y?zaCMG3RkU3_p3(ZrAicu_p378V*Fo1l+2FMKh zhBd%=;X(4>xo}~N)Zv5o)V7nvHaMvQ1voyT5v(IqQ5;r_rRbUgD$rvdB zlHR&66EuS~o*{+FC40S=FFh#2fgqufle;a$Awym&s%c~CwQO_`O-FrqY~Eg7duDZCVfv-zt5pPmO04s4_mfaory2eTk5X1}09El-x5^sxQtL%Oa)5Rvk?mjR z@PPQhQ$%-#BM^&cFs>&e{`1OcNLWD7du$|6+~=$@xLP(#Xm-|*!;GM#T=I2sY}6fs z&8YZ(@qD*hyI;Zxz=!Ck#tg=z5mu4=!~V428JzSkx!FWT8Y$1f8F7-469);pb=^>S zR@d30KT*o6#VMRE1zVIk9b)z;z{p|6N_ZB+pi#l3@Zz6j)Ik#8rjDUQI>zSQE&yTK zR6hv3%HfPV4ygW{8ty5SUBi`rP1zC(_T56VQa`40Kq zwNa2zN}uSGI0N(!@GtsKn~<`h6;Buw{oC}ROSo4HDr+{u%gWp-KMQ#yon{m=A0qV^ zJkEfY(?Gw!o;zm=g~W4^dA*VqfenMF((lHj9NXXw0j?3lAnu6-m@yUrP$+^Ntt6I) z7wbS+V`mQHP*--rsyfORF*1fE?^L1+MHW^7BN6wg0^xv(0lqGjnP_@r( z#(_^Xg_NdG8&ct$%7U$-B6gAD2~4DA8gK7{BIKMmc}GXdoJ?@fqjA0Y~F7ODXN&cp)3*${A1=(TZ zDl$;NtmaIGc>F;@a9|C}ksPd`)SMhn(No0c@wqKLB!v_=?-y{CPm~5&Q5L0L2+n6< z8mCe~EilHU>xn1O!%bO+k!kvxOlig3d0<2Ftx1kabDpd+Jd(^*YAp<$aXwBmLh_z^ z!ZzkwM5iAbMzC0g<`}R3Wp%wM8h`0rTz-Xxu(}crE%C3b(R4lh z2ntWbu~hvm1>u`n%3%VM6EK)AncGraGarZGG6Jv%HLNg{OaYaAr26WX5YH;sd|zT^ zLrG;PO3Ja2wfXyit!d8@f*|8b=eqh&4E%o;Ofo$wDw-;YDq41&(G#YpK?IB#j2^L* z7+EVMx<-?48vM$MMTxOkKhQsj01N+&nY1^ilnhWq@@J3%VA6z|B#Nxd4VwuXVg|s9 zD8>I`0p3Be`@uH-gvD&v(kM5>3k!r{W8P>qUF$rMXmXBKC{@`&!x0mCRFnl#M359^ z#Z^<5L=c=1k~i!GllUAts^oIJ~fELPhcyJR9D1V!wkH{LrE%! zYK((RRyGHUAek<}*Z?{bETo6N!zQdGhbojM~rhLxg-MF=Q-Vu3By=CHbv z;STRYHd~8Y3UPT&%V#}#kJTBFPMPHxg)n?SW3sNaz-yuno}nnPkUokeyB;)3AgfTo zXg{?+yQyh&k)FUmye3F*i}h;ChSzD?Fu2{$ki=-}lnbFqB)Io1p+|qWrKjAHzAu@5 zPjRY|FnHxNPetqUb8g6Vjf!JI4Qr}KwqypQC|d^eTD1ioJ4S^vU?-#LFqvh~g_oBN zAkDTduM3cVdSSy1PFC&Q=_n8Drt6}=-Slt$k@Dd>q#w6PB|;h+d2p;=E{K_EaqU@D!j`8T2M;(?M6PiR_t=**G;sf3cW8)(Ze-`MSwLVKz4ue(@9p*W$fiY^OFJT=8CSwNwp@H!_)*YvF zq;gJmp6^zsAyu*A(Tnzrz@o1MI!S}>qXmtgj|Vk|^ojb{7_i#7?;hzj2@_LJ?ElrKIMP)g zS}hy`kwi(s)lQyrlq-Hny&GcVB9=%^j=LNcFH;C*KIa>;f@;4YAkKh*HXV%y0w^!h zz|_5et!xv_SyC-fRFOy+)3UHa_8;@bxNe_Ifngoqs{iY6lg3GxIs{ZlvUNCoa`1)t zF##`if2GG*D-dfSx?Fz>M#qyAQR2xf%zZlJJ`O?@&hZaL1AKdB)KFuhDn>jC{+wxi z6S0H>nxPjkBHmO&sI`?~(MZ3Ne-QdCU7=hxq&yjtBae-LXOs5x(US|N;K6TLkuDkD zjs4(G2%nLYql1IibOgkv)}WT#+6YU@mOzTb9u$z$PqTu#9MX`Wbb}PE>65g?{sf(# z&_Q~nkN}4pMnzF#IJ6NW!WA3w2Q`xoIU9W2c*A)q;VK_VNqghUh3^ay5R;Xk91z}? z;jkd(%T#Elw4$ZP6#E7Pojc1+ii+!4ja>t{WFWs8+ zZ?rq+4wyTY(U|*&bWBnTe*}*VPHTANX81vnLk}P`L4x&VS@x_b#ZQE3#RMo9T76lr zYei94f329L<#q8=1Ayh~;R6K-4<^B3uFFL^eGs`9&ZAm#7PLmKh<#SgWxWEQ=7kwH zV@p;CY^rBJ5iB13y;V}K z4qmJ#G*^`HTuYVPb7Z1NRxaV*S*fAWjY>^Nh3gf;uQNA9fz0&7g6FiH@%J5K?QU1) zD|g%ulkXbpR2yb#_%T3H_OIA_mibP+EP1`uM70=@LB6WCmltlIb-S5U22SFk^lhEs z2XkKW&D?y#w@g;euv?8P)6JbPwb^Tvmn=x-Gmuivi_SIJz+T=*pbXn)S{lC6Cs{1O z7TW<%edl5F3_r<9GLt+=DJaN^zX&M#A_MY#t#lHq^!idtM}@1Cj)NS<%FeZ`NBM}$ ziNZ+0Go`9LEQk8&8Lm8<$4^qH`tW0btZ&Jg<;-{DXd^`e>rvMVP?Dk|GCU`UiF#1& zEA~mK>Ruk-m5)`-0BHe0+0r?V_LrBR0CRX9Xg?vYAsN!nSdLlCIQ|^P(}H?}ksd^( z%K=u3t4(4G4PBw64?f>19%+we6mh1K&tkk4odgIh3MUlWsqQ&;l=QgbAZO;-Bk-0) zZFE!sX4Hx&_= zjotrnaa=ZPv3QvkD<&!Q1j&)GVlWB_3^FbcghfLW@w&Om3OgnA0vWL3Lwb4=NTDRa zhU&EIm~zKfo}!f*om?L4Qo};`z^m0!$lv$~l-`p1!gj(g|KYrF%xh;-Zm}2XcjKes zNUqtE6CIAhTAJgi%5!}M3%2h^IMbV+7j?usJbAgyCV3N<2}2p@QNV@{!=a4$f^!S# zcOZ`wXY5K=l*03}Z+ksphd37wd5}{MY-Sx>Cg0okX0KXd5f~hLNAGaw1>!ZT_J-rV#fj1Y^8D!&yxc*xrVVvn{j1fLTTD|^LSfkDC)|A z;7KX7bIxPz(<~)pBPDy6xQC0whLtK-(Zvsam6<_Gr-#0+4ZdZbHZ~-oL=MywE zYx>gbi8p1#615}>c>#tK9YbTi9OulcqoQ;V3su%BhFCb>X{DM*>4Q24`VADMt%_El zZs+8utBEeUHeFPan|Ta-TvLV#-dI?eU=@`#?ZaVDs3{*f{E`RGjV;w!TxPT%w2E`FB?o9e_6zX#I zv_6;%yam@&0(&4mX4jqw2g+$;AGy0>DQ9GHe@(9;%S2OX{4WB7R!VxNx?8e7joSEzGY>^3jKA>GIxP}|g{wrRqfE$D5Mr!V>!OVK8bnS; zQYV>(Vs*Gm!Crmyz`q8ZCdqEsOd7u`8*x-@gKXlhxdB8HAs_~o9=;3zl$gCc8Vov@*uM!9+n~En)_5tZf;~h8@KgU%MZ3Sw zY+r1j!T@=qDR~^vuG#5Ip@ffD#WR;-2^TL#G<@({t?79mr%ebCA`nV=&L!xSkS|fp z*t?~|;XAPsYcF+=lmgPNilI|_p5XpUHlsyxmga+ZvXx;0s#-kJ5H8Yd=!1}<+qo!k zAj(K?&y^0rFFfy!gPC!Ckax&gAq{WNQyw$%OnQ}|X^jRf1w+9qCv@hY&>d4Gy5VT8%-Ek|g%gmTV1G^adG z#J5?hFoCHSG#h6a4D5u7Yr-my7d#RElv5yr?}<>c+@;IduBBfBXs#C9;AGVFIEsg; z;#+2@U`a5E{riFj)#9mw-tvA3F~x zF<((-lhrYgtD^aLUrkNL#(x(@Y3MBI53 zgYc7_-18K3LQ$59jnv~oA9EfAH;Hjz99_1iy)=)}+(c_UkinXTO*?rt>FtewQp-k! z%^+dZ$8&IP688XQ4wDwKDGbWehiD4^mhum{S%tsc`6NtG9UFD+3?laA(96H^3(o?e zbD%R~GBzm1x{4~CWkEFkVwmOzHXT2A{(L|+07i7M0ty>T`ExWiYNwsBQHv7yj* zV1-CPP>P|nNGdeFJDiR=@%tHo*h!YL0b0i^Hp5p+YQB-?4fWV8zc^dH7!C;0ANlDQ zw&`q|A>=5qKo)lp{Dpcyaq=1{xyF(ynF6sQJMLwUR0_#BvDKmoObR5-t`HMUo&^rv?O45J)WHACauQ1$Q)@j@^#BC%K*Sws$ptnbUf~d2 z^b%`kIGax4+PEN7nzz;!!U7^2gVE@1VWX|XX>nzg`m136SMQm~Nj*2otPIB*b2F)d2gN`esNMCpQ=ah`9FI z;$zyIFkEDT{(8EST8p+!Dl(^0Ny^?iG2YNQ;v6(=Yy`n|#N6bQCoQHRBjR&_5D-IL zD!82LyhBo$s|ljArt957Ue55 zRx>GPsOPyPjz${)fgC6*?vF%2Dq^IyWHb?X)&$jkl9sS{f!xR5SUm;}<)z#uj)1xWVfZOO(aPPBlog`^ z;;xfvKP{9iW~&0#U&Zc10oiB=%gD_E-Lb~ zrN+v-K9$Mhg;LdmzM3Bv|4}jE4wT51vldZ_aR}CfcmW1JgaiU212TXW2+H+IQc={2 z3^O)?GHTcDP2|WTsT7GI?6FkOOqnh*>j8J0Ldv4e1OS9p7cRhD$;kM?p2qWS&O0}X z@zn}NbC%I@V0qt{@4uo1OE`MhOF|*+6&7ZCizkC13&&RQGrgS2bp$ONoCN6FS5R~h zVaGAbVjXKM{5ZPRagVy*ioHQm{LT8Z}YSl{Q6nYM#*o1Gx&4?DK&?UjqP?J<{2*VZ> zZEi(Lqhj?(W!9hpbo4zBAZf{8O}L~S@RNmQ|Af5|`SpaN)usD#&Vn`V#dR}U_Suu= zWsG2B`H{$Ff^&v()hjr)&9v@d_YGI#RUbxSGvbA@)etwV!~Eh zt#{oWT%Ev?E@f}q`-Cf5+urH|R#H_jonx2mrKY%f2^E>rnmlMBx+X7_AMz>KT1TSr zhkg)gIVY8pysV>!cGVrj?T0)#LJhT2!h7o*c4($4$kG zyO0yx>>Spc3=%6#Vp7aVN1xeMk7`+hg&q2eTp>d^H?xFeLY$ggj)6NC^OFSwBJND3 z8G~$G40{}-aC9uki1>=UXLPW8-&p|l^MPk`Sz(L--bMKFJZSa7v+Xdv091qpup2U& zUtd84U2Y_7KtXYZ2e1^cyeO2`#3|-@gOIe`qr;U=oP!`Ym*O>k zUVh-;U0jBH!XMAE*w9${h1a)AJKWxc2)-a25R@2KT{Eba&G#16^VuXCMornthBMfb zN+8C(wxoun$2b}4si!pXCUE5^_ch+;01Z1=U*i{YcdrneySmQ}p}x z`pQo4`08vIL$m{~f5wZ}OYffyvHpRmgOuJKa&>fnf4B6wWzW~hp826O9zrfaL&ATI znd6}JYB1_dA9o}I&k}sxhb$q*K&m|J3^C`V!NJ#OkGJ)Q=I#Bk4TESeM)&SM>QA@N zd)#g(g`bahPx{k`!;@`v2B#ERU-0_6i>pFZvIzD0=<2M15|_Y2=ND$xB&wkqatWRs}8 z+TS%e8aD}psgEzYPg-&*fSvCam3p|IxvMwK;LEGwq;qN%!l5zGpz??il#vVSCdiRWVf^^+*7g~QtRJFRRP^rYej5l*ToG_o#H@fb&*Vs?tkoKu7qqt2;F#XX5b23inRUR>)fv~MAs$i{&x^^_ zVJ%%sRp~awZn3##YQdl3@lrQ#&CZCqNj<$i2}CJb6koWiGwN0K4#T9Lz1z?QJN{QkxwCJZ&8tHA6Hu>#EONGuEpS;V=x~n0%kOkKh7l$`b4swG+hfvU`4xRSB9t?ct9-``v#Y~g9(NxYB^F~;!e3UKF zsyDR-TI=SvK5-Oum+?Tas4Z+L70<5fkeCHhc=qKcJ(zg+)vO|o2D$ZL?4te7Sx*X=8DPt-I@6OSC{rjNWQL>m9>aub%AaF*>OcxQ{Esdt+=B zAu%}mOrgt0zpU_5RV0TXCFQ|gP{{FP&-G0 zN@h=Z&HQ3#K2J!oPFfkCn@WO(2BXsJd;%nJTpTZPz;U7ozz4@Xr|<$_TzOqDKi1#a z8;r|&gxG>gIu!aA!{A-PI#-Bb>!y)rIL05NpfnYHF^dHW!aGS$(s#i;8RDS~1j@m0 zUyXTb>2?4qsLd}4GikZcf;Na&YA=kH3{?Q8OY0W(v*fGr5jch4VRveP&t$IfS!FKt zPFo9K0SMkvAgC&li^*A38s1kUi||y5i1rE90Iq=xuJwSLkRZWDJ@1I0{m?8?VG#_l z0yAi4w!u|O+4g_~Is7Ojqq}iYoCk*9<7Ogsl>%MiB%I>Y;l`O}m&1biB~u7!#%!m4 zB6y~Golr>P7&8juOalkgcv3(m`q*n&-%)KeZsNb|1rJ&SRwTA)ZZDaf7+{q`Uy@H~ zoa+dLptyzJogk(%>5c8k44y(Dx5dh!l50fvl=H+ha~)At1XdDiAi^|wdXp+!mm!mq zWuh$5W9+oGAnPXyX$)n8SLe}sR7Z$-ssfS0|JczKZp58_{(A9w=KR-ztf#7-SF?a) zq!h=(cl5!1!=a=$Tw#znImo9ZHytQBf}_0NA)%8DHV^8k9W#(gl#!8&D%0lfL0QQs z^C$l5Y?46_y)}yGEm4@W@l#qrjG1*;os{Xo2tz3|G@cBB13WOngq@}Ry>A&*h)^oV zOh(5MP&{iQ=~5u5a48rg!(1S)7dR%-td5KF9HpS>E0$p-lX~za=W<7A>twVXGht;? z3UbacyIF?1f1h?Gu0FeAcsD}ES^6`aK^26otRm|4W~L047ZcSGwPsc>)r_J|LX+ti z=uB0a8S`VUln0ooGE5r`VMOWWSGE~R)AzV~t{3e=3R2JyIL96sPL|VMwJtKBz5xp8P^o2^{AtToII4kf)Dw?A z%mNwgNfl}pxv=0l3tBW2%Ybv;KJU=7DiOR^v$hFy(CBE2%p?myZbg-&#f>@oHXJwS zddnxS)^NQ9IvaW=&Z6Q3ttq)0;-FLA%2O@H;yf@`PSilq5@kIVOUhN9`uR>B!|tP?V>-vS3(2C@LUI-**|_l8f85%ehyS)MY8isYB3x1#Kc7iqC9gT4A8Ce@>50Q^el zY1M-IS#>x4^?A(zP6oFM4b6az)%Qs&1mckjIX8^94>fpatmGovOa9S2DddYK3;Rc$ z*)`z|qWk^?;W|N#k=poA3NU(WpW~sxQbxHubO`To29S>GS8+x_vY|gglgP@T-4#q` zqz0iTGrtA26BFvJ8%0f{Q|f0r8)zlpfyoW$Yzr_5bry&;!L?kqq>jt2Cmc?46aNL+ za!#0I%b}IARHvrgL>Sn>ZF33>J^>2=E=947sOEQ6BTn$YO=HS_CP0c?9)`X5S*!?3 zz@MpMWZ%wY06L$HdK2ysI`81TW;SrG+s;0R` z_GANub(d#u)Dst^oT-px;gfq4D8h3=^iS*bh~moKZqE0Q4PdxE!b1WJyAG6 zp?7gP#c4EWgV8Y#sm;uu3NuYfIws%(Nzmd zi;W=P%)%hyIPVPIg4m(C*3?GsCct4?3^>vHmGlQN&1-IU=)@=k=$kflDc`b{FkUFH zsQ^v}^~73HF#&Et?LE{6WH5|FOqtT&L{9Pjq?-Gc>aUrcbP zG%+h0&-&a?GCG|WQm@&abOZJmqv1$O;k;w)RKmF@&+w*bUUVlYgMZq|O9xgo_Hf^& zOxlP8VG-C8edTPu+SA8Rp4JS|OYTeNC+@ZG6nn}@g%09ZSO7Zha0vqHZ`GLWZhLUm z!P4toa-a7GmsM`q=}(ZoL(2nr$&T;^z($j5Z)Wh@w>_l(W8#Gg9S(w$o#q{!uz7ll zV_;zCut@-4nFgyz7XogKjQva=<5j90O8gT>d$hl^UOewj7gFM!4BW6-yx1>(beK>^a&;?;H1_=(S7i=A z1CgI1uW4Q&?s{D;2A({%G(ow=O|xcj#!?q2a$=(&_XIa#Jts(c0E=QR<@kf$+_qMw z==Gf~rznYfXRCGG$EhFMq3e5h(~tM=rXO*lE;rV722!NqbSjVut?#i0vjxu)PV$V3 z()6r~Kts+Ba$G`lf>UiRZ6>RAIk}VeU=RE)d#d4C-YJ@law2yKyw@27O>liYy1sJ{ z>A=cnKjINdJezf;7icvV(;D%R07gX{@5=#iDitS4wuk!*C)tJi&%ocD5F&(BxD@>RCk0IVhG+ZXwCB{}=lb-^f? zIt=e4*|5~qJyGhABe@PhI4bl}LaLpV(f#F~@y^+~?9WO-NkYc9ZDnEtgn$F7+FidQ z+PtZ-;mIS|wV-qhR0aqOoY$6y;-I7yb|1b(7UI>o8?V}}g(=&sGLW4~2Ov{yJCndJA&owVbltfa7`mv`43uR^rm6Jf&qCc)DC)M>Ft^XUNDYqd6_lhM z=5>lpSuTo~k;c^<+Ui7TbZ9$WeInhaFRGT-D*78uVAuUEM29li#WaL+;sDx8)GHLvjvYH%k*sm4jD60YiKoasA~Af*gF9Y9}9 z8QBu0G*KzhRdxu9`c3(9z*1WGdSZ3ihCypA8Un_QBkkGWuu28~gTa;*{Zfy2lgp)r zs-aGr$X}|2PqXXAB>z&ar*hcnhd{Kre-b`4$dM{nyy4BQ{D+aE&mHuxe7e5&Pe}gH zK4J5ged>(;Ztt#8RT3TJ{xUDtx|mF;W5buN>fEt}RVDD>>v4r{Kjz;m1R zRZAO5FX(2i3{LDNsr+=~YBJuyb`kkRSoiKO2aC?v3|R6Bu!8?rfrzRZW^`GL{+%!G z6)pLB{l&c_AfgYKm)-cQ0=uT~i=YD_ilhkEbh;Ku8862McbID*RbDyBpzh`6Rcr8W ztA)8Spn+qO!Oy>$#FpwodQr`XT=^;t`Vam7PLEd?BwV4*k}-nc35Ker7W(;$nG!{fz_N0S|**@N)m?7PBk7^*Cu3r&n{avb0`heTSM=g6iIuw zjL}syQChmxinF^eElIyi{qkwBO#L!z94%x`uf08}_`N6FdB^AZ0p4>{%{T75)mzHv zu(nK!YMUYl!BHcJv+qKswogT5$r@i1ig78mVat#GmJ1p^oGn&uE>wMBxoMpx68ZjXJ? zSY?_hDa*}Y8Wm}uUJVAEmhsbSV|6xpQl~F4t_drLC?G6DID`HdUP{|huh&%vl(zpY zA9_;@&I|kPzw-+jOWuf!WxMEXg+WfXED;SSV|~?5NeXUDZ%Sg@vDgi`h6tN*=M~~m zqwxpsNJ5TzE#|AOr!rti&mR6`w~e^8r2r}Emhk^w7SaBkKHRhm9(lx02uv`o1K{84 z++Jv+6VNy#vq|fW9>2_I(3wI<7rb%xOg`t&gc@$S{JA05-p)z z43YWv4T^bbP^PW_99mv`mniAqPsy*go?tsFT|g-J%oD7VPL@A{UY!if`WSDL_#B?1 z)|5~6@i~6Fxzhacq~AGvK!sEjW&07Aq)+AJlG+NM8PaN=k9bU=pV)+{I2V<`7`l$E zYfSskx5J`(yH3CT*lj(igs|AhlA8NN10)Q7h}v~}q{z70%aE)_2>7|8YD>ZEN{GPc zU!lF3F@h?)RTQ^l@T)X;t#+P;p0pI^@iXU(wJ>iwSNGoi5qjL(T0)WM^G|mQ=1uf` zqHPn~n-Xo}lVjA+8~`Juc5vClUOj zB5P4+qwePsqV0d=L1O%Ci5V8#lqAC58 zBma!8yh2@XS|tyi;RcY&j5-LOrx@0uNcT&I##)Uhdvk~^Z%Z~a=%R@xb!n9pI7hdE z=HwI&uKxS_JV=X4TuNF}R7Zh3nKFWMx<_MHhv_oJ6hnwn%m&$bkFX>CXONo!ZbE7) zFZfYP4T+r~v&c&_gC%V6M#EI@m0@Z>3E{&SCBY_c_mKDX8f7{f4Q`@NoLe)N z79dKfknF*B3vCz-!xFY#mzkgle&U#doW~7_9VFzLD~EiIn^5P8ibN@KetxOWZ#Z%r zdSC=lkzsqF5xy6D2Txu+-&YiMJewbS(>0S7F014yAylKdVo5KIm1);7PC8$#j26WB zJ+o|Oj~Y(GJ#9AjEP3}9`rgrtUM#r}mp4jYEc4-tqrI2A42d6ozx#3@2s0U60E7?Q zFZTZV^2xW~9fVJfTa)4z(i#e{gkLqK5PI;-!(^A^eb=^ki^Gu~N{S1xb>Gb)j<3Gr zy6pt#iiK*)Pa|jY20y9D6;i5j^o?k895V%$QUBBP*0G^a8gJVR`<$mOT9M?X>~pi% z8u=Es!%>l%4$srr^cPj5GC^A0Hsuw}xTm49RAUp2q#^J@`b217iLD%+-fsvLrz$4c zw_3M9x52RHd0I)xx?d|&w6i8>t+kZV1N}mK3|ykcyVjzR0o-b`d${#RBiU3iUI(L; zK0e$Qsi+R4@j+seO^MKa=rB4+qVP9E!3u@G+jB~Sto@|n$6y}pjqt?RBm*5A)q1_+ zd^)|nzp()efvdB}{n6C~4oe&PcN^w3Sl8P1O&RbulOGPeCGy zi-|?SK>+M;_wtCSM3+QM;QKbd-@pF|a*V4H^9*B_L5Yy+pB8ohMDzbq5ey3-ON_X{ zCUO8Ca`v#8m@drXF*#Nj5Ag~QM9vjhaM*wafng-%pj@W3YY8!3-rS0;f=E#;D6%m2 zu}G>uPTVFQ;)Md1t9WrjAx*laC-4P zf^J#03B1)Ny2n@qsRND=0c;mO{bKy3lw3^0ewBNW16zb>0xLr`cBQU_-mWHUz=%b} z6{P?MY5QP&lkZe6|t6%@;^~IpDg!cz>*f;bxMg{Wvpz-S9aqEi) zZx|e&bOz8V4;mkEwR!QM-ymN=s*{J6*^k2T9#MR)#O?LfuYd4yFQ)zJ05D*Xzrpdv zKIKy5h#sYu(fO-(mZMfuFOsq3@}H3K*!0EXaSusJ*lWRlh6{LX%3;SEujwP*ZdXH8 z$k-8$P4UZ;$56BL)f@ul0gMQ)xfIt}yabT`4qDpjL0Mv&>gqDIRBW&%p~fWK6cO#5$;E9d7Okqk`hcJpi$X?Y6F$^6G3jZrD{3r@P z4_fSm44l6)dy$+N9nuv7kFWxAErVKV?tj1>sJTKg<`6T1fTexmIe% z0>^s~nbRSA35?9JZ{d*%aPlBWtLl8g#`}f1V+9fir&pc9V{G&7jRkfx-mu6a$wb=_ zIJR()hJ-8n6NKhvZ-XEo8aI07?Rg&m_Z_tM!;|MbFL(DFY!c+b+#tg2mVt4HlaSXU z36ky4v)Svql{K@y?Jeju(qkGDl5amB@lp#y5112l7`KJsp@EcUF6wR@6+^L%Y9IHI zXehdl0%rqYm{RnW3^~2t_xfk&(+7>s{3~~)smJ=D(Yh|WpMBQSW6YDD8ux6FeO%Be zFm7UNWTY(%$-M7I6D&9!!30lf(dE6+$KS)U%RJQZ76@2!HVM9qo}%Pv+B?!O^xZUT ztd^4*34n2sE7L$Lm2AD^2YS(-C;trz|Aa*YzT(#RPjJs&dJ=C|M6{iOl0TSYtRkU= z>9d8bTfUQN1E51}089FX9e;8YOsMgfPp&r~ezhty64PCuqmw!s2=fKk`lx$xGC4Z# zLFCm=36~I#@Bk^ZQ6`O1jr_5ZQd07hY+L{8{%AbLUPBzd#4V&3Bgy;47+^=DB>noH zM9BmYDC}Y-LwJH4bCI$qK)fx@$qQEA)bk>)fkDzxiz#BR=9c*eqHAXUxAPKAah83A z$555u>%P3=qVv-Tng+sk(I4{Hy(maDIf)@Mc`><+?u81s_lN41YRI&0UI|L$=mOMx z(AbL2(BS35o+?@ohG4nd**z>FCS-&*#9>w8i=g0c4dz$LnlQXe&3l0#M{Coktb~ek zd1LD~!vl$MhtG%cSO3kM`l|soCQHtRQxwYMP$#u;V3vXgo!x%K>x9VDL0i+6o9LG#t7Sm81v65N=dK5n!6`ky<@xw6VYL%nT)pR6E zH*7MDuhbtZ0hfJ-rEFN}FVHyXAZA6lrc7F=s|%%ggwLt{Ex{K7MG?R>rIUcrqmz%) ze$*nV!-;fztJWHw*68wCH3&(?;Iq!-olm9>$X`T3^Ufplm)Ep(FRwKhSsI4o?MlCipsD% zr|@!?WSUBM%rJ`ctR4Y7$+z}JLJz?wsSg8SELaS{dqbY=fH;fpyJIL*d-dD9F&JR7 zsTg_AzQhgBCpbkIP1gbm{-L56mC)t1+j)o3O9{B-?k^@C+c5@L<6Sc{MW7b=PB9ta zR1|muOxs%`>dn%#W8P2oYqFYQapXMS&0=6aS&RSk#=3uxC^yL$Bd{_i9~a{`>XqW$ z0^U9a;J;0zJFrX)KYK*{=>6ofeet$FM(u^#Q`~2*5o-2Wka8B7722&oKJa| z%ky^OWBYiR%yHhtq;6s=D-Qq56L^wIjODWq)-JE3!?2wF!*m}uO*kN~Mvzaktfe{J z!fBD5JWd!iY)XEo^4k3~Ub5EspT>IQC=4uc4xEn(Y&?2A8R`Ar05gQ%i7r4<00`NG z6bl$z`G@wXhKIDp=YS?$W0P+(r4Z&XO2_g*0LoCni@$E#zHkb5aLHG;9x`~c?giCCg230?v=zo zpnF&H=u#v97-BFG33N;!3+zni6cTP?i1z6B9o-$D6p-i>rDWfa(>T!hlP(Tx^ojkk z=+pB0nw4Ul_5T9J{d-MOW*T&Ic380~>1028Aa$6}hF46?D|WaI5^GQTmPe?bog71x=-Wn-F@;(aOrIC^%v#-QlB#%2nZ79lX}q zSFC8uR~m%u!ehpDM*$2!g7yVSZX(gam7Nqq4}pe=Ya+lLE;+_k5;)>$VZ1aSX;8e7 z{3(sTa%7%R$zSxa({9?^aU|8VvVes8qz#TN;}Z`(FH#H9rXxsRs=7yPsY-r;6@fnM z9-xu_5N%AN{k0qk9g_7z=`DY?_jm!a!P_M;r-MXjOXHWsaoxMK!x3~VuLX3=-%62P zjG%B%hz^=ZD(--@b@8%?eTQRrQ3_qC&`wcJXQL^&Vo1K?;&Of5r6xut{)aUBL=Oa_ z|66>|EB+$wyf2F*I~J?5G3Q<`h+@wcS7Q*G%m zZKRgjywX@yS0HDH*=968=^rCxjHxpir5d%ZAdrmIZ_zkTgY8|2+JI7v!Qzi=Hsz2T z298z98LvXTRgQSF4IG?iBH*eaWywVY`Uvzoi-UqK91cf;!+(P6G=UCK6J3kS_ElXiLum&ueCDI%QsggeP&1kEC?OP{dxYdQTdd5~ZJHH^|;7)OT{ z-;~kQwgL->3W;#V;r)7J4g#o}#$t#Yc_{MH*Bd@_jjfYb|W1if5GB8hvROHH}DULV(#Iy?#wo{2KpYjzbCkVMPip z!KqGtfm=;2fxk10;A~Geb8Bs8kx%ip*=)Dgq!dbjdaHTswpGBT0lc(GXzBc4iTrSU zj=H1E4?K^vsgX2#{LFAteR^whAJbi#2Jj`4-zEE2Kfa#+z1KY-VIFn)0g432d?LvM z6CBi73stme|CDEqVokPxz5Z!)|GV9%PY?dNx4X7xis1DWZ*eQe^Mkdbds(zjUjm4> zT!$8O6zggNkz(VYKm_4hY=7QZ=Qyncotjs`uog6?Xs4z$>Iru77JU?o;Q-VUC729E zQV7k$=bQ9ipck#n5{MZ+*3#AfqI02j48Qb3Xn>)Fq^;} z^CyHpbUA@F8Mynnc3m?0myirQn$zQfhaQ4<(n0O(cY}U=ERz3JORf0zdoL$cARAl{w=w5AA4L!JSem7j3^9B-tqDaa+W}l3dpE z5kK(&K;o5E^eI64GPblJCE-j)sdf(@qlnKqhV%E|u^53+hV;svy%+mWeq1k}^@rd7 zfGs*mT-^R}_P);p_dtd}YCj%$X8RM_X{U_}c{l~5k=9*#@>2t_%vDG(BWc)mO%1=@ zsUa6LeQZWU;CUw$xtR(AYpNYckZhw;hl(YRT%*V^&dYCzjt$8l_`a;xpb~W1MY!7a zR)3;BHVq;REvjq0MA&^O037+aJ}$_1C&-88zCfyio7KG3?SvH2kkjCOM`v7TXi9*C zPLdjgf(wZos3s4Nv;x}Rzd@O)T6r+xi|Zc8Ajk6*zYZy#E@y=hOqd6;P@GB?&?&w4mGrX4ov zs*CZ7EGh70X*k((-It_@>9v@AHh;+e^iMHEK&qF_h3uVQ;Q2tjlc=yA1u(IecVTJk z;}!4l!it-0AoMkHeUi&e;X$+S;Hlte;2tI%_?$OySnR!g@qc!=5B7`u57fuWJc@`_ zZj(~O8;ldAn68I5Z{XW|-@SPL&-(?offvGlLM-@y-mbfxRvzW<*MBvG1IC-@0I=u5 zOODv1mglxC)xTs6D$~3A{<`%qg9-|imlODD_iQuui&D$l@NtJDNy!d?8Zw2((kl75+t zdvqz5skHeRZuA9q=RuXANt~Svh01x)5{F=HV12Ip7B}c1@<+HN&40MQAiMC+3L97( zZFw#iF;)-;3zkrP=QvkL>l^{B=7Jd1#DG!>(7#K+`#8?aNuwHa^GK(^J{8Te5MP;S z9O;fzw#HhKXE0y%5i~AJbag;T<^(JUp5u3iKX<>E7m7%zP;%{6=PMitm+Tjhf>C?@ zI`Wk`iE*;R?u*+l7-3#vyPA3F2;50IKqT&EhC=(a3BWr~Uam2Ta#R3xoyzi6Hmz>~ zEyoPVKs;8l`T51Gm)pBycRvC}%=MCu*FbhQo+$cjFb&ZmdI%SAnJ3l-4Wl4_gwXr` zWv{!2UoW2R@9pBo8Y@y1S#1e8*(cB=xF-r}b6X>J5PkayQL}&=O%1nb?^l?oSf|+&LXtR8F^u($ov_?1V z+wJeHH*^vC^XU#QI+9<3gMF~w1LDeT2)dOWofuaHtS8U^vHff()`wU4V*lXLt0zx) z@P6Mv*dRG(Bo&c%jX({06r+HRv>YNPq*EA>`SonnQ0e2jVfL^(p=(OS(e*;f7zNjh zJ#iv$a|LY%m2VQ?9AD6{3~cBmPXoX#GCoBSBaaOSnb3Zwr9rvgHP{s8v#76+3Rj$a zgV|qi1kx~CE5;WTrj1Fl@vkaqFXf0Qn#W$bm^$^@AsiDoixW+Glu8>rr)&#D!rhdN zPTN@{d_&tJD|KqFIjk*PlUFKJ*zV(~qtOHW=fuYZb4W(*p(3$aGtq{5U9nCf^lxR` zC#Akq{ws6!XjF^C_vf5w(bQK`lk#h*eNRE zZBbY`!27gmU!8r5B9T})%}H6rCQslc1URW}PJg4o0>&x(WbC!7qc92P`cLLIF=QO0 zD@Zw}(3txuLCfgk5nNHIJhKZWJiuq4$|30xGxIM1jeM!K9zX|rIjZ4J626G1D5 zZrFMXiZiMJWoP&Cljm^wisCa3`N(8JR(rrB@Zc@I$C5N91qoh{a86O+s==qPc36Ez z?If#5ASBO(VyY+(kiL@Sx=3J_qF>aLyLQ#DR-!9%MXD9S4$$_q_osSbJ{YZ+S3|K` z(4Q@Ji@KJCW32ajDg@OKjVEW3sIH*BJYq0hkd;N*;cFaR7F`p$WHzYdPEzwlA~K^FQ;kka-_!ntTR^eJ>H zixL-wG-HiWx3L*{B-j)Od437cM^r@AovJKm99(OYbde{lDOR_>SUDtt=Poq5x{f^RbgO>zh9-8 zrbYWDmM|k>ByndiO|`(r$l1KNmK6v#_5lz~UPIg<>~gXb>s+4A$;qy>4YxacW|c~# z*)HPqpa9wE7!@ss3Sv9~evwSmDaKrjtf>i+LVTh%?oxj1Yf>MvwrXB!CvFOvdBEL| z2$hyn*lhsMEzc;jWlQGCsz5XFK%&rq0{WTl=B2<+-H9%f^^W@M!7>_5?$ zni(OQ&#Vn@&DLExp|XcHAu^n3#tE`9d|8 z_%@Qjk}TY4D_|rWgWhCHPtrk2O#_eEYZsd((P4>RaU#~lCKxGiAwd-Invj98Rz%DR zUqj!a&dM>8rrrj#q&&paf$X2zutdJlz7IX(MX_867;IFgqi5Z7vo)F71E$GTnN8Qv zdE3ul9PB-O2{ze%nuMu1Ee2TD)6kWyB*vHS&qZ{;W_cyDZpE5?kR!|z?)v75i+0|k z@u?XrdZH-u+_IRIq!AM??fKo}^(KPy)A;o74U~J#X8z9L0BES;0lex#+FsAL4f{*mLKKmlic z(TGqHWd2YD{U^hl52lU~7>uvUrd~9rdYoc77FtOkhkz}7F zxdi`FSq(}<^jTa8p_0^Dhiv^5wdl$>8F87FOA0f`_1cUw6_GS}}Jvtgod3&rZxOnYk zF7dXY@J!@%AS~WnJ*(PAP!gs%D7a9WjK^g*&7zV6 zq(vC@SwwGJw_ls^S;R+o1sK-I4I~lS?V!LD0K3+am&VI=ki4@CKyBleDIBd}5CSo7 zGAsepwbIap4{n28A?a|2&-VwT_lUabedit6H}z*{pTpZ-!Ahw%?;ZXB&%8Q6_=AGsZT^Rp#b;{t8f`uDfe?y5XW6eiI5n8rbI_Y7rG*{ zs1D5+FN#m5XVJY@l+hpfz0!i@i#H8CE(mF#39l8q5~BvPU%>Y!vQN}gIH>m zH>zoyAaBTt(sB3QVd9a@bYwnx|R?DfbwSQPPE95{{B!U=9yiN4w8sN>Dn{L}UAyscsBM zgVEWj*ub%i_doqRn$t-ueni0~YDpVVX_};R>()AMWKdp3a0NLQN%M{{8~w8+C~;u^ zfW#ml9MjD0zk+^oXb}%6t7q41YG2b~`fbyQ0zwek-pq8OSZWTc^ExqCYs^+aG0P3} z{@QdXM#XCo*W3u#Q@=)k{SGLoa$O8K1SkHqd0FhqC&gZG+)_Wtw$#AOx$I+VEMZWI zL_m?30!6yLo353rca+oBVt+mrpKS`$rKA*6jYDHbw(;!$}71s52p}w!M z=eTGa|IV-acYaml_1ug$=$Wz_kN42Nn@q1x`Xj`PVsJOUa{;tBEA}@=8(41z^C=25 zem5TC=l0$!Y+S$SU&AoU%V+X)&jTI!sC#bZBMytavxQgJU)*yb_{kS{w){mXbMLOd zy1sW8W%l;(_zV2Riyx0i*LUu15*7X|@EdP`iF($1!1F3}O+r@rIGfMT8U4hwDfszF z?_@5%E*kG9TbrAo`$u10VO%3$83~lo)e+`Lcwz00zE4enqb4%1ZpI|r0F5IJvT+Kr z9lZ6Cf+tel@$8U81!$76?ODT6=bZ`GNJ-2HOW*lscWf2)^*?+FTmQY`@GdJ4>V*%I zgcfM|jnNYKykerQO<9E1^r^=76Py{q61GAVi^g*{a2u%GxC#-#nc1;)17PL|-UmKG zRx&_Jp}Vck+hn<9G03q_)`R>N{dl=ituiX%Ml}TJ!NAHVy85GChdK!IyFD3qk3wq?Lc>4?CFN@X4W2iA)4WVse5GC?6U|};g*Lk#2MpBa47g@c zRbYk6T+zk<;g7du-3TQD!w9hzl0@DfVA+Ko_KLDP{67LqKR5`3$U9m)1E3o&>=Qar zE_8#NQA*yrIA)9j4+!CA0dBkm<}mHYZ-Qa)e4Q#1obbJ+^AE7#t=?c7aH?w`sVw_s zSQC1JC9asyF|Wk1%Uj;(mB2Z_V|+j5JtB+BO*<3Zy{ii|@TwgSW z%bSNl(Iddd+7+@Bd7GkEl$HK>UDlfawauQ)E^<8|f_z>E1cw0jhh(TlwL6*Dkba%p z&Sn3EV^#}%a&n4K)I{~<*Xe0fV#x^vkrE-~WeIltQf@W*GT=2d5uf?jzmZuzyMX zSMh|u?@w^OHtx<8ha7Gu%gwvk>RG=#9!*B4)1q1x4q<-T)o5ZZ#RlJFxoh+u=RE&8 zPno)yba8P-<(#R>u!qD8oIZvBmun=h$TVPbmz~^p_^~~SU4Z4l@p?M(p&cQqk%2Po z;hY?1j_BD+PQV1P=xuPCTY{UO2bG9nM4-zVMcTeYcZ|r_DuJnfoC)BFnjXF2lw&q1 zvl$V}yDreUPROvRuBz#o02~d-gb_3|@E9*|z(*qc9!|INVVG6HzK8Ov4%MsoZ4|FYbs?avmNIrcUJa6|W}x3^2%*{t9k`Gzlr(nW){)27YM1b*e`6`bY8 ztJ!R#Dr$rNw*Xnf|CVweC-4a4JHZ%n;neugHE+h8chTX@f8?||TEYLFX>;2CA7sWo zBIeC0XC7IoEJlgz-0vie${aGy_{{;BH{l;=L-9asJ^{bI862x4E1?n5Kl>bfmalMU z^KY9n@2*iQx2aU*vWv%Kqzb>s#eZ}$?zU%W8~s3SQ(6D^Fc8B5VUv?7OuX~qus6P0 zN<&B0?+z`-q03PwjZvj{v<~~&p5H2tvG)Y0)$`^q9)7xlB0X%}e^9Kx9={&0t^iQm zu&|W;bmM@lcwOs}jD1@|X133)c;UX$ws;Jkw_ZG!^Gyra)PHMlZdz6w0#TCGWt{8A zMs|3a-jt5TO*eeYC@*QmU~fz$mq$sn!g^943mWWM3;}yj3@o*=#J?@`XiuY?&Bo8S zCcj`co*`~%YyYpjMe3YG32|ag-DLQgzZ5rpcXidbozT0(69b>*Y<)Wz9d~Rq0*;B* z$9-(ug#X%f0&b@_7ZEA&jej!|_9d@J;@@#Y6T4os0E#i&T3=P5&w+CL7B}HZ!c@O# zaX$xE(+EmM9L0edu+NY~yZRw}+q530_Xi2WSg}Yj{uq!|BIGc+ta}qivSncFYO)GE zmulpIHuhmZ#0b6XZDYNel=~kc9$Ig6xC8x?AI>o$KSaofVQ_$txV5Cs<^GQZ2Td%C z88@Qsu5Lfq-84FqSR@C7<+S~Hn7EPa$?&EiBeR*LRQ0Mxw}I*MYBf|TX{oS2CFGejtT(+HQyl1-pt7b(wbm6fIY7?8>P{`9=rKnvx}p8cz1 zUIomXv~-at2j{t|aZhP2TW17iKt`k@sFi{+L*H^zLwNwj#zP|dhH`v2RK%l{bi4y#zEWV0z zdFUIXt%P<=Mmc1@e$W+O$UAEJa86%c%0igcb5gU1!a>paMmW}4+t3B(_Wnu=Qp){l zi}s$q5Xms{2Ri6*xQ7#4dG)duVXZO~rLHYH=YCFuevaK%ugjXcAZiASSJ{77h44?Ju`JN17pj!?A`d z`Zv3*0s!~B=a4vel9qsXn{H{**uo!mVj|Ap!Wa=}cU#;*c~`B|sPMILh&|uI^S<4N zq+|j0A70|nXI)5t0-u6ljSBoYnIJ)a@Zlf54=~f*r`a3Rv#2lSUzAaaln9!-+_JG- zoev#$hy4PxcV4VL8lx5Rehz#`9oy|e9~yhVJMLdjS>XTZ&xop(`|jY`)4k4F?`h`) zwrR2Cx8wdvrPzT?19VJS=*jNqn`xmz=fi&gOsF9&$!}nGPkcy5KCn5+^?ZbbxP~X4 z@yXNv@whYY_vmal0t9MH?Yv%M?!4y&QIS#9;jP}p(*s>@5eiC@G4lp9a4E??l_ zm~M(bn3n8IuMS!|olfMZ`TNdF48zyzQYwo>-|0_5LI5_+Utw{e$FkXg}wIfW$&~%hA*YJ58E6MiI&Oy&M5|0OLzak0s)`&97f$; z1bb>*xv4AyY= z>A-eU6^$au^c{|ZCo_F!l=^YPR&P!hr+P>b=CBSHbjpk?h!SR7a;3Fjl}I(b+$m7V1iOB7h*T_5ilCsYYk zQmnw997t*aH6C$9=;UD^9L9If**@Tncrs;@xG9km? zi|+qFPxpIcxCir!(gaL~bj)v)W#Z8XZi)4R}lna@!0-CFs+-+NEM_9$8? z0kDHN(#~gXtD~hHIv|P!34?(SFeYF6dHlD{`toL?Zi^Y0oL|^{%`eMM-wV1o}puzA3_Vjc|Cs2V8xH*lvI1nXcy5 zkPa}X+UNDHAx&v+HSAjSse0NP(jc^&#rVrym(BZRBZ@3Ni0eteBl`}lRg=*@dr&j; zql2ct95jf|Id=Rg(Yo>rrX2tx6O9X;*N05fg`l#n{pRnSo=q{#99NlJ#{GBFZw$lbz0BvgI2#@U8}S5{V^^N38)n3_Z6XFh# zdy$?%UP?NNUFK{tRqC-SD=fbxP`?y_c-s&mR?d9EB;MTr%ig;HS9YA|fkzbe8q2mE zE4CcJEI!x?&p0`a&2t_a)<-g_|vn4(Q@w2tESL(a-s zI;AACEmyP~WmnG171xGY)321A)mB^^WmnmFD~coAO5!A}l*>`d+0uUB|LD_w?zwo7 zR7v(4w^TH#J&G$Qy|T z1z_ZzDrFAuX+fHnf`i&y(TZmZieB&8;$6F;3CV6J@ zU5q#)-j1UI&rLw<<RPJK;mU3afz~l$?;+e2@f-je17imd)W2hEu8p)dXUw25YC_st|8g$uZigF0AE$qR8LY;ppe%)@vj9w#{7B{A~RX!x%VOsL70pzWe^b-`l z-3E`I!I|lLMC;16>LL`fh*g<1kdMEsM;>#L1e%8RTvRYx9fW8}ih()MqJR%0vCq#GaLG29^K59?+W{{{40dHgg)KO!eumj6gr3El zET+p9)^^i+aM%P9#^$PXgi)Z}P<;g>4lRCLtoodK-IPI+@RO9@m?*E%C8p?ucT}h# z52aD+V~tU@&pj$2x0+s1)>%@;n6_!J0dRmfBFZpy#K3l19NNJP19ECCv^Fl4I&>-h zRZ1#%^U2bLQ-I3=gp0?hyX)i)C`P&x&t!rxE|C^IIa13|0fmwdV3|ggamS)=LnEv+ zACjqt_GCb#1yy*691 zD@Rxn_G05c7X+{bHf(!T6`&zj&0d~gTWFE33(g(SMORf!A2C>*D~M39daT03Rf1P@Cukefu!n~NK>c4M^@n|yG5zC@#v=5g4S3`t(fRAY2! z8DhS0)~Kiudkr;XDFIfAd}Y_|1G#MpfL7zt@ybbKD+!U98&#pvbNZP-kzoOtP*SO^ z0UI2zF%svM!u_$G+VovM5TB!$9)f3~Y0!8nsw$eu~qC<|OF4aikowb$_mA$GBgQvVO$3HzH?gq_F zaTnB^fUyhhp%&K`+vD}v(w5MeBFLv2uc_EVWBt-Thlp$bGFQXxrxE6rYYuz!4Oj(f zHBV3|>_8W}==1C>#^ zKd4P^a&+$h#4s6LRZJ&kih_-;*x#uUsO_27^muF!r%2>AsuDpHP83(fsHp}x2*wnZ z#&wRb)f>%f_!M*X61)<@d;?v5UDc7Oz(gAX)Ix=O=l~DNs2*N$o z4^PPUse})*0^?VpbGf2tvR8@9Q`+egcXKv;!@WvW(W$83<2G2RB{af8o^;qjtY>RUXFiJUmEymBQb`F$(b4Utu#@t7V zuO}~3h9p}FVGs+0PML0B<5qoTh+0lG;m|)P?q=R@XCPq(kD^}#jxbwWz+qeV!hAWf zHexz{HFyAl+PU{^5aDr;<;c)H9V|q455g=eB9iu(wZ<3O!Z%%-uCB1D$4b}`zEARo zq6mRTc|xX0xJtY)_3$2};n{idi`m|a)UTu&b75IiOH*x%w*}dyWqPe2Cz-JmpZqp$ zOX;A)-qAE9QWB7_yNbgVyrCG1$pHk1{HiZ+0q9%w67xy5-I_NLhCNyA3|bH!738et z%Be?`Jh(j;gQon0yhyy`7wcllhf(lPBq7ZkE&gvstII(Mq7$zY8ko>ZB9Re9Cxj-$ zC52Q4hzoBRdC&jIG&aj zESCo=3#k>AqRP-v%`1x@(Fes^&H-cMiXTym?w}^skY#8k<_QpEU`K@|9Nj1!OsIQ^ zQmNf_0gF0iB9+p6NBJ}l>4T13Z0SrP;?4Et9GTrx5jP;GOGN2X4@W*OSKwC^hrjxo zPeOqNTU`MIZYO9JcLwtBi4$0AIKbi(P!8y8l=Ca4O${Anbt=Kbv^Qeg&M9DI>Fti? z5Zq{aE|IVtF%CjdqN-H50%%`1qjJ*YxK$iz$XMKQ!kGtKcFbhHGIEPf#iGdN%9way zX_82@^4Pse0mW$5s=AQ@qr?Z&tYR(gRgj2{m6%2l&>ciH#=Y+NRiL8r@Fn9Kmxi(A zDvb_i&KjZ3bg$=6gkt1~0Qn`K>ng$QmdwBeKr!SBYdktU<~d)`aZ*)bPt8J;M$?A7 zH&mY?2I=Qn*j5_c*R(KkE?T z4$02`3|DhyN?kDzrNrkG?5iky9!OZ4wM!4otOJH-6i+GD-7YeV#dUhkB1Nfn3l+T=dhz&5>w5i>8g zviM0_s?&9aYuAY!PGCeCLgi7MQvw9+hHU0$hwNdp!9T(0S$xwtZXcRyNxAS^%PU?gy%_CSN?2{4z zP?~7@$T<`)02Ml6R`=QR9=j zx;=NK#MeyXuv@LMDZS&k;mkfH%u!U|9YyplJ!vA(RWgb!4ABK(GqwzqwD=p3Glt7>Z_hsMgBU5E7L0%~0J>ve z_b=smT%0OHjL7E%S7FCD0rNcSN^5x%>wS3y(-iPo+FCx(#IAzlBrJbG{A4I{&-WPZ zzZQmpT+E1y;$nAzcmJ~AP~?>qh9l;U+KQVa5DMeb=op@{9c(ntM+@s#*!5Ujt%V^d zrAXaIkaLG-(yz5~`HGK;R`4}B9^f409xgq&;IL+E&AM;S+gozNgc^Y0JFiw7 zm4UZx?LbV#*aRgqv53w{2nNUVm#$tPAGs8cGvQV#J)vtgGElAZhRoWkpQI+S%(b!n@fb4v<*O1W?8C)LuOZ$C^oSzX-{;Su^a$7n?6|I^YrMp_YEHR zYt`4aTXBg=k5DCbB>7G|3Vl|1OtsSo3`1_}>`Icjoiab!B`L5>yDow*kw!w*vu=9W8qSL4YpHHN9La%iuk{$!nm0?Ba6Irv%xE7XZn{0ca#dGV8yEu_g z`g18cz6C&wy3+~QuXW0dkvaVN4Kd|b_6Gze7!0N>+=By4s!~;}B1Jl;Cdx)^EluE9 zSwION8qZ&pOscsl4<~Di1EY{2PT#^e@vsuITl`SiI+mCRbq-<|1RTBk*IGFIII(b+ z)B|bZd+y&kImF#ky>N%@{hnvH8|}xiY0AB0)L$y9W(Z9m!Irq4l=6gY2#{(ryIGnRI-%l*?kRPM%a0PeB`)vF?gcidQg*B&;i>} z0-zc;;~Cl&hB07Ypss*SRwNiWj2%Zqp8+X}4i3^QHap3SS)cMeT}sObz>HN$?NN z9lVS!;~G3v#)Uw;oF=Nt^?JJtsE|3R{hgm-!f#`fi*j3|eSy+dV+qO2j60>wK(8-| z21_IQm$#cZ0Jz`7Dv77`kw;}^lrI$r%4fhvrNf&}?W{?XMK)ENbv`-0; z56zcpEeo8Ux#@_NUrl6%uR>1}#z+lZuAd;djTpb~*Btq1S>@n>=y=SXOZi+7hxg6? zr(FyvN0VT!1bhx52m~wVUJ^k;8Ynw^OC^un?L-{N&4Q`Pww!F2_e*I^P9_#O{y^&m z_&#hwey#b z?cj8s0NQ-pBRa04U^&00jGkW(wP%9}tb+k8r=La1&?a~y8ihu?p!>SWNC!Y5owW*N z>eS9%aT-XYU9Yx{S_qj^wblou(lb=wtqjjx$9+Mxy&-clLL3oKafsmx#RI{Lx8B`4 z^Hjh9tu&;LtCkVsC=q8*o;=rYoj5!;`*7a0>uC_kL|WmzJQo)%W(TjG}r5p|qf2e&#Pi^~W= z6ZmXxffwrjadEsWzn}}bI=|xbacry%No-L5jS8vaW+e+npSwSe;IQ7;qhUt`m;KncyIqfH@%SF&Ttc_>hyH(8T4a zuty^fJ$yDzs!g0?xY99zrPIU@D)wI71gbkEw1jl%!NPGaPTK?x&&3nur>b|Wt+wXz z_-=RWo$C)5PX_pbaVlqx^(`gB>g%iqH6vp@E!RiYc%5z@)KgG-*siAcMCHftmYrAY zQ_QJ#s3*VSO2jt(9&Xk1Yy#%5&P{w6DrZ6BNG9M9mU%g~@*>1$^LrPoc?C;?9jrr-s*D9dQoWye|w+V_sOkD zLeej5!6luwwtE$kAg}D=M!n;O{T6MI6lx)7CWA5AJb}y&7xFTMbig*m-h1n1+ z4fzF@j*c!f$t{+^s*3|@3^A#&1QK~ZgWijKUiPrvUOjPMZ+Rfh2-AyDR=7?BsyU)JI_e$}Y+ez} z>#5-O2%aCm-fnMQ7#L{bEceaDR{v&gx!GtnR@+$6*c@2l?O{}U1~yy5ItI9i*u#N} zV6f-dwCn;*9UL8QHjbLq?o}MkwG~sZ)ak1NrmbfYU+Ssc?tP#y(Zi43?j^Ih-HWhC z5Hk79Mkb^!w(~8=E?TWyr=HUFedQaj;CEE-A3AIHFRavXGYAUKU-w7JPTT{6$Je6z z61lZA`6v(oV59W;gmUei>UET$1NZEFuv30=MVOw+b*h(fNh(ODpo~oncT@-&&=T=% z#MP;nxuS#PgrgW48~1Vyd6arlSe4F1uhCfM!VLF#3tFi$Pk26@bDaw7JwPCVuC;@4 zcYHOyT=FK)l+uDU#u_aX!H$b@J{h;b?r@y~)09HuTig^WyIu3CBxx0=qgCOo`HV=4 z@X-wKSfVZx5_lO@^%z~|Aoa?Vr`V$NUryX*QXjWFq~u+`p~IoH4phO^>*Yq1Qzv^< zW^}Po8U}#Y^UEF8_=lYR1QG~CgO**8C*)lHaE0aNbEi)0QQvu#6=sBnNS9I?637xZ zB8k>Wj)Ff0CRs|TI_N<{Z;rkHfj%b#fk4O|^g^E^45pw(8<>byDZN0n`pX1J*pQK&H{MOrkwUj_m}YT1UKI}?f15J;$e^BqcwOG^ic5?T)7<4V{Q zrY%fj3LsG@7~k$EM#4hfSbCvTGM}*fZ_k0obkI%|_gNjL7sy-eMjEw$KZ?~Kx+7&n zww8FO7cc1ELqY;n3h0G9nBqBnaa>LYtYGqxhV!U96Ik<`XJ#Pl9lL!ye2JwV#E135 zeu%i6jXD|6R;wqTLZdr4s*! z3NI|pODIwYK#=&U4sCXxWDG!1juAQs+NaArIhOKr6csZb`53f-+AWWLb+_!QgrXdrsG?1f(-Y;D z#0d}-($W?t)(=lgAX}rk8!=PJuGkz5I@^*PmZg+p9EwW(=fa2@nYB{`sCURV{s2q4 zXLq*BLohD`JvX=77YE|Fc!=UdTrZ!!IFP&}VS*8QS#<#+IPjO(>{*lj@uxnbKbxM3O9a6%T*3<2qGKU`X+a10M>im103X6 zl9%9io!ECoQYz?^d_^t6rH*TXl$pA(*rTO#euG{~>Lag=opq<8DeFXz{4QZ>CsGGl zPJSSj_nEAmIfRu*c^Baa0grWQdc4`}o+vxR~3xyAS zGPq>}ZX}TMo8664npFi?(-9%IlD;^L$yEv<#g!V@LVAQ*;D|b}vKzZO+A=m1u@j!@ zxYe&T0}MsXfL60zB79ECib*p-l2FloCr&^;uWaIy#MRx(F0PfT2nE7;%6$#6Kdh{1 ze0#C=3`VDg^}I?0*ReKMIqN`aC~&tbOVEq&ifc7awFifTX}MBs_eUfH6j&}Vt+YUc zH*iFEIO3wR#oI7ihyVjTUTNWC&ZvLNF0G!tK$EH~#6alqRUnc|SJ1{o|4fjNV&ggg#uBZ0iEdM+$U^>y-=hRG1%Ct}t^Yh9o5= zb_q-KCMyfWzcD?NK}tI+z;qfE#*S;uP858==u>{`BtQ|l_jy!#T3e5`0y=S^xV)@- zoZUhftU%m_6R1yRViEJ{l3%Nvs8uaNRU?}a84kPi51^16kyMmTicV;zYUX;(qbX=% z+nySSxS1fyFVaXF(|g>Ss3Zt4f{o1T3Y+8#FL{QoaGHZP*~hO?g9-;oBU_vT41JUp z#~pVo2K}{-#?sOH7HPeFIJn;PQzQj?L`g34u?_6S*H=>EDcg!J}P-+ZrcQVIo{=Zalm7d&SmX5 zao7Tzrc^mw#vlQtk3&8JGM&sne4!2FR^PH%AKfm9D=Tq4O*pC;Gpc7~7toR0ReAn8 zHcp50;rTVZ*he4;92j}@GruC(sHseZjL~7s#gEvCZe*=}OIsx8H$Nr9X910Bt2Tib za9&UF&4F*Im4GwygQd39P@7stE?v6ad$e^MdxXkrYKYF#5}8g-F;2DZ;^&@`T8me{ zW1ZNj!s~Jabk7)6FDwG<64ybmfrcMjeU3=IXU@|5pt09_XHLEd7%1@@FtO5uJFX?R z9E)t1L4tRLmFQHYT0(R>=Xh^_f470e5!Jm;12C5P{$-HgBdNUW6iz0v)WqqMWC|wS z&e-1L%5u^PA!aWlAK3dQTCN>=SZxW#?2HA^q@3A-FtUOikmPs|mG7S8ww+N)xilDZ zB&?}&JFimmIHLD>sw4YNy*>5=1x{j+YFBpK6w2L6iAz5e56Mr3^q%IdyK@Q#7P=IJ z*PhXc)3CVsDL-6FK`wl-U#(jvJU>PH=%3x3%f>tZ5J9@yPI!vP0X~+sM4!Y3$_Zfw zcyX)p>axRHUkt<;akq7(_eU8iqDXHf%a4~$l-KT8EqAtIsJ)T5fiGR!AXhFQu>LYm zGCVOmx4k90zglOa2y%k`=zEbsiVCx^MJtyIIj>%1(V_u#V1&zH5IUD-TC(sk&*k1- z(O}})U$~9eLD6dgLk(G~UvL8k z1|=NpVDvFj?V~LQCO9Y8GQZ9vuEnVe2^`||jSOwnx)048_7{&i_OSP1kTHL6-@Z6t zFPOhE(rBw9V_&eHxd1Hm+pow>2!xsw=?QZ=T)sl_GKw#+%~+L9YQa*uBiCB07v`39 zu|L1dA!i&aW==s57eW=i7Akjb(H9A^g__0#GZqETsRh8Y9u>#8E~N!CLa{g_x#6CT zAW@oaU$FuXxUd`mD`W{uO}+6|KzfG6u_G1Z#wGUwNjuBp(c(6iI=E-NPwN5H2=usV zU9LOgpz!1@_;~r{o`f1PbN6$tJP2`iEn*_p+RcuZa$^yj#&yO@OI1HATo1Iubp)BvQj$&)D;t?9xM(+3QLww^ z!c8ms&3Ga>x5zQLC5^WP`WRufEWD13$-V-#6fO{o*9f8Zv92}xpLRhXsmJ>pxgxwhDYDX0p4EyON#Xb$&qO z%`(D)H12{6nPZ)_GuKdSx4w)uRd1bN1ns>jnWoKlyH?tV1Ky=dnrsY@Mu>43L%dhT zX!Z%5rlb*fmthK6XfFtlt<}rg1N+pSYI^l5tJdY#lDk)v<=C?CWe_gEi*@gMOl0Za zknY`v<622q!!5^Qu-SmU?n}Om3x%t&U)I;=#wSN0%2u(A1{)nI8 zT!LH*07(XT@`C35Z*1*Sqvr2gXL+wh%3Po|QG0Bpp0cOJLef-s=^O-!CyddcM62UX zbw||q#)ycCp^S9y7o>T$?BN!bAnR(TV`&kfZ*USGs;|J(JX2f1B7$flgXNXGE!8CH2ULxEqpbqQi$zNN{k^c__eyBYeW{06h(hH z_dFx^8lM|#T@Js7xoyF2;N63B+-aRUxv;!pHEV^&MTa-60A>eXibx}1B8q`BB?2Ft zk;@v>ngXL-fe_pgp9mhFw)eQ*rv_BUN(`(+xfDzQ#8oFJIvS#}9QbX?8i=4*6486S zQCuTIuk=cgM2096H#SQVPIM)2<^J74u~6vzbF z-OVMeyd;n$`K^eJ3S(xu%bDxIj@emWhnKSSXh*er74uy7mG$7A zGh7SqwF{kb-7okA}YCmV>x}1s-gO&Z!?Q0(D;;Dka26O0e^^ zjbq6$+b4{Jc>h={x*$FX8QjZrXd2`6j`ws~q&eUX<>axtWEsmNX-v_RAzJKFTXeq0 z^(yZxXV=3ThpmuueklGJOPD}JM3%70TQ9U1w9VguvhH{hUYGTq?WE9QE6DWB8H^mfvt!M3v^C(P#M{pr6TX}^yJ9UJhvt0l3&@EoL!R0 zaRWzi77bj#^MkSd7{So;cxH)OY~z!oyzk;(<~lZdA7FW)2~}KN&fWTi;ev{!24e$A zu16W-j@af8Gy-GA^@hEYBwANV2~fa%q`301pDo>52@5)ot4{6c0B09}C2Fs|k+Ypr zu5{4}2s*ha6mg}n995<(jn>=ET@mlpkaBqwd534`K^&Xy?Je4`3JHA73q;0EHLIk~ zU@#7gaEc6u8PkUgFj&z;7x6=q3WnGa!6vo60Z%Msl64?izT(1G=?6?9E5z&^C4|#V z+d1k5S-`%Kkf3%ET2q^t=@|RdB-<(L>>Izvc_# zRR^}l^>yHB{4v4M_yd;pQ{#YCi8h5RM%m+nnaH`a4aL`*zX`xZR!Cpu9AC`S2s~vk zpTpv^7siJsN7so#mbKRM)|PzXKRO!PLetOc@YQPfMa}IGO~s~h z;?`_R?~s4(IqjOz)Q0Lo=4P2+E-X~X9(1<6@U`z0zsjeZ~i zL$UGYHQ2?KK0Ty$fRi=4b25eWK*swuH3@RChB&FY>W7DzrzMj{PIr>i511!Qi4Ksu z-2)dj+dF2}b!@<4enEWDX5DVXF2^JzG@u)y=r$lq-BpHk8UYDcE`#@zI@~|dSpmm? zFQ@?_U{H~x8P!-*gj`&ypA5?qpHmW{2iueOMliG{Mi(A{J|j=WRyE1ty;3o_C@Xh_ z$@RYFb*1km>s?cgnPFRW)_Oxns427YPqB6BPAaQL3$?)vv^`>7VL120Cxv9#HMYX~ zI;$<>k4rXToWk}a#YX!3rDg8nQ5p{nAt|oGCBWd4VOl7~)~?B8`EMq-J*(Ffy@o4$ z@bXIUAOn!T7+f?y?WlaD&jTVAER#o>cK*itHFb>|Fd% zv)0&@8h2WlOpX33E?R~aLadD($Ah28WVpiIauBDLYP=NDy&dN>^->M@bol8TWu#u1 zow;urv`pOm?G%?wJ=g&C+K+T9P<5Z>wukJ_@($$&^>qeN)F{WERVqEsgA90f$Q0O$ zoDgHgOtV^4*_Jf5T*T*2JTW}4J8ow%$vV8pA#E;V7e+tY#lt1!?1!%zzx%+Iqun$Z zzI>;qDkC-(6!uPzIyOY_4Tre@NfAQRd|yxS#F(=)xnyglr7ya^L+YU6N=nvovxG?( zyl6V+!w!j~!)z3+^NmtRd6(5wb;hy?-!e>D=H^t6ZgNh5$wx^?4~{c7vr@-@MZ9uo z83OJFEHs5{586&AIz{QIOE;84ti0MQdAdnMC@^B=EI#7mc#WABWUxb=sesICa3ig? zJ_iF;2~#VuW2-wzRe&AMm=3(;tT6IKq))e(I-EnD4sb&VQRf0P?*wC%<;`Zj#Kbx@n!=915qdhI ziq?1?iy5`{@!s3@2l~3za10mTVF*O@X0OzQ9Lngt<*uVQa?xGfCG)vSYV1f9nJE&G zm-FuQ{Dr+3_#Uz+72j^!NUbqvs=R49=2z#owiMq{0?!RRK3gjIdtrR87-pujB(bN(EQV$s_WgW_3B+c zQUTHD|{bM5VwTB8EpiRo6b0@|#J zw~VAjI$sFSkDV&qy~eC)f8^qUf*P&R4ln2~M2N=S7Czz(j@rrsm+Una01}q!u}GS) z)>l{THMo56TjU0TRSqr8-Gr5^^$auJ@IwztaDPuoCTAR2gPuFGyzDarlo*Svl5qIKahxrFL_7<`?uw5tuj+MBK72j}^cJn2qeuI3RS_ED!|E3oc{)+# zS+c_~Dp)u+NRXG^{pKc5;ianB#M+lzEI4X_ozdng#cNbH-FZwguWYpMUVsRRMp(Ny z!>mCl#?z!`qP?17dA%Lw1z0ouV`pc)6r~YB!Z0K@-95{KrBD+KXdp*t88>Lqt;7f4 z>+xYtzK0AxrX|x`GT8T!UX2Yv!cKA^>GeA>K9Gawj(DTYwuh`eGLbcHIQxk4XZz1# z?#T@B)KZt21A*Z9PsvtR?4E$=49ZC97Guhj?n}2Sl_LisU1!kXgODz!dzGZ?47%`V zr0ba~geF|J1%v#bkuCy36i-snKOTMu1 zm2weD(srRUH>I5Yhzs4i;OnYsQ)*J_gB}bWoF{%RZX82POXMPW1UGP9oJ%r?5{t0) zw7LSHHt@6jq&JaQ_7;mM(P)b964BZ+BqEi3G-+@T7SJAm)P+K36l**_x) zIfeMsLJ-1qhB^nr3n^h|nb)MqiKPRW5WGCF6-Tb%6tUvWKHDGiEO?1wmUo+vKEXXh z$;|#LQWS699^@zV`Tw^B$nE_?iZMAHX!35F6P8$Es>#3cJvZ=6N?JK(Z!nhDs%x}# z)*-Y=Md)dRe3A#7oxWN+V&IkrL9RJ3A$@E4%QL zFRyRaFDNSiyjfD(W?7`JGqg0hTji|%b+5b-ztbhYwp=eSXlc{>Tqx-7;+@Wl!wI=L z>}5fL1C^h{#z9g~rh-pr7*u7bG}h^wlCoei-YG}pPg=xnm!dArxLg9N!y0l~?J|~8 zgzs7j5|GQ{KViJOozm`#THbCpLA?5o^%rEmba$CBRC899=FnK|qDUolg~nDn#5hJx zMg*)RruiSRQysRX2g^WB#n32CAqmcegzK>-6a3V&_Da^`P={n38Cc02@dca1$GA0d zRRtWr+g#j&l%L0t%q-7sg0}ZU$-P}T*z!~MUSMZ>x`2E zHnusV%i78>l7#cd{|X@CVTA@?nO>y@0k%;(omHVi?=_>PHaORX`iiw#k8s#DF3;6Q zd5xJaipiGPvGbSj6&5YhS>Cm(G-%U4vRke5jLtg|}g)z6HSuR6EG z*F~Dg`x!~$(NJDcxZC9B64bG$ct4H|nXc1y&JA#QUaq&t^cW6X*#s2Jjm>@_8airO zPcUyk6*#m_{XWECPBPymG4to=eFv*LdfPn@NIB0A?{CR*zNjkL-w%{+MUN-%1fNhs zR}H!mLzVgu4y4=cxt#OY550Q*1l%!qs>|Er&~Y2Pz>9ElEZ!3^yqytxDz0E9GMttCLbnUJSZf7( z)+dQV7TpV%urEy@hp|NTPL-}gxlpufEBUGK<`{rwU*3G$bM)ldn`I9mjPL9HO?V?W zHv3KhgCs!h+^|Hf74)W1VPa2r8>-?egV@pWyz&%F+up~#PrM4%N+fQh^WF=} ztNlQLfI)(uoa`d9>a51^`pO2t33_?#h|dm`f^_;?qce(^sZxbv6%zvz=pAV&kX&+W zvHNAxRLH}QN}(LJ=kr(6SNf1zpBT6Sp(x{Q;XD{@7HqZq_z(Tv)^tM!UlWJ4v&V6d z0V?Xl`o5$x{|CCaM!uKK7x_hDId*XiXO>DToYsoO7uH(aOMN{gFx+c`P+U&V!7qh= zD92HLoJ)}yi_~bqG3;3F))RHx$8_a-_P{{pz6Tz>K0kK;!3XX`9>zVVA-qbA9Qq;> zymnYF(%{=NEHAX(0HPu8`TVnPAJ221Jy-k|Wu=c?iCj5z<>nQhS~Ui$*Vk@Ziyy39 zsbW({CI0H}7?lSHP?X>#_BYVr$c)FK7d7fO$WFUViGp-37$w*WgmNzyI9Fg42Og~8 zUsQ&F9~@v`amxgPxbDZqM+I)Dh!0sF9C)k(Xuzr&mwsI9;uOIt4voo~wi4yY z2lV*>iBULl#cfI9z`^VaS;!o1Y$UEW>8mr-Z0zg`h^cL~<-B!Hj8;GR;l>ISg#^WW z0hcfcodu<>F!UGj1Cq6^MV_@Cexu(MlR5y-%CI1E!~ ze#1i~T6Pf3z-Iy;66yGP1snu-HWK2d)haDRYf+F?J`WiUMV!vd$9hVmUDdmD#z*GF zUrs6^VtC^TW>@8MYSf>1MN8oUbI8kye}LRVu`CBb=B=$%a#oaq4Sy1cQyf-_mid43JrcoJ5B z;ICu5Ouq1<>hnCUW|t~gu8&X5FU-tNk6a&~pIaCipH-^sGDSo^BYLY?kZT%As?M3d zrz~T;nT;gSpwdsF5!e0!SP_j=aH&j&O%<#K?f1&UHj6w*bc1D5xRQwQ)hBtxcu;Q6 zr3yp9)spwr8+Yr`81%Siw6$~s6Q1;_=fEQ{H+_9}cywWQdU`$vy}|%Q%;}uKM66+D zF#-oUA=+TmnFw;D_&q8Jx9DZuYA(|_rhGKkZE^w~gPpN63ipt}WLoTUg(|YVj`<6B zp~Ys6n;8H{*JQ8~CE^0JLl%4Q(Fcz9_4Pk=?C7xp$Yf|k$fNeRye2Dwf}Gc%*Hw9_ za!R>6iYTqC7vhXw_J_SiI zFD0)tm(@i69}-9SN#5}aLGIMqN6wx<^YGb6@>B$?>UN!4Ms6j+-XSU+uxb_D>h4kl9p}B-*S|W@G2$Kr9zaZ zHhSrBV*4E!z|okRIwZ8dyaF3mvPNxP%o0q@Fg_?qP-G#37|N-TjBm8iI`ehpF_Dlo zS88E8rM%h?DJ9qIwsNYh-xGPa=c0`KWvGBI_=$EWbisFAAzw7VWOUPMzy!wXJ4v{D zc+Jc1`qFt7CrHcPOVLjWb!7Llk%|Og#+Y0dng>QAWQEjhwpE)P(9(`oA&~9ez z((Igj#f^z%ka8#%4@)is6BnV3eaMAV{qss&?<##>8wdxWON|dw`95)6%Le_(s|&t zm?2mI;r+(OZRMWh2l}o}PmUfNSgO^b4d_{8LU@zB(Hd%5W|ZlRr!CbfFpgWnIk^xPCb(lh<1on@o(piXnki}iYS1%mTv z9Vs9va~eigPM-0ptlPV|crO4)&Y$+gA}QMj^IRz6m&_IQFfdq2sQ=%P7 z=Q<^HaRBuN-0QX+m1D7*ISrW+#7k0%Bwu2=R^sLZPn76b=Sw@E4g?zQXu!NdbXGW^ z=|RL2YS-O1Q<5&N$)sq7Ylo~*J{!Vthti#-gOZ7|!FvE>X%8b54Z6v9dDHWe7is4; zS#I{R0|ZQxno=iR@sA08%Fhi5vMOi*7Cl1>^m<Wz|lDHjmOl{6Grr7C~l>(gqUagWUhm5AFr4~ zXA)oYeaoBJG9E9Xm_p`$EvA4liJUwns|WD&U_Oi|rp`f>J*^I+p;d=pUJp5^f4E$^ z--=clP^+{PVO^(yTL$vns`NbY-0<|&*!b1!vqSUa(^Jp)ppn?=0vFtXNvd!P$3h*) z_zk;bo_b7*2O(hFc}hyC7AplfFvZ&g_nt0IuoO7)M44WfWWB338b-XoN|p= z+C~ZkDh3Z?2*oD%!ze|H6sZcu*fA@Hoq;-tkb{0d2LkT6;fu9_8kE-dTIhjJFK4|Z zJoT;ZHsvT||Bq42x5?QoL zvCX^$bo7C2AXy`Vr9Lggbv+UN^sp+%J3KOwl35a&@jg4QT3%M#jX^W9X5fS1s-db zi2`Npm@aC+$Wx`ozb5`_E$HB`#z0HfMLOF2TBcG_9NbF^1c3^Ijz8sV2|U3i5*yVzW6LTXwk6>R99!lR#aDg^!7ZwFN~BZ5oGV%S zwIy?JUj0Y*vZr&BcL?BdH`8(0bA0DT!p*H?GREY(`>N&%n=KZTwxEa!ht|Z(&3ML* zO+Aw33>DaOgPuY*@aJMPb9}I&arPu->PTu4TCk|9!dk@;)Wrv|sB~%GSm)YPl`1we zg=I9F!~>s2Mz3xT3+YZw^b1-bM%EP?InL0%rXiPE=L5+r$belJ5|GvW1GS5sh*n%G z8kIFGhrLi(KjK#gRdPd78Jf5;^yJ*a-00Bk@U?~ObEC5hGeh&&=8$M=ngy67$cChM zAUXR)A+WApzq&9!H8wpvDPr$}{z4%=D|Z*0t0(T(+UqA)TDzNKWFc5lF3S{Rgw5me zxdvQtHwAZyALS*eWNejd*A6?sd>5=>bOp;f z@(@L4CdTLH7skdXMv=p*L7btVom9#oh@xh8!h#YxEIUuBNV^NHYm6{73$Eh&vU{`f9B?gCMIOcAff65Pz|{*APtuN zl*DnUWSYes78d~77yumuYxjiuEO$p5j*D} zUclAk$Mcg%^l*{~`d9`i;{4Fm$k6P_!kNghN@=RZO zcgW)uzbh9Ro}PJfVdmP<+$exQJ3a$oD7_&S$Z5{-!o<+j)$2o7nM}tBDc9=Tk3MSr zigQV8!9DQd=HQqPk+_Xc3D+2&nOPXUIXpT;b4||nA{OOE&Cbt{?@@?+j++cRxByfn zp(m(?_n++%ccQ$7daautbS;jWaof{7?7P$rp0@xnEMOew$EU87s6073M^KdXVP9}y z!~mKDz{Bq$q!(1)9gm&E4TZ(_P6Nb~G^3qx0?ug@>cj?Q1l%lwlw5i%B1UY~-jIkND?(Cj#;VP0w(9Bh{WC>~*<&~%Yi zq*Z{REFYM)<(U}2GCMT;WHFOA%BU3D5RXrO2dcw&dTQcHnbF2*kU=;;WkJZNa=@kv z_Wd};lV`3(}+tAQIe-Tr3Wr<`A>Kr6&{w znhm|=nRjwd&tHSOGCVdhbaf6GASt6p!m%F+5E&bMjOfnI%OE*0Fk0HpkH@^_qygHX zcwYgEGT-#F?)0445FdD^61vM?d4tM(!zOrVIsjn3wk}SLcpnMaz#&vdo!!RC<1j}@ zRMPq4!tlb(#B~q@{^L`?rX^(5c5OLg6MG2;#`^(WpCKW&b~kC)=nnQ@-dJqb#kC=7 zmr)ZNGM0AY1|;R66Jk0-zWs`bx}BD_GL1Pk1RUBHYZ)*?5!OMqQKFzOhKgBbm3Orw z5SIj>F!S_IwpUerhp^8w0dij+ENfE=5GKY%leTQv#b168s|Xzz1?);gu(KQ^o*TWw zg%REcT~WF@4E?U){g^l-9ePyI>R{QtLR0f0{U=FMU{6qtr*aycC@m_eUtB$sEy5+8 z;7n8e7BZG}k~|=C!`l;oPzE@3!MlVsK@m%vM5RmLLF*Lz*ht|-I)2)eXxYn7P)yo2 z-oo+WHQf&7p8V)zS<)$85p(iTR(U=Vc%pznBi2`Sm|jOuEg!ekS?jAkK#^Kp1|*Cm zcf0A#1gyVSMhK&m(}@I0#6LNip-fv&CdS}uRc-3=12{&&DShwlb`xKwm&Cyddn*p( z8)y@&_j#+*?pYDw0tKAu%e#p)w3bJo_*yjc$|S|QKflH2v$Vx7XTqM1r3xmxxR9sM z(Vd7G->Au{7*TK@P!Ue^(8=mYa)?M`1kX6TNm$y3Ewdy$^xfCv`0&{+VJpdrcgO~Lu=DxI@kfLU2Q(J?T3=1@_bdIqXZhxQH>JPM83+QC@FMm zaC#8Xygeg5=V6bvFE^&b*jlZ(YK^-33rd}O(#;0P)2z$Nvt+d@x5F-(Q7+Djizq8S zqxCg-;p6x#p1cRoYW@P@?RP&EI;asS(b&oQhp&;zIBS(_%`Hi(n=oxdaX5+LmX{<3 z(r~klXA)V|`g8*l{YUNKmZUfk9?NTMgt%Hl_MF(^R@Morng|6|g}#>o zDukA8P_s&{IK3yCEzt3k>gi&4N1ZnFQN^MgMqE@BIYJWEgto?q8mgBJ5u56VO~;aR zYHjXN){bEB8Ojv_3}Su-d^9pLZS!vP73aa!pOG~Z#s64gP{iU|1{I2=SNU)<(NikA z+o;E5mSl=XO^UYDa5xHp5*rc}56%xrxvEr;1$37p7 zB!eLm5BilvJ-{N=Yow8qO+J7~Lki!5JBAGm))C9cMHd)#lY$be9!et6C^*SfH%C?Z zVMja51_>Pr46XcNJ7lC#pzGZ`%jK~4Hb;P@p27&ZH<(JEvqf$;KpkK9vGJTeo~nRhCa--@QKO zLmE`4B#YCqjK3fakGU>6XG(DwnPP1B{sep>;~`#aZ*)m0#ri@DpSHR^TAPJ`WnS2P zKz{`VJ7+`LWbo8J70OtS;gT=6n6m|A;jvHvJ!2zsLE>5B`yy0a3Xrd*r2t#xBL=}{ zyKR=on=VI60mNG*YZ2Y!%~C zi1B2yQgNz)Ck-39x+KMSBn`U^1$vGIfh8r->5wD!tH1!$gm|w>G~UkY;@MXk0UwPk z$&pwg8UlIQc~0T`7+81k9~YfG|5UOYKTRPUIMS$it5Eu7M(0-=8LGxt+lUbAFY!oD zGne}06_ONv>eA1#<4*czzRWjAn^tBGi87ac4h9b`!IT}2AE#*MQips!3;YZB?1Y#! zNkJ!xPZH;ng`MJd@o}80TA>&Wnf6l8Dc<@Gz9}1vF7@Daw-e`k9_IyvRLd^j@B_>> z0!oUh@7U4 z4l2>{H=(6@8Cb~g?-gkZ6EGCq+gp(prZ_Hq?(fGjEBLMoH^9MLA30YPJ}^*tT?CNu z?+aL=#VvqQ5h(e6Dizb0TXi9R4ltH`9$D)CPzp!B!(NtOn57s6^q>UqK+(%7I>yIv zgAx|n6${9oQSSF-1G)CkdMHM9l#`AZA=Ht?P(jc;bL5p{;fdZ{EO0Yn@-e!y0+Jkt zbP@Xnn&_^|69+j1HO+^cHJh7Q>hY6J*`O{}vsZm-?$=VQDlvATDyxNTriXjNQzgEu z57;AsfgeG9s|2EFrHl#;;T3qWz#Fux%fXHbD{{sl&#vC%788!P)f;I<8Y9EqCxZ+r z$l{9Eor%y<{!Q_WowJ86G4#PWJD_!5zuJa)c_r;|Oa(S8DP=Rz6IRHRtEp08@crl@ z(*{d#Cn)5Q4(af_`GXn@Q-!&be*lPy)2LZAI=}RqoENuXn)w7Plzvl=H^{%BY{~q-3es zuv)8bthDk2u=~N98+WTXUW9PqI%Y&}1D_z>RVLfXP>?4tjviCSNACB^!F90gp@zW3 z{v8rTY(+T4VY!e2MRsKBG2gTe$UFyQfXJadt%)JPc{O^3ph4z#-8?P5F<2FDw`Fmk z=L*1d+1!91hDasb8EYpc~(TjrS? zI`2drTyy9GO_xH%m*h@9&*^D4IrN@UIQdOZ!r9M>7zWGFKkO@}$=Rz(br`fhb4WZE zpT6IMk6j-oAH7I;H@(e0L61unakqnKD>? zo|VYx5YbT<|GU0G7Q5u>(qX&uZmPipk4QibP!((q=+9x+k9`7|D>xPZKCnnT)6&>Q zPN8hj!c9E_X*O6t7|g87QC_xp0aF;qdt&RC6{dIYxeqEidM_%pyYm{)V2P?3uV$(< zlHLf!%Tax#bV9J|#xq(iGxBmjt5&cTQ7l^sZfDs-Z{-q7j04f&)fHFZ1Kz@0JwE4w zOouLB$Gk^OYanW=cSx(B!xco~C5);XEp?%`h3<6et;rNjao7$^i&p!3;MzqWsxY2M zTfJ<~ao6EYzs#HGNCSq|XTPH~m|lH7BaJ%9aX{>oGLJ-Zcwl0C@j?avRL=FE>OWf< zZfx!1vibFPrEmCH<DrCMsJ8_|M_wHR% zOqPr|HVD|fjE)ch)%u-Uvr*>}lM*kSn`s{BKlZn@xPtR;*S49ea{tklO5f4eF?_4z z*{q^{D_Q0R!hxn2_f1xqr&ZkmLTcKMGTM!bq_ywxyWw50%k3&WY}PO(j<&M7jVwFb zx^Q&mLM5x!(F-rpAXa zRJN(Yx1}pOpUI*Kp9=a+EEt*@m&UTrHg;5lJg+hTU}G%1hGz!PO*~KIS;y1H<9~0! zk9+XE5zm|Oycy41@bK>ir2C_H4`v@k_6lz$xI|AF!^B79=%-y-xsc<^rV zHw=4_@7vjOV`Bruf)Jjah;Oy4@*VtpYY|tbGA({HdUG4N$E!Gy&GW4-HkWV+Zk8$k z%`zd@$g{T0Jy{5Gl@KNrLre=?SqTR(mI-g+6>@R^?V*o+9i9`6+ai)MB)>-3T)Ybj zaFJ2`jbbH46WLDb3fn7nB24ORH|pD)OI4z5##4G20x>aRvTVmH!f6Slf>~?3B`3@g ziR-*y2sN%4PA>zw+sF+VV*V3mkUKFmxaFT^?oy;j*z1VgTiM*kxr!zdEZSkzM15#K z=ocp2+mN0*9e34L8mkp-t0q{pg@wEe_1XpnB7tEvoqdij<67IbYGo0sg-^9S??&}b zbwiu6#8bNKC+gKTk-W1&MpSdG=hNVvQuXf1$ zpz{ddhIrEHUqreu;{8|fK7o4H-#2L0X4yJ^{UN^n7d-cFD{p!`LJ*|&M&lWVLI4qp zio=Zf-NZpQ0M#ykZ*Q4ofoUUo5Qen)9lje0#C!mN>!6)LE0)Iv5hE(u>PCHT8*!I` z@G;zVdge%x$_$`gvYLqy5>AgEX&1k1o6oUvj zdk_~GZ8dlwfz*BJ*6>q|AI8Hn_#5+0#)ZEx^~nfv_$0j#?%GCUiTp%NKS<{T&4sDW zElh<5CqMpIVQ`T@1yxW=nkN6m`T!>$YyyKyHN)khu#jv*M#XM%uz}h#Bv7P@@^cQ- zq|&}C^+^pfjwNZ(0&Y7;OHhW%pYof+A-oN@*VZeWSXu&Rn+!l8iZFn~eQ_n5@YF;$ z1>8V2VrdmwotGI6B%cbL7~>wuVcZX4x6o3WjTSC=hme5>^7p{-2|~WIMCBJl#?rIw zd;ijK)}DtvxPy87nZfKs2><@Eq3lJ3Z~NPW+13vaXaD%04Q6eGKl2Sk*+&pR`fGz3 z>CE>v2c-=DL#cs+j|((4$U59k6%ycw%cU>wj%6Rg^HDtfyI1~&Y>V(?i2FDm(h&ZA z62G7P80Zwj5B?nFON2l0GlSX72>&;PABMd6pYT2Ve#o78|1{!n{5Uk>F#9dM z|2@1@p8V&?|2e#W0P+48m;XQGJInn9%3=O>r29PHe;MHy5gxbvKZo~M@c!edcclrs z$olP%?PFU&uQJet3?@*>4y64ewq?s34G|(&H?~^@j#2`^&yg4rSQHLz?J24Uzfz8~FK%{qC0E^l#ylA3zn z#SKb4yP=Gu{in`?jiN0grJz1hyBGottRLc$NV8e?;&Z?;JRigJaXg>ILwwu8c>NN7 zzl7)0cs|45cgC_85dT~F{aHMp!$Usyd4yYt|02Ry@VxMW;p|KNh9`SJa1_s5@w^?+ zyYN)-9LK{y054lY^Er4c7=P}vWxVdTMIRf@z?T?bTO}MQXSOqZ0&m-bPz|28uDT@m znNi5cX?-%7r0AC}fxqFoi020wsyHvS`688B{JN+LgvM1HtNZ~$0~ge8RG=P+x}HlY zz=8F}X5Uc=kN7(ZOv6yttBXxnf$<$|f#;Kp*dmguqcGC@>$prWF4KD%D=jER z@WV0bi;hM}@4P;007`_d1XIn>COiKFW7)rX3Hm6)2me>lIfVBhoIv>f2{E>e!m~9|jLb!$S>6bw-5uQi5gYcgr-3tic@XMgH2tR@N4}=1@s5ucOv`*!t;p#6vAIX_)7?X80lX^`1QX!nEg7!hkgxo4&epV|2c$z z4&iSj{L4uH1%&@M-v0pM-$m$uuv}-0%MImkxG#k9&_?U0E#&)hRZtFmR7e#==zMD# zq$kK%4Wenb3KI#_t~MD9+=BWbVPoyoNxnDMfEPR2CaOSEU^!{sUFfPSczFLT^;L$n zgkuiZ>cTVFq}H?4tSQ%N972pC#tN*N*g)ZcZ0s(qQ)`BEM3|sXu&x?p(`^}k)}KB4 z*Jpor>|gxk<182Q`s439F*5(h_q}b9Z!drOUkobKray@JJSo$@P?2LZ1?iV+BtlI8 zgJao`{W^FK!rtE;lsNwLTX;YEX^aKJ5B$SC{?Ft0zeoCWzcHBoFdqIO}aR`LeVgzNei1!VHM*X7STlQ zT!`d%Oh$n%4&xQZ-A|jGD3nl#7sK>~>`2X*L|+9&6=B$F&Q5Ku%4Ms~#%7-K!jZR0 zr36Ba94KUDn=M>RW8|Q|vF$ zbxRe~>YZ9+8^}W|I%rWxM|4c!(U;fFREQ|6JMIEyKUrFUlN#&n)sMX}FSNxz0OBW5 zS`-tXi{9D7BzPK+q(L5>W;6UA9IwIrkGJae2HE9Pq3b-8k`YgMpT&dE7_a0#3 z+~_=yU&@9jrsp`o46o0QW;lmoX!fcC^XX^jpPqSkGhtMUV7;Z zUzneN$I{XV-kE*xOqPB41CVd@w^|;2k`tTo=^P6vFuZL zUc&S1c=$It{e<8zY0!J{&hX0g^aQ{0_tf}Ab`{4|%seq9ark|8V)}}_;I|wo#r<22)bxBdF*=pu zz?R{m`7C?)DN&pmkFi058BZ^Co(hXHFm7x9eXnZa`tkN-W5-?e`_n5`px5#biX-}u~M z_I`v*pB>CTfbhN0$9@#yFus2n;g2BxBMAQRR(xlffAjx>y$9hRA>RMUWN722 zPP}YJ-%YGa!T2p2Atbjd&aB9|9?d+E-)RocR+xcdZFSYZRd=e(+lYoZDZB(6il%Mw zE^aPkNFfyb$?P~R0Q!wY#(WJ|A)|C~o$Y!JVw>;*%}HH=WL*^h&6-|p22R1&RGj>0 z52AK#SHL<56fU-LaTmP4F3RUFgjdGhWhDM~EyNiiEon3(2Y+Ia@-UQrv-p2VnM#I&@4>TCq9&=Ov|{@}jq~=%LGf zC~$MBLH=YqD)m=w2sA(JC?WRTS%f8OEes0Ur5zMb&-NjK+%Kwq1TwAZq9!C95f1=( zSJ|>YXfhR1Gmq(SQ4{PpSAWoYNn?(}@UGH#>7MZyd_5A*|m!l+7Ug?l%o(Hxd3dgij;;;5B zmJMYeK==o57|MPW;orvh4H6v9@o0;*qN-dni~;-aJ!r*UD4%)bh_dn0%N?i;ZCicT-Etn@8{ z63IN%Oal(Z0bO43`2|@I19bc3NvdqwiH9op^M}J(aiG$tHVF!`I6nx*7Qf%Et-vG- zET@u=vny5HH4=Rn457-% z) zgr_68>mItUKtW52al-HbSU@hMeFhr>Ix1w)u1c>qRzUA?Z8VdmMkZskXtwrsz6?aT z2~NDowz|cqy-u4Q^lq%V!tABCe1v}po9MfK0(Lam;BK}cf8zIXJZJG-#500t0?$02 z+jv&+Y~X3*c|V>H;-QVM{o{DY^FchcNz-0UTOQ*+gy%&(AH}n-2qZpQ(4jdCaoiNVGv{3v`X4?7r^OHl_EC2LrHu!&DWpgu}g##~#%J4%E zRfcEhFI1+A>ny~7!kg1X#SKKDN5cL4rHuTaSAzZL%)J`xp}xKH%Nm{>wmdI*_~p0Q zFAFxaR<@iq;P(SLIP1@_LWuvM+u?#Sh{ZE{A1~ggC+B8HhcjGv(hs>gs|+kxSj4?< zK;z3j-;#~|&7th~=B{Kvh=<{bh5r6L-ue733z;TP&thoT^3eO`p~mO`7W-pgUhouO zmcMVzJoIurETj+lJpaJ5|0)Y#8S--V>pMKr3+emke@)*%Z{zU_!&koi<^MqBK=_U4 z_50JKZ`C5+pzp6#*v`i}#&6BCFMlh3f3tph6U=vCr$4jr#BW1`?5{?@e+<8oP5yp7 z`u!8p?+-=4e=_?0SEAqlZS?z7_vB^%V)Xm%s0C$-uJ|^D)EhtaVXQA8d<7xnuKm=Q z_@I3hA-nlSgx`d4@UOz=i}0fe*}X3!d>7;YyRq!s5PlNjUqG0>IF@}of-QvKf$;MP zzYF2p?BFC)B( za5j55JBj~JXJ3!Doql-NLNYw$&2xm&1n8Tr`)mR`Yo6JNKpO~kVHR#dW=`{=7Em!;i7 zkD90lR&Fbu{m%>@rvMR$$6~f5?P+Iy*)gqd%mXB9?`RmBPh&uNxCm$b{Ls)JgX9cZl;NOO{T4BhsXdydSJ(Z4`t<)(VsKfsW5&+fEs7XCU8&3otRiw5?jkQRp23` z^=<+pI`;gsh;|T%Ip&llj-YMpD2dd8<#DV?8(16T7V$v47MjZM6;NkVK&5r0svJqp zp%~O3#?oPt(`kpn{&ZC5Jn@8mVa-ZmH%oBF>p8OzytdIJmNuEqW#icdeo?C8tmQl) zUD?DhLP_C4n#mlSLIHO1ZAE&{){w`LHgMkDkrMK<_`MsRetBXC3)l z4OFv=kZpIoATE$2lF||rR`IWM1Mit)iNooHz=^n5lkp&}?gK4a##>YPOD7e{;YkeY zgcRv!8@abp3h65GN&n4rj-dPvjMgIRtpit!b1bK){7(3@R6>E+tGqv^p348$q_h)~ zf<3u|@|%eB`JQv^k!%{Z4*|MUNY}}Gh(%M`)zWWR0YL6K{MVV&`@!C`l{H`(aWJo; zB8+3U@O>2RRPJ{l=IxPeM&8!&?;=WouN`t-#3(V1$13c#eAf019#Y79Chi)5D*xlY z16k6QdMDAMWhsTcZ51u7WKSTDSfgBHIC}{HRX}0V9!wzBlC-BOQe2i*A5w$QDDuW3EMEjb)=X z#I<-i_08bs$qs2-N}YwmS|T;;c49- z-MImMOy1|PQ)~;Ko>hrMLGxn1n4=rs)os|5p`BMPt$x~qkMlg%Mxc6UA$=5iW3kpA zYcvaqhhZC_J7y~zrax5SEBmJH$v?0h#wT@HWT6a+9iu;)X`8$Pw0dQGb(M?Q_#@CJ zYB-&-xqAcdMBbQLai2vB0CRY|$qS~efJSr5HwNI_L}QJx4BvR36&|TT4%7&s2``qRGjcj?jmB~>+=s&|#4|Amvzp)hvclhvUw)3;} zOT?Nko?pWAX*^%R^Cdj*{5h=E;<<=t2G0tf590X!3|Ai{Y(>v%qZ=S4jHKGjeU z3buV3=6yj2adf~79IEJ|g*37wQv5n#Kn(l$k;lJnnSo@Sem%|Yt$5Fam~oJHoR@+Q zsHdaX>iJjT&Vkhgln&b-J?fEu2&+ImZVe_OcfHR>u~rG^3}TcQkzIk!P1@#X!eFrr z<;uCBFyA$nQ5>;$7Ki?9H@RkE?#{K1U943u-hscl@85~+Q7t&3h@EbRkWe50m1@04 z_a>>+TYA0zOucc}+=B5f)T5osh)`H8LQh)R_$C?z&~hP>>#=G7`Hs!05roHaj7{_k z7{X46lh=Hi4Mk-0zJ5XP9D#(ajHx?O`FE6qb1BdgtfaxtHa;*-tEKFCVC$49NV^60 zs11-{TL}{w5m&L92g|oIXrWE+<2nnQ=7~lfz?D(yYzO;}@wn1#Jc9;r)v#r0q#@qw zVT}(k1l`e**V+@f!r2Pz*_FjR4Xhb*JlIw)ZN|8Q_A(V9n2+$7Z@Qr`PmN|qF0)7x znUgqd+M!iaywb+8jcg4DORlfM$g19msPARWpU>d=9G)-Wc^|m;iR|6NL%$!=?Fr}x zQ%Ls=xciChJ>cSSCzkm63~Vc#+pcGmAiVS7eK%13{Q5TNdo7!TvnamF>NdVhh%_z! zKsvZn>cThDH|;0$NhI=a!;mlQBi_(EC=Esyd`9;anDqZqHpb9sH# z*Ydb`qfEAS45e{_l@c~(4Xpf1$al8t-PSjTv^NhSTthgAd|L>oAtkaeQ|LR_HbxLn zAxg`qtf*3;)!t?B8odpw6Vs~2_G^0@=##!{ zyHtPE7;j@5Hv|aqZ^pt&gad%r3jW=-`uWEHjzj9@cm79xP-B=%Q(nuiK@#5-d7RoK z|EPmFb z_kYmzU;MUVXc)uU0w6Pi|M;ix!+-;0hJbscN&I~FB>EV~O(N|SV&;*07GJ%LDYSMH z@w3oyhNCj(@y>Ytd&50%y!TCSe#=|mcI4~c{`KGRjqmuTKljb={PW-Pt?&A_zwqtf z@txoG-Ie=#?tkFu_w@E1d+?#-C;A6Yo;rQz?74>@IsfQ|$1YyF{P^I|m0{d+boJW! zyWcZ0IW;}=y|Z)k*Ppm?^U3eKb^EFJE?|ES^1cP$*U@a-B7e(nz>-OC@d@L3ICcwEEm8#EmJ zfBvnu=kI9v!t)kRX*gK1_y72BwBFy)F#Bl>H#K~D$ii>Y@P$A4*INItYWTuWYnVN2 z@z?DAK70SqU(|N|6AfSfguVYI4PRKa@S?@P%fdhWeaScYTNeHu3x7nzm)9&DwfF}# z%-(3>@BE(j>sK@!{26=yAq}%74PU-u?+@7bH)%Nd`@gICe^bNk6B@q!AkS@<{6TwfS@>QH$1Oao z;mhA`?_Z~3_MiTxl%M@83;(f(gMVMcm;aW%f7rhNkcD*%Z`t>28oqqa-tV{X-(=yJ zKd<%tI}Hc_xrQ(N=Ksgumq%08#eX03P-IM#j1fghkuKkxiljoJR5COwluA^}MJb}B z!8}zejc5>E_gorC%9uH0rp)u4cb|Jt&-1*k-?P@c-nHJp-sdcQeD-J0`@8qq=j^-p zCEeeU{64AQBJl!=CrET;Bh7-1*cy^+kh(1CE@&p9R z@-jnxHX^>9cI9U>G|yjlGfZj=@}K0`oAd(K?UYpS5!FC^)a=my*YF&Yg8MHJ+;%yA zZaR4X;_C9XSJcO-%<}Y)Z()bhe#5pjls>Jd;rMX|?7p*HAxRy@&lL784`Ko@=S$HK zYgZt?T<=;<7Hm4{@;Jx_`KLZtKC|Qy82j1>Y}e=MbMnFe%6AE zsD68!JZ3y8g1hIrXNZJ>K9*b@lDM`QY)jkQ+@p|ge&gx(>S8d+4pbP@9;2X7E;Dwx zl|b!alVQn`w?g3IKDX+o5dK2|s z;4Nwo5ys*nhgZj8O%C`Hdl(9;m#92Fm+4 zRD-=ttiZ-ssC*)x!siRCLAF3>L3W@L;;tF?HZ}0e=kx8%m#Fx~ZF%lK zU50c8*@%PRf!n?9wt3ea)V|X~jh)}&QDE0n=SJkecg52Vw_0epb+uk#HV3MobtNx` zYhhesw%V>2R8+o`vD#wYE)kZ=CiZvA#tl{$5tuS9<&rCevSs%QOpBDN1GrUi zT&y)j_9Y^ldx+8qXA)JepPY%(r|pbaIMWFCB~mqhCT~LJS+OAIS{1%6+{|o<+b9!D zZhFQmS%uSrQmb!Fugk**L(C@@{56j$YL zTTD4DiTv~a@QW$It+}(-&h|#}AA2<_xwHh=43(~LbQQ|Pdav^@$SB6O?&~}A-O@qj zi+|d0QjDA3e)UVhQx$~|ai&%l;abcG2IWSme92}dE1Zh(dl!PWKU&D4^g&Nxq7YA9 zD}X(DvINCn6|^I$5KpM{G+Uap7KI-Sl9w*T*EEag9(SUk`fxk{$gTj-+WJBEl%zhY zKYfAo{$FwCRXQVBhyd!e!$)V=VH*L^B`CDxn)j-?^b&redDaeX6OGEE{16OSSVx6&1c~D$j*&P@;s}YuBo2`{Na6sA z{Ur90*h^v$iQOc2k=RLM2Z=vPY$vge#8whpNNgtY2Z>E2Hj>ytVm*m!Fr6iWHkycD%5s8H)7LfRr#4jY~llYm$JQ6>Vm`h>~i62S) zK;nB6-;tP2Vip@Q5qA4CXOYi|BvMJlNMs6=I*C*gF%p?Vq)sB0M2tk{Oj0M2N+L!g za|WrCNF@;?kts;(BvMJlNMs6-I*C*gF%p^lq)sB0M2tixAE}c_B@rW$$xG@aQc1)} zWb%+YiBu9X5}DkjP9l{=j6^0Esgp=05hIbwN$Mn0NyJEGa*#TSR1z@~ndIjdX(Uqr z@j1r-X;fgl(Y#d5<-p28V^}~yP9M3|FIlb)(t}#{K^6jtGwtIT06ME@Zl`}j^C8Us zu1?~5@PDrCfGtAvDUA7f^AS@hQ;ns|IHLI*rZM|dxH+iuKV8~+2F+(Nk$s$A+kj!3 zWT{+-=FezCxz)}ZIwc�(#N>2s^eX!_Wo-OUHi=u2({wDYQyyH+T=$UHZIb6`|LE z6|jT4r|TWHY}X(@+L1YI4-3T7z8z37LA-uE?vNvxX@y8_uwRaNd`xiFUf8b{mgewF z6Y<_s|4sWri{XoNZbkD?EIi=~;{dD~UOq3_4$X%#DtENTA=pLt({kExhIsN++8rmT zU7&e9o`=Y9{M9;%BXEIs_rlk^YKSw_#4b4l{l{4Tb@ymCE_ovQxZA}OO<*feQSvHlT%K9 zJPFPFi(YQIN+tYn%k_4LJ;&gvb|~?D!v8^T)hXx}-QJc2M1ClIKK-X)iN8XKu?>-* z4f9$adw?6?T3Wz6BK+lF=KDQCLu`gpbtsyjVmn@DEj$e$IA6Wj#Uv4zpy*bd205w4 zXZ62qKwS3r%j;(#W8YF^nJh!ZOS4y+o`vF={-5gwi1=@2Gq}CLgR%UwW-DP|FK=yP zcmY1TM`VW@(f;9-dXICUu^=Q??jlkD@oA?u&x4oz`{xrDggq(p&Fwf383RWG4)78A zyRd|D&l{qpG93M^2>S~O)m}ye)nHS*&lFX}DUoF-Y4D9VKiG2yVIRU;DMd8!uqm)! zv}Ot7lP|BT`+%p+k`p@FME&Y29KGTLS^G0S3v#O6@00}p|Ra=>Oeo#MOR&W6lIu`1O zY(w*Z?6Pv{(u<&zG5T;vyaM9djM9LMur{yeNM|;oI~rEBT!i=iPu5ExBkIHOW$jvj zP=2iCIrj*0!Ud;9Tf9Hyzu9gTHIJ~LKj#bi*Ylp9!cEgY?ut)9^Cv9YxWM8RzF+Z* z^Z0yXzEOX*c*q?uJRMtlZE^$RK?h4C+;Oj+#k(n7H9bTT8yT@)Q0` zxej+7!?_>Mnt0exaIeqXuOlv$`;57am_ODltXpvmH=BN3uxvTOd*|*+J&H^C&NN{> zlSABINORm3=a#s$E@dr|KYn=*LsvY&eXighIf55%{~F+ee;;G0seM87VJ!Svc#$)H zVav;o{$s>^&s@oVt}`AwLKDr7K=XU7jiKmp1W$T#`N5WCQN(pUTOJ<9Q+Ek-3#p;` zE~dLRvDXPd9r*10qlHBM=5BJ(b;5UDt9)|Ch?ox+9-K!zgdd>{lu&hv_SkT@Fy|nC zPRFk*v~v;S{L_B34&p8cOEr}q6Z3uStoHT;_^+7PCgM}Xe0GjhW9)wXT)Nr4jE8Fw z*DVQZ*@q99v?V28}S{sX`3AIFDo~Hh&_&;m$1Te^PBc~-c<$!kVYScD^8LH;%irHV zIAVaFCox%tje|D0gc3hbjT(_Z^Q$}kcHl0*Dzq<}5YH1-o&93l@dbHiQ}K6}5%EbM zPO!qGUYbNWPZ9BjtZ~w?#C^8A&3NQaJFuf-6MpARP=%om(Z2D*(!(bBF>U!pSH=kYw_jE!Zj6`c z4GF&6PQ-7Y>!-O6UzW}z9xO-b)XGI>Yw+~+HT|pP37z)Nvyy_ltNvUmzps#vc=lBZ z9h}RP-}pxzt31zx%{1}UXtTuytI_irMicFjP{uFnZv9?ylPC|idak4t{!8_WmTid? z!T-Fd#*!n%drRheHd5EI5nIbf+8UBmNp8SKsy-W;t4Xd$a$Ppkbl8ZkB)K-pmy=wR zjZ_UbGSx}0M)Ds>$=OZ~U8!;X>(zr;@Npg(j(-Q3Vm|`P! zlH}tgA7djn%0}8S$%jZjz(#668=1W%?;&{?$vfGIb+D1vPVzRAx3H1g%tmGt$s0*t z&qi7u8?joF*O0uL`Nb-9mzeDl_He&H?q{WgvhU8Ifq(-ul8BX#rlHXz@?Is(s8zjF@ z@(_~WW!oL|4jZW?(h^Abc+xIOq{gv1Etb?{NS#D#H0d5ix|2waWOG^ssfUv~iPSJs zzfHQ6NWDe6-((~62C0)s4JGyKq#i=*B+{;t?pH~75~;y#P75OSE2K^$HIUQ;NOuyc zmr3_aY-IYAI*HVaq<(?a{YafenlI_@L%NelrLj59o7B&fI*HVCq`MdCP9pUzo72va z`e{-pk?Kk69;AMXjZ_k8?xg!kHew`FPp~=dIH|jlI*HU{r2A3QokXfDo6}rK-I>%$ zq#hyl!=yWjR43B?5F42XNu5OM0aD*j>ibBYMA}}`-H~)Bk?O$aG<#CFBXts~dq~}u zbSIIzn{?mBM&?dZCy{DH>N`k%JE@aMvnJiGNOuycmTXSjM(SHhokXey>26NClSnmV zbJ`YC-%RQxQa6#hDXDK{Bb7v&3F*FpjTni)@Be?!3x0DZhP^ScX^+Bs<%#DVES>|) zHBhg4?YzB(cs?R{k-_2WD@l%g#PbZX2d~^9B4>}urg_Bk3YN@`eK)xKWwFx*CF1!9 zJH$n6FgV!qJi}n@%)3m;bZn2 z2u#cnWt`nia7=(YXy7mMI`Z=b@w|lD3o?fc979IxZb}o+uUPmDY{cLW|H*yrZ&n^hpiOYmPasT>d}B3mZv z!YaR#FvbbNZ>&!GS|}3wj#*4jFn2P%dUg*h{csT)7Z{%^*BQI5Md&|7sodc9%(-FM zWeLGK#p}62?w1ql z!l$HT4G~^dhA9Z4^NcN>p0oHJS^F6fU>f)B%0pIrAD5S%2`%f6rP{lR6aF_T*3X2= z{_~%P%USiMtrRZ=sX}iXcv@KgOO>g@P|#5-bs>gjKcy}8d2Vm4I-+_rNSxt=ASudzC*5E1A(xK=jSiFkj44bSNnfjOc(DosOK@k`7z z7X{wfiFPT53ohXo>HAe406sQ9ue!S_hpi$RYakJTp`R(suC z$P|O@)!vqY4J!$~P)1f9N+U%-_J3|C@+T@s69>2O?Wzh~EPIrfFBFG_!|8&)yIK60 zf`$Zmndt0P7-qHCJH-$Q*poiL#bMn>B0g!QUJ2;UHdv~x$;z*xGF1}9ee`0NY+>2& zgT?WZK+`<)G=s|WFRqHshAZJIeZNv!{pI2k``O@e!%79E9&QfkMPs|4=$kfbqqV8L!Ap=yUzNaS7p_w@T&N^@GEg{ zOJzE%Jx}WO&IM!Jvc{9_bbyoj&*X5Uof(u&tnfY{s{n(_ z(P0e_S?L8)HzCBqW1py8QHARFVG+Fb5m!^KCdMBcr@7}M@H@?rG3r+*xSB=$BG|oZ=KIx>OA*JywiYe|t?X4h zC-<@JhuacUg7Y?}UNS;_g#Qy(8cMMJ{oBm;#jNr^wBDfv16#8@158-?UAW^i(xsWR z1jJeP8)oxH3AipC%vBa6@`s7BRdwY;y@d8-IhxL^X&nCb&T9wb}TG1svP%WfAM0*t=4@**IOm;QhQZmVd8r z%Vy&l4`+^j-M5+0waca@@q^hZ;^lLR_f6Q(vI0q5D_PpDDq54^vE^}+xSPYg9P^~L z1h=R-Dv76KU%MBrUPW-uN|40GTJ5fAJZ9-0l`@jJ_Q6FfTN7F7DOL4I;G09ES0A}d zCH$9E;S#v+BhBeG7YqsRU42ag*L?3j+wijl!BuMPB=F$3&t}+6jM5P=s#zw158SwV zcd3jXp?iLx7RQU!zxnOSRwKA{ZGkvG;1l2T`Wx}S2@9-!E{^lz{iQJ#CInZj^A^Wn zitE|xn-TA0u&;Gy;<$o`-Qmh>tn^*$WyJBs5dXzX*2@z5%!VE@TkPRz8HH~&+xWud#<$YqtgnqJVnHb)%cKqrkMOJzh zO|!)C-Z6#l>8BeA-Q`D}D1KY5Q?TBLRsS5#&qeVR>xiE3uB`gG(d;dXTUd$g89vEM zPqD>J6c-gw<9&CbpQw-278z0e9iywxDW4Ud+S((6n?BO*_FAn)#Gl`ai{R9(#Z%mC zSoLY$c1;Aob-!%oJSmnvq_iCs!N2hSd=hWTYOhu8Ac8Bow~VKxs1xDw_E{o0*LkDK z6M8KF{-yXVJWp7li|4^cLJ#_8Hw(X7yGiJa39EiX%BF>J{Y$0AOnh|`m#(O5NAtZZ8FOJ3AzRS$ScKx&8$ z(M}NOw_KigXV8@3^^Ma4_>Il#DP?z9?fJSXP5`HlrrAw>WVKh|4-mj-Yew%rWYtWR zXGe1#KmM-uPTjFdR{JQmc=O{Ib2h#S^JJB;w?&2@FE4)bao&RkMEtK?k+pi(OD{V# zQ-|QXZQgwNrkI>lwsY1Ke7sGD509`8|333AEBu>woEPVKVaUro#mfKjpLV=>>)wQ+ zmK0WcsvXlj`0W)+{ByEc{ja_wjt6%u5*w>4VucUw1Rh*(-u11|5?SL3wX2RBuelnh zIE#mPpN(~Pd2{2!rlH?CiU1C?2qvL@L}1vSpPH! zzDW7_gPDe`{NC@6CTo1# z-3h-K77tVTg>hE?EW5m?7~d(@>*^<1^|7uSpJZ$Tbx?z$bAsM?=`2{Bt zUv7{01mn%2TjgJ@SoZU<7awO>McnzA8_wz<7y72h7$-w6aZ%e?>F?-=F~*#MTesp1 zSoWzn;62LluMgN;Ho%I%a{wP)(0@{aqnKn z!yoNe1jd&V{Me9p5953d!^%dMRbO+4y}KEu?}|Q0rm@N+GBVxC@Vyu{ee4a(zDGyA zI~XqCbwwsLS@9K)PPa3Z?*!g&abe{@e$2a#5z+D^y0?mD4~NF5TNtM8-8LFb*7znn z;oZ#0k8^up_)~^RKX+ofiShJQyWxTTtp1=eIo-%uG&eOyD~DCS+)3{S#@9rt^^ap& z_qX`*={iQaeWJc_9qWD(KQUdy_&g!HUf)ZQh>tcoUCD5-6W@H9#z%1K)N~mm;@PSn zcbP2v@12@1VQ_pMf4eGcCCmS>=|aX4DT-i=C#yZMU(=r%{eElREVr`8BigU&9LC$x z#)}R6SoI-0J)On4FJ@f9Jjf~^b$U9DVRxudXb}%9KeXxT2aK7*SGR3@#Ts8w7XH3w zzj@(U>ir(Nx9_`s|J`YWy)Un*8k@hu`PMiYtyM3>**@8yOCbJVv3w#)e|9gVeR&h9 z;t)k~pm`WXF6{+d{$1PrLfq&VyjFbCfs7`@lW0Jbg`w5B+CwknUJu9~_w#R-_Ob6==`+XsBZ>MQ)WMbICmAJV*3+6R<lhE-M|<-1qo{`E)tp|YJCu{_h4;5_|f*P0XJXQdw2#{iYx8&eJWc5nMx`!S&<*#o~S_T~lpo%o#$lUS^Q= zyt5zt6uzyWmFGs0Ow$zn#XA5$d^L*?5BgIyD>h9YlN$gp?m1UmqXQ|^v-hR)QwAW> zX5|Gr7BF1&_3fb3T`j=Do`1_A z6wrFO&qQsYY{#U#)*K#$Q-S-6Om9Y03MpBy?)VQvuVcvTyIChFwI$`fs&@y$no%_1 z_&I{2th(}B6+Q@!hMmvD)}EsXeV>X*tr>*no`>nxo#B)ROJh!@jtxT7cAH{>7dPpK z2MX>~%^rg88<|cQPhF$?n#EWz(H?>)loV;k85(_7sBPXoiy????&^s?6ilyDezns0 zDAI38b1Z!uL7B=4D-{bKf>!CsT@AN4gI*NWGX;Z0{&oJyW7+LpB*PilawfBM8 z_+hZxmnyFPJ&KN3&2SfE4#UhD(+8~sj?UU>~<-;AJC$EzAPKN$rjQ-ABe z_S=-8b+2Y?6^z28rgp78-;U8&dPqh*>l+2QzOEk#zQD)21@WF@5Z6fxiwEa^cwwX$f<|)U1gNW63YSdA(u=c9Ll)*wz=c?~GllxT6oG~1yR!Qy;CS>A z$eW6#M{m9OJ!#UEWm=jh&`qzf(&6h(d zuUdI;x?G!tb-QUB=ug}zP9Z%tvT2i0ZPH%!naiCr{?68FM%^SFkMZ5Mx+0kF6~MPq zjc*E~&zxSXLyxBS2MFg+1#?DD0A`+b_-M4N)P zx~?l(3$N2JYE^T!Bu#XZon0O_IR(2+=a_v- zyiQk17?Lnj`USTo&iE+54xOV%#;5XN~sFD zcjx)NUvOh_w3I@0EWPjGRkaNTzaZ$T=*xi{(RAgV4&C#|e?ju=d;WttF_f^3BRWeK zO~cx&LU>}r4GML=>#kGg(-4?Yq1I3mL6KlYKb<-|4FW583&q~X&?`b>eX<@*!@?Vh z1KT!SqV&04TasNe4Xvm5-#*#lNMYaqb=ey!G2OMl&m;Ve|NHy^-N)IN_ixlZ7})Av zVFh?RcRBx&C*ZLqn~PM@_N!Aaq1s&jsEt>rpW&^=>z>YUGtzk{o(<)6d@{Go}Lz>j;vfeGkosw zttnu;)dLs%i(1%}gZNefbYea_O>!&x^|3HyyZ`>noSt5heMs(3)vQm`WDLE)&yCJ; zLBAOHO!n_@-{^Cj-kiVv8&KJG)kEg+cNxN&Z13SzHvX;)He>(&y)W#y);mqQlX1!Z z(=+iu4gDaEhWs%RG7+*aT3)}28~W|DH@qW+euHfWMzdvH6FErqL6yP!z8bdozwKd1 z!uO{9@gEdSvssWXpA1sUL;jPiKTxoD&y~JACPT#1g+~rfc2h99IUA?%B*S~({g#^D z4HPU!^R#?ZGK`Cy37V8@rC`z`Z;TXDz&dz#T((0e1yjrwbl#f+LU(VJdZyM>uwqyD z(7P$H_uHuz9di+Ho;#V|lmZ%IQ**b>L!41}r%@pl1hTT^`4uZDm}9xi%zdd4bnv;w z?2jltuQ?7n_fny)J>wdEehme4cuU*+BNc=+W^AyFNAK-LDhoptUx2NZv@FcZrC_>O ztnhs=!0X((fxGaHg01Kj>$&#=&M%4Lujxne&5;t4ZGHhN8V1h|pMIub50p1-UGx$j z&t>|UD7~Oyd-QTI?tcjea(P$9^4cia3Fqq>i7#RQySOlD8v3v@azwMdC;xw2iVN403Sd8*(B~jq)@)n#d1-jrmvM($GYpr%Y zr$br5{8HX%)L)ca@4f4x!(n$h?Kihj{%1GmscB?FnbgCPJ3Yw0EF|+Ij%5N*MzcR2 z*hs;iy?Vlzk_neMUx%Nq=|J|!hbrAv*l5HI(4tC*8Koj z0LW!#qJ4;zTOYGde*ohr4M!I?^&c51yXa@zWpS z{sN7}(A!xQEIvQ~IW-3oB!pTurBV8g%ZK0lJIa)Q2cT!>+7>~Ajag@!^?wc zd@8E@!oi&jV`Vu4X-!2GjCnXgWqmG~N&I^GBmCG*qV|2dLtx`4kmEBq(^)l1!G0~nEv|fmq1Dpu zJMN+KY_x5R&;JD0di(WC`}&b?@oiWr4|=w{Jgl)ttW5Vf4*}@ z^*0c9K5ei7j;ed6)<`0K(Q@8>^9v!ye^fm4ET)fX%RU|5R0yJD>LPIHu+J|mNA_~pZq9S*Vvuq8F7RwWpFVaA zH?p@c2C-%1b7vnQ(o^72x?2plR%~;z)JFcd``^V{iec-kGt$p{(kYm1QDE=l5@@WI zNC|zjz4TIeu;|*Z!BzbFSfl9k{^?RU zvQ?zSC>Y(p3sA|^#0l0su|xBn6%ViNyz~un69SI#s3QM?1*6M9e}iDvC7w=; z(fupMxJ_wR8RX?07j!<&t&i#b$XULv49sC^6CdoO8Kz~0SYP9HU%rH@&U6`b&>0A16CX;&1{c*OnS z#yNTgtZn6TemITnyCD5VB1a|kUAwKub*~!T@2(|v8dbulTS1Q-&Z6)?0NIDX&KDtjOANhcWLpLFm|cdKBP_Du=qdNkhzA0vxc`9rUt)5^q|Ggqd!pnyLR5srPP9^nxM(B zXAK2A5Gjy(p%!>dp1!tyfbx6sqgi!+Ep&aZ@?3ZsmH&l;jG|;6Tygb26HtNr+f5bO zaN9a~;2Y6$=mWBMT=|#I{W>^pdx$x276R?9gwxYelX#8_L$2&d09{3_& zDONp3`s!yzq11XXTX!}6y2^ZgEN-e-e}6sbStQ%8+l|Jfro1dHupW$DH+)%RgXRwl zz=jvlBTiYyJ>Y=mu#aTvQADZ88!lN}Bpf5Dds!$r;AI0}IY&qKi z>2J&SmTIE%9GHqb^05J&%wGC)s7UBzi7#A+MH?Y_q2Q<9t;pV-ch&dXHp0Su#ShU= zBKnxevKRXvH3DN)`psHDl)uQupUg%Y!QGdx`QtmmFD`qtt_gU=WHaU}N$X>2mt^mR zHi6TxwVE?*QGV7t@rpGx!H)a-5(i5VH@|s)rp6Dj;alC_*Np04TGsZq_YYVYcsI#q z3bo(7nB%+oJ*m)z>ccZ&4fRkf3|*Z6wy+lU_nj^#NsLzD={suPK}C9U zmCaAdHfZ!tDZshV_`}C>VWw*vxPK`OhZm?m7KI*}o6`n6*Kbx+-h=WZvd(foIxk0W zXLg>e{ybDaLmJXv?Z8)PRy+9s?E{;7^-{H>9bC?t@D-m$@q3T%H(&7+mW-dW@#NyA zo}#YEo+G0rL z2(?ev5AQ?!r?1Y)-uVXYpI$7oAYB0Ac9SonHfSHU#N3zr?xTIw6=$EN^r3y!Ui^`x zdI+U@x2^I;`>AWE-kium`>8p@5+4X649*xTHAVZXf#c#tAlg^$9HTxz7wxMq-}z-Z z7s6|?tJhP|{_4B~+ssa&{nhU`c3V6|`>R9GG;gX#`>VNeQR+;DXHT0hS&jBtzq@-_ z_z>D>-BV!H9*Xu^t1V7@`xfo9zT+Btu@UXFrcOOG;zJnsq4Adr+HYNA&h>0#K@Y6O zo3}Zk{np;OdwMUT{nnCi4gKQLerqSM{WH_ierx~Bkyi@Pe(RCGm)H-q-};urr6a>= zzxCePKR)v!+|e8@Cz;a&x~u%0<WCqKJ9_NM_Rs0k5PVx$5s_S>;c2M z{mCEAaOyJ0r>KM;c;dVHuwq;fw8k&bx)+V&JwgAxAhHK$I<{X53hM#u zxR(P1H+z78k*LtNP{fmd7A0Nlfx6p^sG~v1ojcCTAg~9t9xWB|y3_-OHPJ397mz>N z>m#*3J;2Mcil56H*{?|rRmQ6aveM1Pm!Ix|papQw;8YLXj@{t3<^;0$_q@WCV<z+faGo+(K>h9%!tnwl3O)>_Q}y^MFYYxQlF)>0giR5zk3-H|zn; zmz20(gC2-k{CU!GHEQ23KRjQj2QJ$l{h_Aa1H$IPhx|2Aea!r}p=@anMCh&Xl~(Bi zlgb;SyBDGMUHWz4x?B(ZOn$Qnm+pZr!ZP=o=JtRh|NA4;l085>>Tp_2tOryj9)4ON z+yl|Cz-FOf4}2N;s3^sY`W2Kc=Hu*v?$5mr-BaB#y6W_cHzVD!N>XUm)}C%y6xzBq zx1}45p5JYnSBqBm)2UxfzIDUlIkJ70=zO*pTwEHIcikYP#eaA@y&JNQC7q8=?gpu) z>6_+0=mu)5UFy-8ZqP}X8=P>X8{(8-q(2Mjh8xz#5@Fum(7KM|xzoKH>~~X{qldde z#n$+vn|(LT*mP;{2di#)w$c6Az{YM^Ym@zc0=l6C+o4md-3{|wEMp^8y1~>?pg;qi z2dA`dASPa{8=S4)*0%F@L)v|f#^2KgV|v^dY8tzMS{RK7lyyN|_a@(2 zd0n8;w?EP)vkPR@jK1AX?t+1JmeVPByWs2g@uslbU7#wIu-Pi03-ZrA;`w>D3ugUr zF12#)0znHkpK$vwFqoWMoU*kGtZ%IgN?6wgN4H0xIJmM4tom9vaxLzHIR<#i$+=yi zm4D&uTfr{y(X<;cpXdZdRV~R+o#=i+3#-2Ry%V;)E$3JH+6j@Njn-j|PHd6gJgEbrP0l9cE&Kif|5s*7)Xy$OX|dE@ol z)t#_X^+V@%)lSIt`nF;fx=-bCCF-T{b;1UJ(>r5B9dJTtrkL1|4(R)AqBdUC0ltc7 zZl&NI(6ws!)wa7GaJ9_9OZ|KY@HDDjf3&>=LiX%FVWZIjdRvd()#2{|mjuT%YwCW2 z@wm3~iB~_tr1B~!?SkxSFN9rYxeM_JI_xjbIHn3FK>t0zczmy ze9#Voms(x3J=($4^r3R7Q9JkyIM3T7f$ryz6Xj(a+Mq8iDW?dXd*@$uZl$wF8%Rq} z>$OqQId?icbqxgCK=ERx>;ZJn-F{^i=k2#!;mkU!u)wZXxcOl8u-n2`C^N@4`q#IB zQF>A?C7}iKEtEF~*tdXZW$Avqg)N{eyQUqqiNu)eN2<^TV%iX@(;|3Kge$ znnBQS+p?s$KOpn&X-}`CKfpY8rR+YrA0WE(^l_7-CU`w@=drkV6I`1rNqwN&1cH`T zIjtp)&{s1c_X3@7R3TG%ws~PA#5~iuY|U(d%evAgm3tcCY1{KPtH8@ zZ8X+NQK|=qvM7(cYaOJd_^L=;ss)$f`%$NFe}{3d^eB2%4Ll0=D#^W74NK)2trmV& z(6KCBINPxj%&nJC$fD;MKE}3y?(Q-WI@Z|LcIz8h`#5AglP`sgS(eTVt`&p1TgTxA z#f8BCYetbx#upGhCenLv=_iN~bCxd)cn5oJUIYw9zDDmmC6+7QzfNI)Px=_i6X@jk zrZd@k0;vm7+3(MPuV1j(?{$ygh~CQowOT=XN2koNt`GQf*7X8UbE$sqQhgt9zjH-B zqwO}3?vUfQ0R6-P=^xBG-*4%Be;r8wpm-deGXL-rL`D!-5U|%C$n;Rc>;!**-sAWB zf(XW5r}3wvkAr7wIr7gW{h>94e_p*hZ1&&!z<=ul|E&-Fw?6RS`oMqd1OKfL{I@>v z-}=CR>jVF-5B#@2@Zb8tf9nJPtq=USKJefAz<=ul|E&-Fw?6RS`oMqd1OI=m50G!m zY5#a@{`1H!U;y z@dyr?N^napc!o3P1?)?ZNd#TZZ&uP$5Ai#97Hp6+4ujAAmleJ$Cg6#0>NUJX4nU@9 z%2sPvf4swz$iUy}pEr>RA=007FpQP{tPO^DSEKZWQ}FKhD1H9mS!dg$p20`YQNOat z7dQte_wTI#1AB2c`MpjeOJ(!lSV5LIodYH`)Gde@xVBF98Nm8YLQWM zieS-^hS;1aL(CiPG8sbu|Ni~W&mni>_W_B7-{1aQkQ=)l{%`h@ka_supcgb~ zPvA>ms?hICr4!|;pQCI*oJ)YNUYKx_;NX3F1D|oh`-u6rA3tlEIuxMZt z>wB(b5Az-ROV{d|j5zI^2kd>0tbItb+TX8`>DR*!55=4L%d=pOl;jZ_o%Ow8vJD@6 z|I#l7wM4!+@dWNB2&){3Vts#Ey~gk=GX8ql5*^~bbYl#jl1b}J)n$Ee8ZD*%)L-G< zcdp;>j=m4WC|IlDNC@lu*JN9^pZqJl+US#0n(_CbO6{ZdnmE??xYeiFUn2EF_;%1n zSVsB@p1gacAk2~VeE_mjj_!Zy=F4nm?N5lpl@=A<4Mv~CL6t@wK6jZ+pGok4T)*x4 zZznQ5{7ZP8OdtLweVoic{xA8*$@0U$lpiPS5B{b8aI*gKf2n_*Y(M;8+7Bn&AODy3 z$4UEvf3Y8&v_JS4`@>25g@3VMoV0)V7yHM_{saHge{iz@!N2rBob12wFZ~z)yQTl9 z|9bxnPxvSLKWSgRf2L39MF0Ow|0Dl|PW1o3^gqhaiXZj=zw|%qkI;$!Pe+KZ-aqS~ zrKA4;m;Oilv2@h`|I+_xf0mB=Kbe00pZ3GjQUCu-|HJ+Wo#_8${Plm@FQF6t|F7_W z*gq?N)c^kq|408}#gFqyG{*(f`TxnFRmG_4~f|x6>cv$DiT< z9Dn{yAO0A>2%Q+ee&-MV82?z||55%w$4^54XZ^t+<1Z`zf7Jh<<2Nh*f3zR`G5)jk zf3!dRasOcH|FEAw?>{X4ANB`-+`m}*KkOI&xc?D4asMORrx*UXe-ipX`_G^EUqb(9 z|NHa)&5HjY{rC6hLCY@dJatYy)^ zKM%uJHOThWp#1V(vfo&8JM}lBy4?Nt83Jb5l4le497~1ze%9&q?9H?3U>njC4j)*yo~fV8QS1 zuUm68kitF(g%i8^@L|)`)!XE`C!AP+t4#P3V;XtR2`8rcSiq%W#P@%(_a<;XZEyeg zPDK$ilMsatMUMGM>#NCBgpj#WMbl{@P6(Nf%pr3eLkGuv%%^YWnGPNE5aMJG4Q9{h zT5Ii8bpF5l|9kG&eLv6Zw|iZ^*Sgl)YhQa!d#%0qcWqm-u1P6nXIZ20No{A^igip% zDZ4jqbB%JHEY~g3Rl2>oG^G8lIdYv6UFExRHP6O3Uu}DuuS=q<%p5v1;=79*Z1ea! zB)UqlOOp*yGFu`tw&bbl#h6`yF3rL{F)_`LO3R9|v2pu85v8MtAk~yJth> zIwE??wyk3ioS!>Nt{bAK7&ft7c6M@-?RvgWh@MjB?~RAH8LqRP$kzqYS7u-NSX00C z0$Z^Th`wUzZhG&o&s@3ghrZG})wk)x?-S%YANoq!`0OQdm6L4a__`jYmC`pOlxf*Z zY%TaY9;KBZ`zDolowdaFEMK>yv@*5x`KkMwuCr~%*Xby&Y$_jWHs9%7@l2?3topRbwlZH= z!$7&CbFymHdJAoz@^v%}lwFUr4>jL2ORk$?pg0bDJz#R)Pqt#63-g++vOGJHAduIc1Cf z!tKo}t+sXM>r#|c&fOYT_R8l-Td@vBIc2BOR+}v$5w>F82_t3m0K-4$WW>pJCXAGT zTJwXfM*7QjC5)7vOYXEW3|nCP6JJN7yt1K8$MsWwiL@2#MwC}J&p-5++rZhjVx5Tc z_!?_B(})9!wnO>45apFi(|_>P+Z|~u)`6&?H0o+Nzj5e1+iQH?hYE^;*~U%>kIc3e z>pWCYo=^F+#jn0U%XJ+pDl5*-bL}u=w5?djp`vW}YXA9vp3jT^weWnA`X(FO)4u)n zNQQfP^ACX!^AV0=0b{~K@fq`BV{9NTAR)@IpiqBr9C=uXUv#9`@JM{7zF-sSSCsS8 zib_R(mt&5-N?da&%7g;86YD|P_@7wN|8+B5N5m%5`SM@Y3##Xk)i$O@h8de zvZ*wXewFj(cez!{?{ll)=~m?TxeI8|=Y{z6kK|SP9d4E3(yy|D^sC$~zQe7kl;7u8 zRLbvjD=HJkceoXm_R_CXexF-Wxm13STc!GaZk6xk_qJ84-{)58EyGo+-{)2-zt63x zl;7u8R65J=ajPsP{VEUhdA*{joFx4!?WJF3Y3WyaT7G9+rTTqtmHlM6%DU38@}c~` zw#seNuX3#Pt8|urmFoAoRm$&kD=K%$?`^A`B>gITOTS9>``jw!_qi37>i4--u9V-| zRv9V%D!WO)N;BzK`9^*}TV>3@{V3#fS_cXe+aaEs{-nQ0ZPPi<9-O?B zex)~#@7Alj;kFa(m(mX}#d_m+fEvA|rEaI#@z$<`{OhugN}6%M8F~0!eN(;tPX@I` zdP~pb&pOLO>dwx!e-3-hMvk4<70 z(wL)B;KKwP*qeA}i|Oz`*~}4Tj1*IA{0{QP`-$IOWJ^B$I&SRN8c6T%!F!J`vcq*| zRW`E0`FAeTs*&0b!9&AV+_ z&pLb6(K=~=u}eeW#c$Fy?x?(Rv8XxaGOP7+=9}i>mK_y?n@UaFD=c-Cd;Sfxmau0n zoBjyDn||fs|8^_R*Jj4gmlChC{>|%_8J>sy@zk}{>voMfj*Q=S@Nd{3pv^IVd5zW6 zysBn41M5Y{-q`eE$#qugmubV6XH~-Q-mAtecD%uEV6B%{BTybS8?QX~_y+5>Y}$+z=yJGZtE7-qu`N_?9SkQ?I#^vp*qr6V7o76Ca z#otZr5oFb%qjFO-xI#F7Prx98?)Sj& zEDxq!-DY!-{WQEwNa+vQ-}Y49m80*mG0R#2)8$FJH{UDJB|=$o&`zS0_WYNibuJ5Zco@xyX| zX4z!j7*;#KmvuXR^bgAI-7EUfWcoJmKN*fh{>Lu5y(KJx)ww&*DEdQtgxk$-c6})` z2@An@=h1#s2jpM1M1e;fOU5+Fq1z} z-<1=k=Gtv!Q!k~R?0>O7`hz3ZEx-Sb)pNY|=Ehj;pL98_lI1ow%5AIV%-8UHzIt-MY~uyH*z0xP=Qba04tpOBjnLoCR*hS2 ze&GoEr=)JPMvvUX#)c$S&eFy2zEd-Ml)Jo_O>*gR_JM-)(O5Lt)p9>;-QT}e*h}gk zHn{Izcz~VgG0I|S1^n(f<58(bj}NjlXPQm=?oX@>lN5AlmwgI*{1Myr zXKC4n(RTWSn9qrb?=F}2A*%sTm)ls}P+`tCNSVpzfv<{NG@hCM=k zOWM^Z^1Gv~RriC(49`=3e0pbb`Y7ua@X+4D4C8Yf!^%|Q z=X#G$F*A!-K^0n}erR-Ot_?ZOCJbu2Z}SuM-`Ty7?NQFKh>gB3fjKx{*xQ(g)6cN* zr053bpOIghX8Z4#KFebM=zFf(8Opz)^cC~Yvhd9h6P$NrorSqoyxvzj$I30*GVew! z^k>E&Hw{>Ijzt7rIcWY9?DHJoyJPk9EOSA#SAi>VJxD5b(`xB?Htpraxw%geJ~v^M zN4*P7nOo!Wldb5F!fw=yU2=g1EP9ncE&}~UnxWI6vT1BW!#?JNMk2qn-nWhIlE#i# z-nVbT2-L^q`u$o=OJiGJYsX#pf_-~O)cNsv8pAr6v${RP`6gdHckOc;i(T#Ec;YG$&na!2K=L+X!-7ulG$ zD>nzWM}Hq%Zq%W_E;6k_#gyBRe?WaWuwkdsCHDQ*-L;m?M|=KO|5d2}<6S~8*&j+~ zXI_UHlO=WDT@T)XyBK#^-Gtcw*Dm_;T zze;7G@T;6#TliJlTt3J5SDCt4_*J?M5Pp>zCc>}M^Zr@Bzsl^j!mlzcSol>c&4piO zZ2lR(ze;>kocjw%neol-Kkpy>=luiz{=$FWKlt|hE5)CGQ@bks{>y*fKM?O9sqar{ zkw1l>pB8`rg2D?we=VMW6#k$0557FV|IhmeMc@D8_2)nD9~8X*^Z)tx576F=Z10S% zrsw!BVnvjNJRJ`|?0dh_B>Db?7^fEsRx-v_tI#nxn2Gl+4t=N{fvTT?`kI8IdLAdkm5ZlL*%L@FJq6S^}`E3eYR?hsUN7Ozxz3rJs z234lBDP2E|I@RPd+i!4S^puz=RzJ#Ocinc|nNf`f0~Yp(V%A-pr%j%4nAr@{uQczz zFEjHQZfE%L7k2;pqw;rjd|2tzb?R+N-pbZk4~jp~W*=LZKfQ)((AiHWc_vIh)LQp}T^;S!Z?m+UV2xuE?;Rerp!9 zcOKt6^ty1B8D5+7`~B&2*kjYtI_}#Kv)Rr2z3x9}Iy2v3nOd>!36}kKkC{tOEPE8- zP=Di{-OQ)v49nfS#<1)I)-`6I+08s!^v%ePI>oAHu>qHIn-$pWcI>vUuKn5qJ5+n^ z`cz$TZ1#ZyeOjB1hB=pZ7x+Im-gVQh{dRU>+t{l^TmHq?WtBZYY3C-^s72K=o%Y{i z>6T50z3jb?1)IDYu&G)aTR84;gSbaYY~%4&cZ2$;F}Lrm<|N_^8CI`$rg|qFV|&+5 zu4%eEfoY~rW_v3fVK;pSw>B7YiFI6ZzK&~H@Ihf$HuKpuspo?%^{1EmK3 zI5>rwS+$wE$KX19*4nYV{iS_uo~CQ}KPp{gDIUXrKau%68})I`g88YJu)ba6>&8oV zu%>r(Tc*tTQ@&qyo%vUd-`#v>SV8`_<;(43z2jn=&I8@E6<>aNkHsl#tz(DXw+iB| z`YoXC+VC6fv}sz$eIFmNN|trj?+Cxf%Khfl?=99x+O)Gu+Ar}}Sk<@XwDVtPGMm5q zxO(D#zcP0wYq=zo88qK}ugUl6taWmm0e7+P&@$8G+u!fJ$n>XnezDT;E_>R_pw=Ff zyX`Oe@w~>e4>jM9pdV!ytEq=}Z z@M-mE+Oy{@$+$w99l5XB+PPP9jE(Rep4mN;Ox$zWrL!a2;TuHk*6=0`>i?d@#*hBB zK@EdfETmgf?>RMcSqo+3t+w@Fv6%GEjyf^9tj>hxbt^4+#pb@<(`2vL1NL&U+W2?D)a3PFL|9p|Gvw9_YI1gvC$;jjA8A!R}GVMdb?o17B@OxYxF#=8Ttj_qeyY ztn$?r1GjCgq^YSKiCmZaj@{3)nlSTI70o;M*`HcPd}On{%KICwuCDP2+vi$2L($xF zykjwJc`eQPka2+_imqnT%drk+{54d)NoYf2B=rax#~HI3+hJ8MM$TU)04Re!4n&Lh+#-{|pPvx0J{G^1hK>-}Z5k5*-x#}C}3ZPuWVb+z(&cD)j+*WVGo znjLg)c;NR-cT;9}czffm33I# zSDP7q?M}Vj33eW1M_nmh(*ehM+$8J3DXrII^U;CkKm4KE`+3pmJN6I7eX_##o-gxx zU$m)tfxVM{?6oc$k7OziAsI+B8%1-~mc0Z-{)Y{q2 zSsL}nRJ*l%Qr2#-+mR`$7rr~+^_bRUK{aj5pV}AL8}z*1wCv$u+uLtl;y91?CI$BD z+?>ARuwI!0dz&_ijOeESEM=NgoJW-=%e7;E4(QwHmn(LcU$%UngX_Tb#GA?OXJ}LQ z{+_&8D(#+?W|K^ z?@6^tm!lRrP;+)jPP8xJ&{##oO`N>T`UfJl&wnqHq6VqjKiphWm59X&b?ZGUM)mwfDv3u7%>p}As-^1QE zmnU@Ir#*P_@Zq;Zs}$I4zWw8?;lHcnsW0`{_l(KWS&sSz_J(%ccH`pM_jYPMvR-7r za^04{QeM0n(=gF|ruJIhpDjC8J!yArR;_J6EM3g{+`H5KQp~!P(3ZpdZ;bKM9&t4E zyxe1<-J6XWYn_jKv*kPHjQr((WQw7l%gx?zTWbC8e_r-f$H&fb<*gE9OcR4AleNO8RmL9hzzrfz5hbo2JnI28C z`zvH&&i%34@Cs4SYtKDmm+O!|uS)(R7F2h58^e$VDH9ijx@9%!rDf^CXYgSf0@+VTG6nkgB>zP48lSD8{Fzv-M2c?I^WdX*CzIbPPcJZvZ47hY6QZ`5$_c^OA{ z1kbYbucmWwL_e>u!Ur!LQ?JMM{&wYFv|2FwkEUORtLcdT{cC$={ubC*LjKyx{ADHN zubs?aRzm*T$^2y{MxV^ zx1c_hRDWgvP+)Hf{ezwCA6NDhCG`)o|1Ge$g#On~ z_P?xz{?|_SzpRA**G~4otc3oT$^N&%-je!Xd3`Iew}k7PoxHxW60UD{^7_U~xW3uR z>l-WK`o`q-t-#)ru5WVuP+)Hf;|C_k53GdogPj~duoA`(c5?i{N*F&dIesXxx1{ld z9G?}~Tf+E^$?+L0VSHvM$7igB@tK_*pRp3gXH1UI{!sg~lE!Cp{F|!AD`EVrhNqS= z{!NwR-_#Puzo~Nkn_9y7SF0XhQd7)d;Fa@>R8>*J{32D(FSI4hFH+_FLR-T8B2~^W zQcIX$XyyDu&NoY*Ue!TFUif78nOo3@1cn^w->v?a{nv~vEYEn)tqmGifP`DRJ; zH#t8pu(yQyu~yEHwI$4twQ_!}En$AFmGfh53G-vEoFB{l&1NOdkLCS`0((oi|Dcuk zAG9Uhf6&VN584v$KWOFs2W<)WAGGrRLqYzQbpJu#-zl)Sg!?;Md4ES+!u=hsyuYI@ z;r@)5`m2+7j-cY32PhZ3*|! zwDSI0LH?F>|4iPWEU>qP`;%IEe^Oh*{YkC7KdCL@{-jpkpVXFce^M*&PZro)()~&G z_iZXgzZNZjPk`TbX&m|Q+Z07o`efU*J>wOUYQlFTax${v@?NOpx)EW$gyBEA$)zZLqC#9lL)@eM`uq912ShoC2ln9!bn^;v_Vaf23hxva?&>wBTU1E5 zsGv?!{(bzydiq562p#3<=R@hb55$_(N*~2Z>8-R>zE_$lj*1KZXmj<8i170Fvk#5# zuXOf{?Bo*}ILfboV2E$%X!5&wMM#}nXjl}Cfv#_aUwBXN5q>_AT|0`f^m7gj@}pz>DbWzxnkhawPo5=kd*gh0ws3tY&jB7y zDIFU-HFjdGq8RH=jh#9vwyM6qlatewDO2?EcBDyF z9ebKt#!ipLUW#XcXFx!JfkBz*u&~i#Vfe@=;u|Ps!2fK>0kg0EEEV#(miT@>Sg z{EJh*{wuo2MCHq$j`FYl6;5H2^5w6T@~{1=`)#@>fddU;9({|N4CXs~UrUtpDn9O6I?K!ZMn%QZ)X*cD7$)87p7h!lk1_ zjUaa7q{(qprcRqa*}HH5frEdfXj2azK63Qf@e?Ocoj!B+-1!S>e_p(l{@3LzSFc^a zar4&gI~jNH-OqgR@b9cgkDok!_WVWm%U7>+a`W=vynXln!^cmb6`fMLditfy7#Nl< zXH>pIMdM19t5mI4y++MiCbdoL)U9V$zk#_$!$yrQziZOe%DP!|n-<@sL|15#ts@hWau!@lEZ(B88?2y|MdC) zPapr^&i_vK4vtRFox61H*4@RmM^Cq2z5BTL?bqL9z(A4zC5Qj7$p3hs&KM*0z*q?{ z9u;yQE#%IU?oQNTd^;nyi$d!U7{g8b(qPj9?tw}d{3(i+ba#cj72FQc_JP~6P@I55 z`niST?~;0V#J>r*daO5at9F|D2{+l(O1f!e=Puo3PmpwzopYs|>`9jHZg8KJZVVNb z4Cy92jfRW#$p(#dcZS?yL3Cj9V6Y8S4q-M_2H;=yTY9* z-DI2I2$2q*m!))5o^+9JDwiPXrgNMw-IV{^q*rrgUryxoIBar-Unk7^8bBAqW+7lfpVP#0XLRl_125Py)Ug$NHObq7Xk4 zseAE#)%3j)Hwftk$|H&+dtrv4YVZpp9EXlX3@l&>_Kbk2dev`@SfSj;k&4ov#<-hQj)99Z#GUSsV$WW79dJt5Bk#p%kq#(V3)RK{}az7AJQv-p0br_%Q3)`=1+9w$(qb8-C$Zm}2krc}N@ zW(cgIt@wTs$REm;0L2XZ4d=%fDSRE5@--0W8i<^v8eyi4fkYw&nm77`U+)v8jQr+t zMb3RIegwClEFoLJU0Mpa^h>zN(ZaQ~=>9=CN1E{l!%{lma9CSBjR<8l&MbnLsDf~6 zWx=o|1hvHr`-s+}wt}MJRBOr7NNzL5^y4{IJe_abQ8cZhc2IeXmi=`rruzM*^;*Ni zFJ};f^9+aI4`-s<{I!niYA8yG&XZb*g0n5YpEY!2q&|X|rf7MQ$P<5vX!Vp5^(A9* zG|CONKZ!s)CAWGGRKw|fLts1AC912%e=4RH?N3J-;?RNuulrntRSdbBScLD=+l z8lse`*CZ>y9sXtiF(t$y8=^`GqnZ|u+z2fwHR^AQ+e-N$>ZKQI?bqR>kP2OO#v&G# zgUC~ACv?4_E2uTU{)F(b5NIilFYQY0&(!{lNNKnPb>{c?I>9{{?#>wD&=?^IZW?dW z_#&FS2jaS~C~<}2BouOQgPX>fG_Fa7yNlADj}g+jejvYN__UBa2W~ZegWxZ2Q@Gt= z0gYu^!A)b3?r=N8JrE;38jpCw-Cd5?q6_I03h6h)?T$Yhf6e#qrW#q6szsQw2PDl2uY zVU3MI6M)XJ+PeW(@mF`O&v_?3!3Q-5C$Dx)i zu66%(O)VTo7K$^YI;LKS>1tE-I$Suc@IGH(j~n4Uie8VSa7L8pRKF=t=nVbgR`2Z= z4Hvy4#ipL56Ql;G1L~53Iv{9dWMWjtxQ(8Xfwi8lo{_P!K^+4fgh$78#(h8B!vJ-` z>7X7s7t{xnz|vqcSO(k%8i0F2LogLA3!VhafoY%-coi%UW`GsIEU+S&4bsZMZ@@~R zGOn{y8885=fW}}|uqIdyGy|)HmS9cL2CN0P15H2;SR3pDnu2a%9nb@;3wna}zyQz; z3<89c&2B1sj2hU}G=|{0>Y8n}FNErr=)C3QPs9!INNfFb!-2 zUIpockr`k+FbixCW`pzs&l?aANEBs!XGH@TfK-OYAkCR-g4C~=fzHgnGWip0>p!*z(i0N zTnp-fnd_VteQ+E6rNM(>8So@%0H%Y6pypR+r5u<6zY+KpEDyc`D}Z_vP#-{J&=@oY zD}kvUlnWxRs%I)b zGz0g7^}(ZH127FV2XBHFU>4XA%mEvLN-XLFXas%-)&!e?7GP7*2DAcgL2J+*Yz}&Y z--AJ5D=->t4aR|Oz`0;sa5>lxOa|M7yTBj7RImeh7PJGef(*2FbVYw9Y|GB4?K%Oc3w8hxf_C6Z&;d*bM}kj59duM5Kt0f4GU_o{6*L6Rz_Oqf zXau$cO+ZJm12`BQ2?l^V=+L4;JunU|1I`5v!R26CFc~xgcY!8gD%b%$3+kZ5y9w%n zSzsA32Q;LPFb?~JMxYT`6Ep!Wzz(1-sDqBt1vCUbz_Oq(Xat6VCSVNM0ZagO&{3`h z4Z&?-S@0le0-gjrfEl2U8R`{i2z~&|f(BC%4ps$qEMNy{2-<*UL0bw3yHL0#?4fYb zm%_m?(p$kE(u31U4}gFZ zfjS!G7x_U8q9g1ic7dHl7uZR3LpufaV$e=O6EF;nYd(zuw!!ZN z(luWXJP1FX=}B-JNMitfFdcqmAs-9S*dPP`HQ-Zl9{2{NYm?se&dN^E7~BNXHHNNj zrtr@MX$(;Yw1S_mVKkO70NcU;3+M=*1Kq&Q;9&3+7y!-(BfIJ9NY`PA4p>h8hadt zKM16;NFy)}{s1D*r!06Aeix9&8qJ9CM}RaYD+lJlp8zU9cUIN`M&K5(Cg=)UfV)8( z@E}NI1{$l_!atq#@Y9&43;Z-@pfR2i=ng-PJ!ou0V<1oXBS8)P=z5eO__u=5AdNj} zOjsU_gTE(8V<}5;F8m|FX!z@a%i;F|Y3x`5Ooks#mX9GTg1g{f2&RI)Kn435183o< z`CJD4*5Fn6!$Af1w*WHXj|RKI{{xr}|0K`=@hgEeMq=Ox=x2aNGigqSaOl4WYr;PU zOhWvIpauN?U?Th$pbh-PL0fP(NMpjLU>Epnu8^o^;wu9e;2!9RG5&i)njYVxi zH~6Q3gTW1805}Sa1eb$duwN}O7XCrtbod*9@$mbCiC_wt3?2dZg4@88;9l@5cm~V@ zPk?UN-voRE|6q{%_WPjTtj@||&=mXwYz3|dUBD%v2llHC`og~y38bZYKFV;q zZ$%s3%MqlrC%=eC&k|^(88vMIGJb%(&qVipXrnnYZFC=0?`&G2jJ$2gXxdd$y1?MB|qC17O1PbjA6JyJX}SgZVJ+_~ zQN6In2&yyUP~EWRPYy+Wp!zY8?@M||eto8TL2f7Lscu+f&)#rTeXvH&>&o49kB6R= zih5xUtqXTYLEnYvevpr^VD;M~Tf+jXGgQyXP4$ZEI=Lx* zs_&F<#m|N6zgl+Q++Z2^DoweGwC>1JQ1wE5EOM9L}RF5f-L>UD0Jng~ri27zq zmBvG&T~U8b^@eN`c2IvyDbux@`dX@Il#im#i1?z-2z?0OSCoxtC-hbT*-ZQPK}{}N zheR6??Y3||bKw1k=u1R-i*_Q)+n(oF;j%0oUp;>(J~LAL8z){D#C3=6zN_cliQ6a6 zxf5@@;+*LX2KAgBcwH`h4uy{=>=Wmp9@~xQl6q_xUQ*)NeYhPWzg^_9?zBABJ!;(KUGZbylS7$j_;8T1Cr% zo=~a1shU<#)LgYbcav_a&;7W)5wiCXC8zE)K%R>z6MMct?c*SiFIsHTeMCzXHrw-Z zQ1gP?ifCuu`2HdeJ$WI!DBVuH97G)PzLR?HG@cf5L`@cPy7O_ih~p;PtFVu*O(IUG zLgmqy*FB-PFBIRckiM5JOG-!d8sfP0bV;>AJnIsA@s5yi)6=NJ<2gDr(cX!opC1dV zwt3;4RJS+!c-8I8Mb~MCC-h^a&2;S$y_8R({z+)We|qw$ULVDE*_)>+#yMgXEUqu& zS(#{a^qz$^di*c@im?#ousRl^Cu!<6!k?cnjg_s@bB}~xv~}8Fb&F?iLQnryJw4z4 z;-*oa7@Jw6HHZjk~R=7nSQr$Fq74?rwNOjX22CCZ!`T50NSpOwHJug$^(9``dZpwF& zzCX`X;SRvnP1OhU`vO8w&*D_Q7(oktm^^oCSM=oZOB||aB953r2zL~(E5c1rv?&ga zOZ=pp#vc@_j@#%h5^~cxNz91E_({wHL?0|>9>OikOLbG*6K?viy6IgO)g8?9M2tnm zam5%&)Loi$S>xE0&tj~m>giffby^(`RRAM5^MdAZ9@{H==x@x+2Ca)Z$flFfU^UfmOSLlnzr#Y6I z59C(Qo#r1SWj;_k6hb|Km|v@Pg`UrUanpQI+*J_sDN){Ze}VK=u42w2%0tWt>AfPF z<4~M198sO`h>>#Pa}XNwzwkMTIp8Q>U&Y)_99N7hsV!3Pqh9aHPc`_9pX#}Y8^Dh* z+`+tV3wH!>Yr-ALui;`|D2{{O_xAm`ime!J-*9{PXB{bR+&!gRth)x~Ze^Bd+Vptg z|5oa^k=~-_^F6HJ0?)|lHTW%PsP>XRasOXhmU!~}ke|Ht?cm3jO8TtlmyfWib;{3l zJ^MzcbCmR5e%g0}mHVkxrK{m}Wqke}U$-lB&#(r04UF{{RF!(Zj@X;1K^NGZ8%-{k zKUqrZ(#I3+2miPYaE zeZqLZ43=#D>zi7`YfC-JH+wK7yIqzhyO&43K=ig{tMFM_p*;z%SP)fIhWIsdcK}dvXb$oMZQoi{(veauYa*7SIM^?RX@<@$C#b)T@s;WaYr zH#U^<`TA_T=7m3DdGXKM{nXu5#^?H{VYi>K_6zq1eQIwZeRr|UAZ2ypF?8@{iuKl9be8{Ev zR&P!}V=ZRYY8Pr^CGFvQ$6FT9nW4`g&G(m8>q8#J*OB{ZDW#K$oO0zb1m&T zGrfGRZhM2u(*AskA7*C$g2i|3G;3y5M;V{%)5=A>VAG?OYr878lJVd0_}f!oFhAen z+D4~jJBsCct16|lnReRn1KZYjlJVbC`~Yo}Y&Pm;nf+;bYW{P*g@2E1*4t_D-U?~m zW&C#(|7evTvsvSzcMbmR>Ll&qddKtgv)Qs$jl5PK?jqyAr}!?bc4o7gjk<@|>E$fr zbG_A^OWEv}dGoc6v()zcf#P3X_bQtW@i^yG>a%Q*v0Se_F@DJm($cq#J6}(x|B>Rq z@oV;yz35uoxBo(UUa?%CUZ?v@_Upyl70%C9+w~`k?@`PDB}@3dRg(oeYJ9HO8#d`B zn>6U$iK#l>Wcr^ee$CwFFIkJJyB^lHk>?f5^>gp;e#y=S_?W*M(pkn=G>C82B<&@; z)+;Nz?BDWyHC!K-p7oOXjhxtfOS2v_AGm(=1LYMve<|hBwh)cbr*eI^cgA*n_W(?Sp3?lant8Xdt$l%>HI;j*zA^7oi6?) z&o`Cp-Cu;gVhMeQcU{v*teUI}~UNQSN-@R+GShfod*S~4G^cAzaK5=czA8n=m zTyJ2pp zzGjo!oO=}DE&D?a*DIf_U$gTgt!f;uB+o0B>pc!&oyYAqxtk{COFL7!es8^=ui3II zv0XhbsP$Q&(oeM>_L^NiR3p87Sy^5huFw8E-V~if6Wp*zC1SX zlk8VhxjtybtkQkv3tX}kP2TazEXHNTx~j~cFz+qdyG^EqSQZ$U+S znLk`_^Lp=V*7kaGv&RE!3VkZqyFEM(`|AvPVi49y=#?^*{?&x^*DRn_yzBEfvOZ|I zezS7#HFLTgb}UFy?dSUCOaVNxIrU7*P60j~V1J z>m{uEfn(B64cEK1tCqtOEQ3;GuE~6h<@%(s`Z+A>#DhOI_hq|F<$6t+bq?E8sbA%% zZ)HEJ7*hIc+qKVOs~=2Af4Qr*I4=#?+n#aEVfnkO9FM)C)=#bv_|7ed8T9saJl#{a zi&Ub+4AGED7F4@{RiVMIc(ISo>qb0>h;Bl(m%OwPY$d7sgcXM z`3^FFxPEWtLpf~8pW|xYfA1pnv0R_(aXN>&gszz$zekQwQiVSH63Q{X_JyP+>iDEQ zrJodWGl#Wt?&?^ZNxg>a+jV=8!{V20O7V|S)8~5MkI!;grD};uV~pkaFqP}GX65BD zyH$-2*tC~+DitXGgb$x`m|oe0_qNB>`pNZ{-OA)L${xqlgobXUnTLWfgI1oa(%m-19Mr~ zlan_5xT%A%Gl}arr+MYF9(D4XPZ=cZe=65oZVt$0)n{~EYZ@%`EraWALPK-eye`IV zTi0_H=_r-R{w^k?a+%wnuFw1p>(wsp{CCB5H)Q%6u8(apIhQ>%Zxwgo z`>rCsC)X#;osr8zYtQ*BGgP+gSfS6DlgqO6Z&?2tC(}>j`Ye-0xhy^4mxP8!^17MI z^?I7+xvXyQP0>RqIEwT$xZcfeO)i^S{kRhGvs%8D$^KR@$*_OUu=-_FRQtK!xYd?i zwmEb159YIFIcm7x#b8%1`_X^)4~-0@ot|9Z<Xj;F|G_bT<+4p5re%KmRjofR0xs{LGl_NN!Q%=+!*OHD)7{O5X$2|2ke zI$^<^%6f9Vk-_yoiFOO}Vw8wd{I*wJU zk^PZBndh<3>#l6ovr_Y)>uv5g&SNv27Mcy2DBGik>#gjp^4R>Rx2Ljd@_OXS^-1wI zdF;T}z^*@5RIjI8fAoB-JoYfa>-B-()b`8uC-vIrvA452&5b`N>v<~IXEy7Y$Ih

-(_yX1!z zSC7d4O~dt1eO&X{x>pUW^{QM!m6pU24lHiPS}wt42UP7PhhOuTL?j;qul z`%Qy=^H}!#=j&E}Qtjt@W3zxfb|~yzy-PW=ztV8M&8d-jY~=F_rt`+hcIV0U%L795 z*oxm1_6~0=>uoI8$9@XWV~NY&lp6Oz>XW!$nK>$tHJy@MK5&s*f4E*#VQe0I;{DRT zaYtF-GPvGz`nWtc(BM?&xlFbHtV#Aye;b>}T5QRlk+?wW&A9%iS6m+JeQ$iVSywVe zd1<&l^3b$A)~koR;l`)3zIk$ebj_K0>~jB>dOnZT>nYb8`OnT{51Lx8{CQ2DNGFNw z9e!aQ9=r1hS>KdeWWQC< zWqIt3S#HXvkFwm&xZW{pWgdI6=v83I5E);?^~)Em&SMP|W-mC@QuZ63T<^AVZ5|ss z`=E1!elj0oxjuH^hCDW9+_IK7Z5nxA z8m_mE*@yb@_+-8OIBBOR*Cz}(n8#wqEw0o-tImtL-pWRs$JX19GJ1bcUPqF+-al-?;1PJhsVq z+NFeka$IM|_3od~=CS?1t+n~iMIE>8nvdyutpD$~Q!Xj|%wugsH$P~;N#>hkO7?%~btjK)=IT@Jl4T9D9`Sr zOedD>is7BZ4KAk{$`lZ z?5oTSsT-x%AFhwTRxY1?+>!1%)vBq;w^*+C)v1urmhO5`PXD-!pTza)7RLE3NN4-y zT4rVmp?=d+tm+X%xCw9w(9=p8JWI@>ys9m#2v))5uM)bWqduWOLc+&rCHZ)mFapE7-meAY3s zSGiti)c(^qw{S0jERW!3+^2n8maL2pIeys>Cvf&qO>sY+QShi zOLXS!#r5=-2E9+x7jIZtK;H%Of4gO*;+@?M^8ME*I*L-SloC6r)Hjp+gw+cv;H)J1 zhq!c7euU7bhNl+_C;bVJqTy-9!jJSX8eV+=9RvS4yvmS&4mThA&*9yM{d4#MuYV4Y z_x|Vb-+cZ#{G)Ht@Z$NS7gaR;WU>646jL<(XtD4!u|>nL77I6-R5ZMJ{+mxO8eX{k zseDVviEwhKA#O?az+y_#bc!FR&D4Jm|NiHH4lf`7&*8U=g*z6@hr9EN?q9qddMqd! zUOXS_FDe>dyd2UKiiW2aJHGX{qT$8MY4Dvdc^gp3T|8c5#+P^-3b|v79Zxs&pN|(- z)t-v9pgfD`$JrYH9G+I|pTlp|{^#&64T^>ruP;ioe-2M>Q8c``{pKx;h8Hi-dL2bL z<-sl6FYc$(t=9A6`R~}KXu5^%-agn~ISQfeTl?=DVy~Qq6b>)EKj~)?^#}*AQ-d}}JDjHtcezJQ~T+#4d#q9T= zW?ztZk%ipF&%@;BqUjVrk45n!oQ^pX?r)XHz0J@=Xj8+Bmq*scqT$8QXLYggZN<)K z=oPVlrqo9k3qSX&X!^y^v-{hk;l=a&Kv@TFKmB`V=umJT#qF$&urAxn6vXv}d)Gu= zC1L9q-R2!|$LcDYLh;T+w;A!iwT1j%M(MGyj4}f8CqZUI)okmklB!xkj;?8kaWmn zh_VaEgP1_9Aa;-*kN`+DWHuxTvJH{~xeUpIeoGc(eK-?jdAoC%+AZH<2kkY$x z9*{PWZjgQuUr0D41~M122C@Tk6mlMN9r6(J24b|QjA9D0fjC13Lq<9DEb`I>Lfd}iC2ek09@uIr4xwW0 z<<3!ozKRkWp6cTP8#XOkeAkfR`hh?@8=gDfmNWLqB|+R6z&@88x`c&#Vf>@l2TTXj@E|m8vA*)b8N-%UXk3z$pv6gmJxStx1kJZ~x9Bs2+@5rFOA)}G)N`0i{6YA^d z78*z?DxVZAA704cOUEfXP+j0#8U7_#MJYRB0db|a(kYxT^xY{eFrrhBK3$bRbzP%^ zA_MKmMEbdh_7C*+a|rMXSI+3GPWRA)aIDYJ+t16F3Q585ad|xuX)E<~9HYX5aDtJt zPT_aWqGro{L5bWWi`@N!!`!?g1K__*eo-dwfhh9`N|;KytRbyTp2`Xjg?WmRzS1bgj{H2;F=2j+zK)|`kYD6iS)L;H@(PcP3Zn+oU%8{- zAH~JlNh}KE};2oG-8*;^XHK8pTVcEX9|$O5j0y}7 z4GBh->4Q2L;va#OR`7cLt)P&iEsU36Jxa$h9818v@4 z%39i|lY<);aG(@>I=Pa2Cr`+A_ar>m=j%8JMMVTqM%qUWAC9gR4_FEvn^O2d zJvPOV&C@AbQGV6$?-dw{o~jQDJIGJ0$*)-Px`3w2Z4&G8uP9|@y{3crS5E7?qJJefnf^|;sNU z)?r7*?Kb@Rkm^oKYHcUwWD_SPthtjC5aFaGk9NZP3r>pXG$*{Bg79@tisj#MKXX#{ zzC^q{C#B|RCuOdSvtl{GSuq&wtjzVHyMAA`9uGAKw%5|4u81uv5$}RS)J+GD>X1@W zXhcqKJxEQ6KBPRPG{g`>qdX&sfh6h3Ulu}P zAf3x(2!+Q%$ewwS?;tCse>F((RO6#pn=ifxom;V8v|`Ee-Df42YRz<(_VGFTYCwL_c1F|p2#9vFZ9|11;a=7IQg z5x*p9CJplP3B}0b%jOk<&psF0ydu4d_=`nXf1NQNK-61CE~|$JQ4Y_ z6-!Y52`ebLteWkOa+Mu>|m8vPKUu8-9lEVM1 z@zr!xs{8+k|9%b>{ff%hzEZ_DUAd`8ZmK%|Z5o%jiKNX+-2UxD|8Gj5)Tvy!S9(SVq?_2S_&7NX>z~fqZ}{>zx%n zhylb1QWatfv4Ge>Y$1*iH;6mL12P!m3GsylK!PA)kVr^0BnA=-nGQ*SEQe71wUEt_ zU66y2laMsXO-L5xDI^>60b;ztxnQhs3bzHs3epN<3vq_tP|P9q7t|`ODSq6l2#g%>tkeN_fhJ%QSQ|_P>w#%t70?%~4`zZD zKu3_~j%MBD+hsQ;8T$1lY>E;V@8A3KutrjHr{6V$>_Nt z%}KjJPjh)Ukk5Ugr_-DcHUK@KHwOoU7N95C5cCBbfkB|y)N%708nR-0A;-}Llh;@o zG1l0zpnPI!0YR_CNUS-; z6%qtd_vwJJfskm(V#x21bjWK+CB(CV^ne6IV#9<@*)PTT->r{3V*TH5ZjS%n{stGZ zE-Zx3MYW=^Th&p$9*cUKfP&$3e?6QBJ`qdzO{gwtAfM5<#)1=(0F^u4k1>N#Vex%H zbyJ^Dt9~V^l%~LaQvZwl9slhV57I0*D$u(fsaW8)$f>NlV;3b>SKVd@Q)&sfp7vDj z*{#UE`q|m@wGUtFp)t!|`FmK?J=J;tg5*vIr;O~|xc#jBu#6b{ zgSTeie!JrRlkppSIjnv->i*eR-!+e$*vQe+xW>jeTa51?JNnM?$%gdT`A_b48@up= zli%4B^D3<0l2q&dFV5xtY8XXLoEkOiR2Nqd`^*gs+IMKZcgM3nt{WZ<>3D2NhnXwq z5Bxl2efDazRQGl9ZN2{dICJ34s||g{>7 zVHD0+Id}HlS+TumKXxp8<(Dw}_l6Dh?Qo~(ti>Uf@mrT8m#qA1-Seoi=g*fct)h8c z=h~e&AKUPpY1Z!!4BD#yN871SSj&zF?Z-a&D?xktZ+7_O$Marsl}4Z4l*yXA47%%j zG$mo-!&7Yj(LNob9M4rvS-O?o_TQh--s!ix+b<+A3-jam4({*J!fWPO=GAw=q4C@M z#8f^oj9q!&d{*g{`BSRBcVrf9-J|yF8b6+p+=PwWygKusZ;W%=t-f|XKb1af?ml|* z=Xn!5&U^Pmt+7509a^Mrwyi(oPb@dSc(3yDN{5@%Jq&7Z&$>T#y>5r&aZdJSb}xx) zd!yM8Eq^q>n5#V^ZWPr0(Ml|(0i5RfIY@m4ZCJqw`tY0 z&zAEe8_pS^w>I_m;c@S?j@MoJ_w<4N=hfKs@_5X!gJCzV^I=r7mn~CM#f`9aNSo9y?(9?ST^e&|*TG=>?RBW?l(ZNAxqr{`? zj*VM1vuSMRN6$R*_@rawzV6P}?HZd!;4X!)R}h|pbZk7vFQPHCD__>uDVhw!2VNOj1lT zOpcu>bIff~8}{PKVO^?*FQH8_JCI(g3I?6~aU-(IiVXAQb*&W!1k@9q^hds@GV z!6!F+y{6orJ!^8WxW0cJ$L;%{pT+I_cWZL{{&8`5yW98oxZ#I?*o%MpV-$EAzlb5d z$y~C8yiL9$rr{X@-W`AMg?48BrxZ6M1Hs)xk=#9<_?C_113dxOI&*<#M@Cc2`o(Q%|V_>QnVs zW!uClbl!EwxTeSS4o&>wJ{A-Qe#XlvGMdcCx;`M^lAlNmBg$~GwlT)-#$sc&u>tEl zYsAz3bRZpx^-Z8t>D}~xtgx7t(l_WX`e)ie+p&0NvZ+|X=KkxS%uxmS)<)6`nE4N<+$o?#!czqNm{ z`#bZo)6Y6@Ie&4!aGJYUx>c_B(!IIfaW7U6&}n*t&e0F(HOPWb^w+wD-_y7KOn*%< zA9A46@DfFakx|B1*z<98A}wW4iFd?4aY}qI23SL_d#%Tf}00 zoORA_=L4s$OWc+221M;XZ?K-Irv=wv7Q*GgokkCq!|&%C_)*?oEHamxCFZAATiHRr zF5i{)c6aAh=U8yf75E}EUS^Od@hIO^`T#vf!`U!4l4Y`)Y(94JIkuVIhdMrr+#Djd zih!A54mNYlb>@fWr{*~`+PcO{vr4VMS^corWpajEuAWepszxQ+H`x>HHTE|9bGx(+tU@U?M`qXa+kZS-DlnP?yGLOyUqQRd&vFF{mzZ>x_Y;I8QyfS z*n7h};a&9F=-$YRX?nTdtXukR{Hy&Devbc)zdLx{RhMe33t?m^nM4*K%i9^k7-BRT zJ!lm@&i=+bi3i2=qFi)0`Qp-0++*(D-g)nU{>TqBUecyvWFmQ#JZg-g%V=-5f+v`ZtwJkU zYyIVoaL9B;kdSO5+0@XCN=) zTX;wDlK4iB0C8(?!U9BI1yFpd`RD!j)u-WV4agvCIzam@JM*vv2T zGoqP!+`P?NC&Sb(wO1Xs>+K{b%$=e~`A_)4#v>m8zZJQTq>?OSgX}V^TnQW$*L$QZxX1bYSW}1`CEbL~EnQP`@ zPxH-%W&tw3&|GO2nZ?Nb^=7HL$t*L=&8_ARGufJMJ&&uBYJwW%OmOaWW?^?%I7Ln! zaOkhjwQe8erh}Y(-|g&m^O!fud&Ilo{o*y#t@P!(y}n9!*4=aub?3){f!P}hK25Zn-dnAJ1?Xw z=xcNbJwd;wtywhV%xB}-9c&4ElD)|)SRFgV+VGjEj@A4*?pdkURBNvFy!Dy2TD~Z^ zfSq1d=hPK8wddMT*jFJsHsX`!)Hz(Z}`oI@&My-}7t0_vif3-FouFLY&_QdMb$wM!t_B4*;E3kd@>OvH<-5 zlko}Vi0NeZ2$Vn%Zt@rTW?scV;(NtN^RQ`KIo31QLCXXRoRX`cG>V-rZln97*GXTa zXX*{QT6gwu^KlQd!m!Y@6WWtf5^GEd9owNvbTGY*wnaqVVMp0lY%TwuJ0e;1Gmn{9 zS^vWtVU4%)tU7Cy93$_L&&$u`WOa{v2WspS^_`-q`hE5pyRFm7xeifz$oU%h-NNnV zdhSi`?LdNu+&%8Kp5+bl(!BZJQg5yInz!BC6N=U`?=!E9P6v-~*M{%-6a43c`&JTm zNjs2UgafM!$P=W5yiUF&V~u-^g{b%%<8!FYX3%jj(Jk}s$sNsFkdGA|?tPib^p^QJZzOcSV{hkFU{RCzWlPzRxdAV#a z$Eo-2kKF!xkN&If<@bfwd&=MCAM-#e}@u&1F`YpYI-N%+AsvUR&r(AN)lc7al;T}+Sfw{swVxBb5 zm_M5>EX&HYW?2iYJ=QVnlogPzu&U0o8#r^I9EA+MUCxsa%W}D2c2u6aP2H}hsK?bS zP=1x_kUF6*x37er?q&D42VzH7+ZA@TebPRQ{K|4JLIYk644Uh{=I!#1dY^hxdN%am z!@5N8LYAMY|EuZg#dh2OZ*$hN?RQ#Ga?0 z&|m3({;mG)ey;zR|Fpl&KY$}N3w;*mWpaj`Cru=U-bBmj7&Zawbp`vCMRSKQw&uvw zs!??WemZtPyVhx`FVk)H6{!EKbr&#IwBH2|0ry?MKX!i%&XD8J_4ED5{bFSLPXB%X zsDHu_-E}Og89rN$&(vUVt|YOf2k?LpNj!1`83I(eg^Ys|m_}w|-{z5p$e9)7X=s?| z$wu-jd6Vn}OIDI9@*(+zd`{}o{hTKkNtn^vXlq0pu|_xSa$kdkY5GHP4l`~xMjM$% zmT{*s%ec?THy*~`KLH*39D0sTsIYCuE@MA1{V+KDr11qZ;yYx1fVQA*Xa^cYyU?EW zI`jpKYV-y}=?HXF`4`Ed|c(XGhox z)&f{IoX_L)p+dLw-O!L<@F-Djo> zK$eopMzQgh5k|K{d9P(}fCtXANZx~cJd@}0r+EqA$Pe*Typ4EHY=kbX6km%Q&1L3` z<`(k=IJCVr%(?^hw^6z zF1ntcrx)pEOrld*&8pbvY!(>d1E|+uM1<)DM#B4GbS@WR?A4YYJx67Q7?i9Dk zUFVj$+o7O8aL0R-y?eX@?>(>Hdrrq9Hw*pH-7%Azg`T(5+u-Pcqxg8JkmjO|aK!+T zDjr27wulPx9+bve@i);Sgr+&r$8D|d)--E2_UlP&FDie8{6)5d9v!GQsLiTcT>!#H zLhsMBKe0PI-JP}01}NT6?qYW<5OamM4p{lFcgpLf$NEc9AI}E&Dz#bYnOd94Nz%M+pZ7uCuhjGY^4--4VPhu&n0eTRLQeYZUa z%>RJ>nEivD4J~vWYI%m6hpxo*9B&vpkO#d*9@Un1^(bAe*XftRxN-iqzCb@Z2?$(_ z-NijdGn<7xibxWywv|SS@q)3@c-7cyw4qVJmoz#Sj>L4DgZeo}PtjkYR`S?u>}@zD zt&o>CPk-kC}flYs~XzmL=dZ+=;&JpnOS{qdWXWt+6-QuRGhF7u~b& z&u({b9@wZv_w|SPGyOnLv(P+by}8j0kRdg!?v)=`~_al_n=D8^C^gU0IJeAhoB->nJ<_>AP26qlC5>X{s)nFwtc~gal5+x z+`HYw=xn}plf4n%{fJa4b~;xt1!|X}bKj>2`8WGx{b~MJ@IXR$R4lx-@6CaQCVYwY zSkEEjV}nu~zS4SVx#RQ;8UZ|cnC)YS*jX0FFXtWkV*Vsw!`q6^qL-K`9v7?8&3z)Jco>Jsd_0#-J+(d995uJLUnFZ zr_}f0jW9a~PETL^Mtc}^ax$EWADkt~fDOpsN+`guy&uqm zhkKHiGz;C`_F=M|JVn;RS2<1E0F51B?JBwn=yMWRa~B*j6ACmRe%NbJ^M4T^o7Y;0 z)qcCie#_mBuD7|D>pkohd987tPUtUuJrFqZCe+1;`co~Tp6>7;0e`LYUx$XdgeCXF z4O>hegO1rkcA<){hZ{D`xXrj1zRqc2dOO+`IBp};u4Fx+-iNa^HVs^V1Z+Q==kR=f z9R0^nyoKl>s2Bz{KTZrW-!=cyV&n%Nqfm^bdf5L~0G%-fpF76d8#AfjU zI-y@gbM!%-&2H!r1v>k|=B?-gA2MGvx0_?-ba@uNhOKT?)3N7s)I(~iTBlxy9yth1 zs8Jj3a{IXbm7RtjS-A=D(gs1zYp4a>bGNQp=$Um-A}9A_7f;aB^k*7i?N~?FlO-~W zD@bBDhGIGz%3(UYo6Thla1F13ZFb-q4x&@K8j5NN>iRwYF<%6?VMi#sRlw95IHJ9* zpih+w2PNIgfVP@sWreV9hgAWb?Jb2&mAA?anF$0~Bwv-E%6i!VKk(ciJ=AcOrqWf0 z%2f9`UEO6~fgif-YGbp|^T|38i`+#XhYtLeBtReJ0ht=0tTW-Nr+%xV6e&`OTf##uSd-Z^Fxd*wog^WPA zF$>E5eW3Rz^hbI<6y!*LJL+{Ee;wU!j7Wqlaa25R9)yy-#=6!L))TmzpJWTwO-;hS zRHF{Bum{*f>{0N%K19dyiyi0mLTCPnv&X4%PC2dJD0i59i+i`f5QtmqzwIA{whGG#ie1KIc^{sHbu5LBO@@+9gI8eV9D%szArChL3g&9;<1=t@ zcCdB4im11v!f}5A~cWBn{s9yQGWJ+sH9? z8*S-3bTWLPr=gqQLq`$`lo}`20mVDvtdpQ9+E`tzu~7J};imd>fSfOP%WITU1Jn>` zj1#p^=!fT!Z)z+22#&^QQ8zshzyTs2}TZ*|0uHe_! zW9k~5X(u|pLv|fFaDlVa8Q{L)Z-I~8Jmj-p$NDnHEasrMz5#VR1eH4qwHtH-Ch&$3 zXI%^5VlHrbg}DaKAUaF=mOL+8sk3S?dg*}sj{6Z=8@5Q?z~s-i@%*QI(BII~=D z1#ecsfvMC7p(3l%UDoP4eM;Bs27LkhjV}-R5q_i}20W~~{XYp*XK`uO!d8n5{z7if_ zF;9XMnj(h+gVJRBZzz-nX3degVApxz)va=etdiBT20U@lLQ|* z#Tg3!D$PlEGT^;WLQjwlCo9*kh(NILOcAOpmha;Eb4aLOie;?C- zCxaXHxqr&yKw--eU)7LWQb+!0HJ^e~{k@`_(4U6W2pUO)x}X7blqMFASVkkqPD4&lLO$mrmlq;dfs&S?p6Bmz;$I&*ibkXRjiqr=#|bo%63VHhE?l%^nu0z! z6)HX*bI(ll!dWyMT0WQlz89&#@$f(AasgCFFrQJIbuB}_&|q}NBl#^n5QXoj)miK!b|W{z}5v|wbQB%DN`mZMP3 zF{tM_RCEGrnxLvB>N*LPor2m Date: Thu, 6 May 2021 16:01:29 +0300 Subject: [PATCH 123/652] add contribs to fasttest --- docker/test/fasttest/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index a7cc398e5c9..da05998ae0b 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -185,6 +185,9 @@ function clone_submodules contrib/dragonbox contrib/fast_float contrib/NuRaft + contrib/lizard + contrib/density + contrib/lzsse ) git submodule sync From d60c70a8da0d9d07eaeff27a652de56e7d6164df Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 16:17:55 +0300 Subject: [PATCH 124/652] add submodules --- contrib/density | 1 + contrib/lizard | 1 + contrib/lzsse | 1 + 3 files changed, 3 insertions(+) create mode 160000 contrib/density create mode 160000 contrib/lizard create mode 160000 contrib/lzsse diff --git a/contrib/density b/contrib/density new file mode 160000 index 00000000000..67bd584bd74 --- /dev/null +++ b/contrib/density @@ -0,0 +1 @@ +Subproject commit 67bd584bd7414d4bc1418e5b7fcf9d68e44c3f28 diff --git a/contrib/lizard b/contrib/lizard new file mode 160000 index 00000000000..af8518ccb8c --- /dev/null +++ b/contrib/lizard @@ -0,0 +1 @@ +Subproject commit af8518ccb8c68e062a8c80205ff07d56a2e77dd4 diff --git a/contrib/lzsse b/contrib/lzsse new file mode 160000 index 00000000000..1847c3e8279 --- /dev/null +++ b/contrib/lzsse @@ -0,0 +1 @@ +Subproject commit 1847c3e82794400deb56edd30d8aa3f445fd000b From 94b159c51039404a1577c8d16289c37b3c509239 Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 16:34:27 +0300 Subject: [PATCH 125/652] add newlines --- contrib/density-cmake/CMakeLists.txt | 2 +- contrib/lizard-cmake/CMakeLists.txt | 2 +- contrib/lzsse-cmake/CMakeLists.txt | 2 +- src/Compression/CompressionCodecLZSSE2.h | 2 +- src/Compression/CompressionCodecLZSSE4.h | 2 +- src/Compression/CompressionCodecLZSSE8.h | 2 +- src/Compression/CompressionCodecLizard.h | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/contrib/density-cmake/CMakeLists.txt b/contrib/density-cmake/CMakeLists.txt index d48bbbf05b9..844459c4105 100644 --- a/contrib/density-cmake/CMakeLists.txt +++ b/contrib/density-cmake/CMakeLists.txt @@ -44,4 +44,4 @@ SET (Headers ADD_LIBRARY(density ${Sources} ${Headers}) -target_include_directories (density PUBLIC ${LIBRARY_DIR}) \ No newline at end of file +target_include_directories (density PUBLIC ${LIBRARY_DIR}) diff --git a/contrib/lizard-cmake/CMakeLists.txt b/contrib/lizard-cmake/CMakeLists.txt index eaf63db0acd..8aa40418ec1 100644 --- a/contrib/lizard-cmake/CMakeLists.txt +++ b/contrib/lizard-cmake/CMakeLists.txt @@ -16,4 +16,4 @@ SET (Headers ADD_LIBRARY(lizard ${Sources} ${Headers}) -target_include_directories (lizard PUBLIC ${LIBRARY_DIR}) \ No newline at end of file +target_include_directories (lizard PUBLIC ${LIBRARY_DIR}) diff --git a/contrib/lzsse-cmake/CMakeLists.txt b/contrib/lzsse-cmake/CMakeLists.txt index a463089a5b6..a4c0d9dd8e7 100644 --- a/contrib/lzsse-cmake/CMakeLists.txt +++ b/contrib/lzsse-cmake/CMakeLists.txt @@ -18,4 +18,4 @@ SET (Headers ADD_LIBRARY(lzsse ${Sources} ${Headers}) -target_include_directories (lzsse PUBLIC ${LIBRARY_DIR}) \ No newline at end of file +target_include_directories (lzsse PUBLIC ${LIBRARY_DIR}) diff --git a/src/Compression/CompressionCodecLZSSE2.h b/src/Compression/CompressionCodecLZSSE2.h index d26bb3e2cf2..b42ae4e7aca 100644 --- a/src/Compression/CompressionCodecLZSSE2.h +++ b/src/Compression/CompressionCodecLZSSE2.h @@ -30,4 +30,4 @@ private: const int level; }; -} \ No newline at end of file +} diff --git a/src/Compression/CompressionCodecLZSSE4.h b/src/Compression/CompressionCodecLZSSE4.h index ed1cc8fb26b..9c59e56a053 100644 --- a/src/Compression/CompressionCodecLZSSE4.h +++ b/src/Compression/CompressionCodecLZSSE4.h @@ -30,4 +30,4 @@ private: const int level; }; -} \ No newline at end of file +} diff --git a/src/Compression/CompressionCodecLZSSE8.h b/src/Compression/CompressionCodecLZSSE8.h index c9d006ca13b..3d51a5ead5b 100644 --- a/src/Compression/CompressionCodecLZSSE8.h +++ b/src/Compression/CompressionCodecLZSSE8.h @@ -30,4 +30,4 @@ private: const int level; }; -} \ No newline at end of file +} diff --git a/src/Compression/CompressionCodecLizard.h b/src/Compression/CompressionCodecLizard.h index 95bef7ab39e..8b39f224dbc 100644 --- a/src/Compression/CompressionCodecLizard.h +++ b/src/Compression/CompressionCodecLizard.h @@ -31,4 +31,4 @@ private: const int level; }; -} \ No newline at end of file +} From f01b53bafbd7e7450d51949b95291c40cdc808e2 Mon Sep 17 00:00:00 2001 From: fibersel <0583463@gmail.com> Date: Thu, 6 May 2021 16:47:23 +0300 Subject: [PATCH 126/652] add newline --- src/Compression/CompressionCodecDensity.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDensity.h b/src/Compression/CompressionCodecDensity.h index cfc8bfbd76a..ffdbd8f3619 100644 --- a/src/Compression/CompressionCodecDensity.h +++ b/src/Compression/CompressionCodecDensity.h @@ -32,4 +32,4 @@ private: const DENSITY_ALGORITHM algo; }; -} \ No newline at end of file +} From 6aa7c0fafcca9c1493e6923ef68592ab68b286da Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 May 2021 16:13:26 +0300 Subject: [PATCH 127/652] fixes --- tests/integration/helpers/cluster.py | 2 ++ tests/integration/helpers/dictionary.py | 4 ++-- .../configs/dictionaries/ssd_complex_key_cache_string.xml | 2 +- .../test_match_process_uid_against_data_owner/test.py | 3 ++- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a4cfc0cdc4d..3de6defab4f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1854,6 +1854,7 @@ class ClickHouseInstance: instance_config_dir = p.abspath(p.join(self.path, 'configs')) os.makedirs(instance_config_dir) + os.chmod(instance_config_dir, stat.S_IRWXO) logging.debug("Copy common default production configuration from {}".format(self.base_config_dir)) shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml')) @@ -1873,6 +1874,7 @@ class ClickHouseInstance: dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) os.mkdir(dictionaries_dir) + logging.debug("Copy common configuration from helpers") # The file is named with 0_ prefix to be processed before other configuration overloads. shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) diff --git a/tests/integration/helpers/dictionary.py b/tests/integration/helpers/dictionary.py index 41d87180c8a..b32b516d82c 100644 --- a/tests/integration/helpers/dictionary.py +++ b/tests/integration/helpers/dictionary.py @@ -7,12 +7,12 @@ class Layout(object): 'flat': '', 'hashed': '', 'cache': '128', - 'ssd_cache': '/etc/clickhouse/dictionaries/all', + 'ssd_cache': '/etc/clickhouse-server/dictionaries/all', 'complex_key_hashed': '', 'complex_key_hashed_one_key': '', 'complex_key_hashed_two_keys': '', 'complex_key_cache': '128', - 'complex_key_ssd_cache': '/etc/clickhouse/dictionaries/all', + 'complex_key_ssd_cache': '/etc/clickhouse-server/dictionaries/all', 'range_hashed': '', 'direct': '', 'complex_key_direct': '' diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml index c8fdbcbe0ef..729dc11d0a1 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml @@ -41,7 +41,7 @@ 16777216 131072 1048576 - /etc/clickhouse/dictionaries/radars + /etc/clickhouse-server/dictionaries/radars 1 diff --git a/tests/integration/test_match_process_uid_against_data_owner/test.py b/tests/integration/test_match_process_uid_against_data_owner/test.py index 8adcc4463c9..754ebc731c1 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -31,7 +31,8 @@ def test_different_user(): with open(os.path.join(node.path, 'logs/clickhouse-server.err.log')) as log: expected_message = "Effective user of the process \(.*\) does not match the owner of the data \(.*\)\. Run under 'sudo -u .*'\." - last_messages = ";".join(log.readlines()[-2:-1]) + + last_message = [row for row in log.readlines() if "Effective" in row][-1] if re.search(expected_message, last_message) is None: pytest.fail( From d0a1dcd6cc514f12086b241f216108357ef3538b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 May 2021 17:27:38 +0300 Subject: [PATCH 128/652] better name --- tests/integration/helpers/cluster.py | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3de6defab4f..d3f88203eac 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -61,7 +61,7 @@ def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=su raise Exception(f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}") # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 -def get_open_port(): +def get_free_port(): s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) s.bind(("",0)) s.listen(1) @@ -252,34 +252,34 @@ class ClickHouseCluster: # available when with_hdfs == True self.hdfs_host = "hdfs1" - self.hdfs_name_port = get_open_port() - self.hdfs_data_port = get_open_port() + self.hdfs_name_port = get_free_port() + self.hdfs_data_port = get_free_port() self.hdfs_dir = p.abspath(p.join(self.instances_dir, "hdfs")) self.hdfs_logs_dir = os.path.join(self.hdfs_dir, "logs") # available when with_kerberized_hdfs == True self.hdfs_kerberized_host = "kerberizedhdfs1" - self.hdfs_kerberized_name_port = get_open_port() - self.hdfs_kerberized_data_port = get_open_port() + self.hdfs_kerberized_name_port = get_free_port() + self.hdfs_kerberized_data_port = get_free_port() self.hdfs_kerberized_dir = p.abspath(p.join(self.instances_dir, "kerberized_hdfs")) self.hdfs_kerberized_logs_dir = os.path.join(self.hdfs_kerberized_dir, "logs") # available when with_kafka == True self.kafka_host = "kafka1" - self.kafka_port = get_open_port() + self.kafka_port = get_free_port() self.kafka_docker_id = None self.schema_registry_host = "schema-registry" - self.schema_registry_port = get_open_port() + self.schema_registry_port = get_free_port() self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) # available when with_kerberozed_kafka == True self.kerberized_kafka_host = "kerberized_kafka1" - self.kerberized_kafka_port = get_open_port() + self.kerberized_kafka_port = get_free_port() self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kerberized_kafka_host) # available when with_mongo == True self.mongo_host = "mongo1" - self.mongo_port = get_open_port() + self.mongo_port = get_free_port() # available when with_cassandra == True self.cassandra_host = "cassandra1" @@ -294,7 +294,7 @@ class ClickHouseCluster: # available when with_redis == True self.redis_host = "redis1" - self.redis_port = get_open_port() + self.redis_port = get_free_port() # available when with_postgres == True self.postgres_host = "postgres1" From fe0194aa4b27e5124c273c9b3b040a84299231f6 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 May 2021 10:03:53 +0300 Subject: [PATCH 129/652] s3 catch up --- .../runner/compose/docker_compose_minio.yml | 8 +- tests/integration/parallel.json | 554 ++++++++++++++++++ tests/integration/parallel.readme | 6 + tests/integration/test_storage_s3/test.py | 21 +- 4 files changed, 575 insertions(+), 14 deletions(-) create mode 100644 tests/integration/parallel.readme diff --git a/docker/test/integration/runner/compose/docker_compose_minio.yml b/docker/test/integration/runner/compose/docker_compose_minio.yml index 99792369f77..96a5f8bdc31 100644 --- a/docker/test/integration/runner/compose/docker_compose_minio.yml +++ b/docker/test/integration/runner/compose/docker_compose_minio.yml @@ -6,7 +6,7 @@ services: volumes: - data1-1:/data1 - ${MINIO_CERTS_DIR:-}:/certs - ports: + expose: - ${MINIO_PORT} environment: MINIO_ACCESS_KEY: minio @@ -20,14 +20,14 @@ services: # HTTP proxies for Minio. proxy1: image: yandex/clickhouse-s3-proxy - ports: + expose: - "8080" # Redirect proxy port - "80" # Reverse proxy port - "443" # Reverse proxy port (secure) proxy2: image: yandex/clickhouse-s3-proxy - ports: + expose: - "8080" - "80" - "443" @@ -35,7 +35,7 @@ services: # Empty container to run proxy resolver. resolver: image: yandex/clickhouse-python-bottle - ports: + expose: - "8080" tty: true depends_on: diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index c783798055d..143d21f52b5 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -1,315 +1,869 @@ [ + "test_dictionaries_access/test.py::test_create", + "test_dictionaries_access/test.py::test_create", + "test_dictionaries_access/test.py::test_drop", + "test_dictionaries_access/test.py::test_drop", + "test_dictionaries_access/test.py::test_dictget", + "test_dictionaries_access/test.py::test_dictget", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", + "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", + "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", + "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", + "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", + "test_dictionaries_ddl/test.py::test_restricted_database", + "test_dictionaries_ddl/test.py::test_restricted_database", + "test_dictionaries_ddl/test.py::test_conflicting_name", + "test_dictionaries_ddl/test.py::test_conflicting_name", + "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_dictionary_with_where", + "test_dictionaries_ddl/test.py::test_dictionary_with_where", + "test_dictionaries_ddl/test.py::test_clickhouse_remote", + "test_dictionaries_ddl/test.py::test_clickhouse_remote", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", + "test_dictionaries_dependency_xml/test.py::test_get_data", + "test_dictionaries_dependency_xml/test.py::test_get_data", "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", + "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", + "test_dictionaries_null_value/test.py::test_null_value", + "test_dictionaries_null_value/test.py::test_null_value", + "test_dictionaries_postgresql/test.py::test_load_dictionaries", "test_dictionaries_postgresql/test.py::test_load_dictionaries", "test_dictionaries_postgresql/test.py::test_invalidate_query", + "test_dictionaries_postgresql/test.py::test_invalidate_query", "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", + "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", + "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", + "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", + "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", + "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", + "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", + "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", + "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", + "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", + "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", + "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", + "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", + "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", + "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", + "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", + "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", + "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", + "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", + "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", + "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", + "test_dictionary_custom_settings/test.py::test_work", + "test_dictionary_custom_settings/test.py::test_work", + "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", + "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", + "test_disabled_mysql_server/test.py::test_disabled_mysql_server", "test_disabled_mysql_server/test.py::test_disabled_mysql_server", "test_disk_access_storage/test.py::test_create", + "test_disk_access_storage/test.py::test_create", + "test_disk_access_storage/test.py::test_alter", "test_disk_access_storage/test.py::test_alter", "test_disk_access_storage/test.py::test_drop", + "test_disk_access_storage/test.py::test_drop", + "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", + "test_distributed_storage_configuration/test.py::test_insert", "test_distributed_storage_configuration/test.py::test_insert", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", + "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", + "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", + "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", + "test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database", "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", + "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", + "test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine", "test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine", "test_mysql_database_engine/test.py::test_mysql_types[common_types_1]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_1]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_10]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_10]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_11]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_11]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_12]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_12]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_13]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_13]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_14]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_14]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_15]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_15]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_16]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_16]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_17]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_17]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_18]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_18]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_19]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_19]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_20]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_20]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", + "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", + "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]", "test_mysql_protocol/test.py::test_mysql_client", + "test_mysql_protocol/test.py::test_mysql_client", + "test_mysql_protocol/test.py::test_mysql_client_exception", "test_mysql_protocol/test.py::test_mysql_client_exception", "test_mysql_protocol/test.py::test_mysql_affected_rows", + "test_mysql_protocol/test.py::test_mysql_affected_rows", + "test_mysql_protocol/test.py::test_mysql_replacement_query", "test_mysql_protocol/test.py::test_mysql_replacement_query", "test_mysql_protocol/test.py::test_mysql_explain", + "test_mysql_protocol/test.py::test_mysql_explain", + "test_mysql_protocol/test.py::test_mysql_federated", "test_mysql_protocol/test.py::test_mysql_federated", "test_mysql_protocol/test.py::test_mysql_set_variables", + "test_mysql_protocol/test.py::test_mysql_set_variables", + "test_mysql_protocol/test.py::test_python_client", "test_mysql_protocol/test.py::test_python_client", "test_mysql_protocol/test.py::test_golang_client", + "test_mysql_protocol/test.py::test_golang_client", + "test_mysql_protocol/test.py::test_php_client", "test_mysql_protocol/test.py::test_php_client", "test_mysql_protocol/test.py::test_mysqljs_client", + "test_mysql_protocol/test.py::test_mysqljs_client", + "test_mysql_protocol/test.py::test_java_client", "test_mysql_protocol/test.py::test_java_client", "test_mysql_protocol/test.py::test_types", + "test_mysql_protocol/test.py::test_types", + "test_odbc_interaction/test.py::test_mysql_simple_select_works", "test_odbc_interaction/test.py::test_mysql_simple_select_works", "test_odbc_interaction/test.py::test_mysql_insert", + "test_odbc_interaction/test.py::test_mysql_insert", + "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", "test_odbc_interaction/test.py::test_sqlite_table_function", + "test_odbc_interaction/test.py::test_sqlite_table_function", + "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", + "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", + "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", "test_odbc_interaction/test.py::test_postgres_insert", + "test_odbc_interaction/test.py::test_postgres_insert", + "test_odbc_interaction/test.py::test_bridge_dies_with_parent", "test_odbc_interaction/test.py::test_bridge_dies_with_parent", "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", + "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", + "test_odbc_interaction/test.py::test_odbc_postgres_conversions", "test_odbc_interaction/test.py::test_odbc_postgres_conversions", "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", + "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", + "test_odbc_interaction/test.py::test_many_connections", "test_odbc_interaction/test.py::test_many_connections", "test_odbc_interaction/test.py::test_concurrent_queries", + "test_odbc_interaction/test.py::test_concurrent_queries", + "test_odbc_interaction/test.py::test_odbc_long_column_names", "test_odbc_interaction/test.py::test_odbc_long_column_names", "test_odbc_interaction/test.py::test_odbc_long_text", + "test_odbc_interaction/test.py::test_odbc_long_text", + "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", + "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache", "test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache", "test_postgresql_protocol/test.py::test_psql_is_ready", + "test_postgresql_protocol/test.py::test_psql_is_ready", + "test_postgresql_protocol/test.py::test_psql_client", "test_postgresql_protocol/test.py::test_psql_client", "test_postgresql_protocol/test.py::test_python_client", + "test_postgresql_protocol/test.py::test_python_client", "test_postgresql_protocol/test.py::test_java_client", + "test_postgresql_protocol/test.py::test_java_client", + "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", + "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", + "test_redirect_url_storage/test.py::test_url_without_redirect", "test_redirect_url_storage/test.py::test_url_without_redirect", "test_redirect_url_storage/test.py::test_url_with_globs", + "test_redirect_url_storage/test.py::test_url_with_globs", + "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", + "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", + "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", "test_reloading_storage_configuration/test.py::test_add_disk", + "test_reloading_storage_configuration/test.py::test_add_disk", + "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", "test_reloading_storage_configuration/test.py::test_add_policy", + "test_reloading_storage_configuration/test.py::test_add_policy", + "test_reloading_storage_configuration/test.py::test_new_policy_works", "test_reloading_storage_configuration/test.py::test_new_policy_works", "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", + "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", + "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", "test_reloading_storage_configuration/test.py::test_remove_disk", + "test_reloading_storage_configuration/test.py::test_remove_disk", + "test_reloading_storage_configuration/test.py::test_remove_policy", "test_reloading_storage_configuration/test.py::test_remove_policy", "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", + "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", + "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", + "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", + "test_storage_hdfs/test.py::test_read_write_storage", "test_storage_hdfs/test.py::test_read_write_storage", "test_storage_hdfs/test.py::test_read_write_storage_with_globs", + "test_storage_hdfs/test.py::test_read_write_storage_with_globs", + "test_storage_hdfs/test.py::test_read_write_table", "test_storage_hdfs/test.py::test_read_write_table", "test_storage_hdfs/test.py::test_write_table", + "test_storage_hdfs/test.py::test_write_table", + "test_storage_hdfs/test.py::test_bad_hdfs_uri", "test_storage_hdfs/test.py::test_bad_hdfs_uri", "test_storage_hdfs/test.py::test_globs_in_read_table", + "test_storage_hdfs/test.py::test_globs_in_read_table", + "test_storage_hdfs/test.py::test_read_write_gzip_table", "test_storage_hdfs/test.py::test_read_write_gzip_table", "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", + "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", + "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", + "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", + "test_storage_hdfs/test.py::test_write_gz_storage", "test_storage_hdfs/test.py::test_write_gz_storage", "test_storage_hdfs/test.py::test_write_gzip_storage", + "test_storage_hdfs/test.py::test_write_gzip_storage", + "test_storage_hdfs/test.py::test_virtual_columns", "test_storage_hdfs/test.py::test_virtual_columns", "test_storage_hdfs/test.py::test_read_files_with_spaces", + "test_storage_hdfs/test.py::test_read_files_with_spaces", + "test_storage_kafka/test.py::test_kafka_json_as_string", "test_storage_kafka/test.py::test_kafka_json_as_string", "test_storage_kafka/test.py::test_kafka_formats", + "test_storage_kafka/test.py::test_kafka_formats", + "test_storage_kafka/test.py::test_kafka_settings_old_syntax", "test_storage_kafka/test.py::test_kafka_settings_old_syntax", "test_storage_kafka/test.py::test_kafka_settings_new_syntax", + "test_storage_kafka/test.py::test_kafka_settings_new_syntax", + "test_storage_kafka/test.py::test_kafka_issue11308", "test_storage_kafka/test.py::test_kafka_issue11308", "test_storage_kafka/test.py::test_kafka_issue4116", + "test_storage_kafka/test.py::test_kafka_issue4116", + "test_storage_kafka/test.py::test_kafka_consumer_hang", "test_storage_kafka/test.py::test_kafka_consumer_hang", "test_storage_kafka/test.py::test_kafka_consumer_hang2", + "test_storage_kafka/test.py::test_kafka_consumer_hang2", + "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_select_empty", "test_storage_kafka/test.py::test_kafka_select_empty", "test_storage_kafka/test.py::test_kafka_json_without_delimiter", + "test_storage_kafka/test.py::test_kafka_json_without_delimiter", + "test_storage_kafka/test.py::test_kafka_protobuf", "test_storage_kafka/test.py::test_kafka_protobuf", "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", + "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", + "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", "test_storage_kafka/test.py::test_kafka_materialized_view", + "test_storage_kafka/test.py::test_kafka_materialized_view", + "test_storage_kafka/test.py::test_librdkafka_compression", "test_storage_kafka/test.py::test_librdkafka_compression", "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", + "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", + "test_storage_kafka/test.py::test_kafka_many_materialized_views", "test_storage_kafka/test.py::test_kafka_many_materialized_views", "test_storage_kafka/test.py::test_kafka_flush_on_big_message", + "test_storage_kafka/test.py::test_kafka_flush_on_big_message", + "test_storage_kafka/test.py::test_kafka_virtual_columns", "test_storage_kafka/test.py::test_kafka_virtual_columns", "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", + "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", + "test_storage_kafka/test.py::test_kafka_insert", "test_storage_kafka/test.py::test_kafka_insert", "test_storage_kafka/test.py::test_kafka_produce_consume", + "test_storage_kafka/test.py::test_kafka_produce_consume", + "test_storage_kafka/test.py::test_kafka_commit_on_block_write", "test_storage_kafka/test.py::test_kafka_commit_on_block_write", "test_storage_kafka/test.py::test_kafka_virtual_columns2", + "test_storage_kafka/test.py::test_kafka_virtual_columns2", + "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", "test_storage_kafka/test.py::test_kafka_flush_by_time", + "test_storage_kafka/test.py::test_kafka_flush_by_time", + "test_storage_kafka/test.py::test_kafka_flush_by_block_size", "test_storage_kafka/test.py::test_kafka_flush_by_block_size", "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", + "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", + "test_storage_kafka/test.py::test_kafka_rebalance", "test_storage_kafka/test.py::test_kafka_rebalance", "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", + "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", + "test_storage_kafka/test.py::test_exception_from_destructor", "test_storage_kafka/test.py::test_exception_from_destructor", "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", + "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", + "test_storage_kafka/test.py::test_bad_reschedule", "test_storage_kafka/test.py::test_bad_reschedule", "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", + "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", + "test_storage_kafka/test.py::test_premature_flush_on_eof", "test_storage_kafka/test.py::test_premature_flush_on_eof", "test_storage_kafka/test.py::test_kafka_unavailable", + "test_storage_kafka/test.py::test_kafka_unavailable", + "test_storage_kafka/test.py::test_kafka_issue14202", "test_storage_kafka/test.py::test_kafka_issue14202", "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", + "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", "test_storage_kerberized_hdfs/test.py::test_read_table", + "test_storage_kerberized_hdfs/test.py::test_read_table", + "test_storage_kerberized_hdfs/test.py::test_read_write_storage", "test_storage_kerberized_hdfs/test.py::test_read_write_storage", "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", + "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", + "test_storage_kerberized_hdfs/test.py::test_two_users", "test_storage_kerberized_hdfs/test.py::test_two_users", "test_storage_kerberized_hdfs/test.py::test_read_table_expired", + "test_storage_kerberized_hdfs/test.py::test_read_table_expired", + "test_storage_kerberized_hdfs/test.py::test_prohibited", "test_storage_kerberized_hdfs/test.py::test_prohibited", "test_storage_kerberized_hdfs/test.py::test_cache_path", + "test_storage_kerberized_hdfs/test.py::test_cache_path", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", + "test_storage_mongodb/test.py::test_simple_select", "test_storage_mongodb/test.py::test_simple_select", "test_storage_mongodb/test.py::test_complex_data_type", + "test_storage_mongodb/test.py::test_complex_data_type", + "test_storage_mongodb/test.py::test_incorrect_data_type", "test_storage_mongodb/test.py::test_incorrect_data_type", "test_storage_mysql/test.py::test_many_connections", + "test_storage_mysql/test.py::test_many_connections", + "test_storage_mysql/test.py::test_insert_select", "test_storage_mysql/test.py::test_insert_select", "test_storage_mysql/test.py::test_replace_select", + "test_storage_mysql/test.py::test_replace_select", + "test_storage_mysql/test.py::test_insert_on_duplicate_select", "test_storage_mysql/test.py::test_insert_on_duplicate_select", "test_storage_mysql/test.py::test_where", + "test_storage_mysql/test.py::test_where", + "test_storage_mysql/test.py::test_table_function", "test_storage_mysql/test.py::test_table_function", "test_storage_mysql/test.py::test_binary_type", + "test_storage_mysql/test.py::test_binary_type", + "test_storage_mysql/test.py::test_enum_type", "test_storage_mysql/test.py::test_enum_type", "test_storage_mysql/test.py::test_mysql_distributed", + "test_storage_mysql/test.py::test_mysql_distributed", + "test_storage_mysql/test.py::test_external_settings", "test_storage_mysql/test.py::test_external_settings", "test_storage_postgresql/test.py::test_postgres_select_insert", + "test_storage_postgresql/test.py::test_postgres_select_insert", + "test_storage_postgresql/test.py::test_postgres_conversions", "test_storage_postgresql/test.py::test_postgres_conversions", "test_storage_postgresql/test.py::test_non_default_scema", + "test_storage_postgresql/test.py::test_non_default_scema", + "test_storage_postgresql/test.py::test_concurrent_queries", "test_storage_postgresql/test.py::test_concurrent_queries", "test_storage_postgresql/test.py::test_postgres_distributed", + "test_storage_postgresql/test.py::test_postgres_distributed", + "test_storage_rabbitmq/test.py::test_rabbitmq_select", "test_storage_rabbitmq/test.py::test_rabbitmq_select", "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", + "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", + "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", + "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", + "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", + "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", + "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", + "test_storage_rabbitmq/test.py::test_rabbitmq_big_message", "test_storage_rabbitmq/test.py::test_rabbitmq_big_message", "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", + "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", + "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", "test_storage_rabbitmq/test.py::test_rabbitmq_insert", + "test_storage_rabbitmq/test.py::test_rabbitmq_insert", + "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", + "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", "test_storage_rabbitmq/test.py::test_rabbitmq_direct_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_direct_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_fanout_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_fanout_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_hash_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_hash_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", + "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", + "test_storage_rabbitmq/test.py::test_rabbitmq_headers_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_headers_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", + "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", + "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", "test_storage_rabbitmq/test.py::test_rabbitmq_many_consumers_to_each_queue", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_consumers_to_each_queue", + "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", + "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", + "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", + "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", + "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", "test_storage_rabbitmq/test.py::test_rabbitmq_vhost", + "test_storage_rabbitmq/test.py::test_rabbitmq_vhost", + "test_storage_s3/test.py::test_put[positive]", "test_storage_s3/test.py::test_put[positive]", "test_storage_s3/test.py::test_put[auth_positive]", + "test_storage_s3/test.py::test_put[auth_positive]", + "test_storage_s3/test.py::test_put[auto]", "test_storage_s3/test.py::test_put[auto]", "test_storage_s3/test.py::test_put[gzip]", + "test_storage_s3/test.py::test_put[gzip]", + "test_storage_s3/test.py::test_put[deflate]", "test_storage_s3/test.py::test_put[deflate]", "test_storage_s3/test.py::test_put[brotli]", + "test_storage_s3/test.py::test_put[brotli]", + "test_storage_s3/test.py::test_put[xz]", "test_storage_s3/test.py::test_put[xz]", "test_storage_s3/test.py::test_put[zstd]", + "test_storage_s3/test.py::test_put[zstd]", + "test_storage_s3/test.py::test_get_file_with_special[space]", + "test_storage_s3/test.py::test_get_file_with_special[space]", + "test_storage_s3/test.py::test_get_file_with_special[plus]", + "test_storage_s3/test.py::test_get_file_with_special[plus]", + "test_storage_s3/test.py::test_get_path_with_special[space]", + "test_storage_s3/test.py::test_get_path_with_special[space]", + "test_storage_s3/test.py::test_get_path_with_special[plus]", + "test_storage_s3/test.py::test_get_path_with_special[plus]", + "test_storage_s3/test.py::test_get_path_with_special[plus2]", + "test_storage_s3/test.py::test_get_path_with_special[plus2]", + "test_storage_s3/test.py::test_empty_put[minio]", "test_storage_s3/test.py::test_empty_put[minio]", "test_storage_s3/test.py::test_put_csv[positive]", + "test_storage_s3/test.py::test_put_csv[positive]", + "test_storage_s3/test.py::test_put_csv[auth_positive]", "test_storage_s3/test.py::test_put_csv[auth_positive]", "test_storage_s3/test.py::test_put_csv[negative]", + "test_storage_s3/test.py::test_put_csv[negative]", + "test_storage_s3/test.py::test_put_get_with_redirect", "test_storage_s3/test.py::test_put_get_with_redirect", "test_storage_s3/test.py::test_put_with_zero_redirect", + "test_storage_s3/test.py::test_put_with_zero_redirect", + "test_storage_s3/test.py::test_put_get_with_globs", "test_storage_s3/test.py::test_put_get_with_globs", "test_storage_s3/test.py::test_multipart_put[positive]", + "test_storage_s3/test.py::test_multipart_put[positive]", + "test_storage_s3/test.py::test_multipart_put[negative]", "test_storage_s3/test.py::test_multipart_put[negative]", "test_storage_s3/test.py::test_remote_host_filter", + "test_storage_s3/test.py::test_remote_host_filter", + "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", + "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", + "test_storage_s3/test.py::test_s3_glob_scheherazade", "test_storage_s3/test.py::test_s3_glob_scheherazade", "test_storage_s3/test.py::test_custom_auth_headers", + "test_storage_s3/test.py::test_custom_auth_headers", + "test_storage_s3/test.py::test_custom_auth_headers_exclusion", "test_storage_s3/test.py::test_custom_auth_headers_exclusion", "test_storage_s3/test.py::test_infinite_redirect", + "test_storage_s3/test.py::test_infinite_redirect", + "test_storage_s3/test.py::test_storage_s3_get_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_get_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_get_gzip[gz]", + "test_storage_s3/test.py::test_storage_s3_get_gzip[gz]", + "test_storage_s3/test.py::test_storage_s3_get_unstable", "test_storage_s3/test.py::test_storage_s3_get_unstable", "test_storage_s3/test.py::test_storage_s3_put_uncompressed", + "test_storage_s3/test.py::test_storage_s3_put_uncompressed", + "test_storage_s3/test.py::test_storage_s3_put_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_put_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", + "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", + "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]", "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]" diff --git a/tests/integration/parallel.readme b/tests/integration/parallel.readme new file mode 100644 index 00000000000..3a8d55cf4cd --- /dev/null +++ b/tests/integration/parallel.readme @@ -0,0 +1,6 @@ +# Parallel tests are currently generated manually. All tests except those in parallel.txt will be run sequentially. +# Current list is generated with following commands +# 1. Generate all tests list as in CI run +./runner ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' > all_tests.txt +# 2. Filter known tests that can be run in parallel +cat all_tests.txt | grep 'dict\|storage\|postgres\|mysql' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel.json diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index fae6bd8e92c..61f119fef76 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -149,25 +149,25 @@ def test_put(started_cluster, maybe_auth, positive, compression): "space", "plus" ]) -def test_get_file_with_special(cluster, special): +def test_get_file_with_special(started_cluster, special): symbol = {"space": " ", "plus": "+"}[special] urlsafe_symbol = {"space": "%20", "plus": "%2B"}[special] auth = "'minio','minio123'," - bucket = cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = [[12549, 2463, 19893], [64021, 38652, 66703], [81611, 39650, 83516], [11079, 59507, 61546], [51764, 69952, 6876], [41165, 90293, 29095], [40167, 78432, 48309], [81629, 81327, 11855], [55852, 21643, 98507], [6738, 54643, 41155]] values_csv = ('\n'.join((','.join(map(str, row)) for row in values)) + '\n').encode() filename = f"get_file_with_{special}_{symbol}two.csv" - put_s3_file_content(cluster, bucket, filename, values_csv) + put_s3_file_content(started_cluster, bucket, filename, values_csv) - get_query = f"SELECT * FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}two.csv', {auth}'CSV', '{table_format}') FORMAT TSV" + get_query = f"SELECT * FROM s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}two.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert [list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()] == values - get_query = f"SELECT * FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/get_file_with_{special}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" + get_query = f"SELECT * FROM s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/get_file_with_{special}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert [list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()] == values - get_query = f"SELECT * FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" + get_query = f"SELECT * FROM s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert [list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()] == values @@ -176,12 +176,12 @@ def test_get_file_with_special(cluster, special): "plus", "plus2" ]) -def test_get_path_with_special(cluster, special): +def test_get_path_with_special(started_cluster, special): symbol = {"space": "%20", "plus": "%2B", "plus2": "%2B"}[special] safe_symbol = {"space": "%20", "plus": "+", "plus2": "%2B"}[special] auth = "'minio','minio123'," table_format = "column1 String" - instance = cluster.instances["dummy"] + instance = started_cluster.instances["dummy"] get_query = f"SELECT * FROM s3('http://resolver:8082/get-my-path/{safe_symbol}.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert run_query(instance, get_query).splitlines() == [f"/{symbol}.csv"] @@ -442,13 +442,14 @@ def run_s3_mocks(started_cluster): for mock_filename, container, port in mocks: for attempt in range(10): ping_response = started_cluster.exec_in_container(started_cluster.get_container_id(container), - ["curl", "-s", f"http://{container}:{port}/"], nothrow=True) + ["curl", "-s", f"http://localhost:{port}/"], nothrow=True) if ping_response != 'OK': if attempt == 9: assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) else: time.sleep(1) else: + logging.debug(f"mock {mock_filename} ({port}) answered {ping_response} on attempt {attempt}") break logging.info("S3 mocks started") From aa4c91a2b3b19374524e9fb0ab411d9ce105c2a9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 May 2021 17:15:08 +0300 Subject: [PATCH 130/652] disable parallel in flaky check --- tests/integration/ci-runner.py | 13 +- tests/integration/parallel.json | 1813 +++++++++++++++++------------ tests/integration/parallel.readme | 4 +- 3 files changed, 1107 insertions(+), 723 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 2dfa99e11b3..3b4e9568116 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -12,6 +12,7 @@ import csv MAX_RETRY = 2 +NUM_WORKERS = 5 SLEEP_BETWEEN_RETRIES = 5 CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge" @@ -254,7 +255,7 @@ class ClickhouseIntegrationTestsRunner: def _get_all_tests(self, repo_path): image_cmd = self._get_runner_image_cmd(repo_path) - cmd = "cd {}/tests/integration && ./runner {} ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' > all_tests.txt".format(repo_path, image_cmd) + cmd = "cd {}/tests/integration && ./runner {} ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt".format(repo_path, image_cmd) logging.info("Getting all tests with cmd '%s'", cmd) subprocess.check_call(cmd, shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL @@ -330,7 +331,7 @@ class ClickhouseIntegrationTestsRunner: logging.info("Cannot run with custom docker image version :(") return image_cmd - def run_test_group(self, repo_path, test_group, tests_in_group, num_tries): + def run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): counters = { "ERROR": [], "PASSED": [], @@ -370,8 +371,8 @@ class ClickhouseIntegrationTestsRunner: test_names.add(test_name) test_cmd = ' '.join([test for test in sorted(test_names)]) - cmd = "cd {}/tests/integration && ./runner {} --tmpfs -t {} --parallel 10 '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( - repo_path, image_cmd, test_cmd, _get_deselect_option(self.should_skip_tests()), output_path) + cmd = "cd {}/tests/integration && ./runner {} --tmpfs -t {} --parallel {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( + repo_path, image_cmd, test_cmd, num_workers, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: logging.info("Executing cmd: %s", cmd) @@ -423,7 +424,7 @@ class ClickhouseIntegrationTestsRunner: for i in range(TRIES_COUNT): final_retry += 1 logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.run_test_group(repo_path, "flaky", tests_to_run, 1) + counters, tests_times, log_paths = self.run_test_group(repo_path, "flaky", tests_to_run, 1, 1) logs += log_paths if counters["FAILED"]: logging.info("Found failed tests: %s", ' '.join(counters["FAILED"])) @@ -505,7 +506,7 @@ class ClickhouseIntegrationTestsRunner: for group, tests in items_to_run: logging.info("Running test group %s countaining %s tests", group, len(tests)) - group_counters, group_test_times, log_paths = self.run_test_group(repo_path, group, tests, MAX_RETRY) + group_counters, group_test_times, log_paths = self.run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) total_tests = 0 for counter, value in group_counters.items(): logging.info("Tests from group %s stats, %s count %s", group, counter, len(value)) diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index 143d21f52b5..a01518fcc23 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -1,870 +1,1253 @@ [ + "test_access_control_on_cluster/test.py::test_access_control_on_cluster", + "test_aggregation_memory_efficient/test.py::test_remote", + "test_allowed_client_hosts/test.py::test_allowed_host", + "test_allowed_url_from_config/test.py::test_config_with_hosts", + "test_allowed_url_from_config/test.py::test_config_with_only_primary_hosts", + "test_allowed_url_from_config/test.py::test_config_with_only_regexp_hosts", + "test_allowed_url_from_config/test.py::test_config_without_allowed_hosts", + "test_allowed_url_from_config/test.py::test_config_without_allowed_hosts_section", + "test_allowed_url_from_config/test.py::test_HDFS", + "test_allowed_url_from_config/test.py::test_redirect", + "test_allowed_url_from_config/test.py::test_table_function_remote", + "test_alter_codec/test.py::test_alter_codec_index", + "test_alter_codec/test.py::test_alter_codec_pk", + "test_alter_on_mixed_type_cluster/test.py::test_alter_on_cluter_non_replicated", + "test_alter_on_mixed_type_cluster/test.py::test_alter_replicated_on_cluster", + "test_alter_settings_on_cluster/test.py::test_default_database_on_cluster", + "test_always_fetch_merged/test.py::test_replica_always_download", + "test_asynchronous_metric_log_table/test.py::test_event_time_microseconds_field", + "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", + "test_attach_without_checksums/test.py::test_attach_without_checksums", + "test_attach_without_fetching/test.py::test_attach_without_fetching", + "test_authentication/test.py::test_authentication_fail", + "test_authentication/test.py::test_authentication_pass", + "test_backup_restore/test.py::test_attach_partition", + "test_backup_restore/test.py::test_replace_partition", + "test_backup_restore/test.py::test_restore", + "test_backup_with_other_granularity/test.py::test_backup_and_alter", + "test_backup_with_other_granularity/test.py::test_backup_from_old_version", + "test_backup_with_other_granularity/test.py::test_backup_from_old_version_config", + "test_backup_with_other_granularity/test.py::test_backup_from_old_version_setting", + "test_backward_compatibility/test_aggregate_function_state_avg.py::test_backward_compatability", + "test_backward_compatibility/test.py::test_backward_compatability1", + "test_backward_compatibility/test_short_strings_aggregation.py::test_backward_compatability", + "test_block_structure_mismatch/test.py::test", + "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", + "test_buffer_profile/test.py::test_buffer_profile", + "test_buffer_profile/test.py::test_default_profile", + "test_catboost_model_config_reload/test.py::test", + "test_catboost_model_first_evaluate/test.py::test", + "test_catboost_model_reload/test.py::test_model_reload", + "test_catboost_model_reload/test.py::test_models_reload", + "test_check_table/test.py::test_check_normal_table_corruption", + "test_check_table/test.py::test_check_replicated_table_corruption", + "test_check_table/test.py::test_check_replicated_table_simple", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path", + "test_cluster_all_replicas/test.py::test_remote", + "test_cluster_copier/test.py::test_block_size", + "test_cluster_copier/test.py::test_copy_month_to_week_partition", + "test_cluster_copier/test.py::test_copy_month_to_week_partition_with_recovering", + "test_cluster_copier/test.py::test_copy_month_to_week_partition_with_recovering_after_move_faults", + "test_cluster_copier/test.py::test_copy_simple[False]", + "test_cluster_copier/test.py::test_copy_simple[True]", + "test_cluster_copier/test.py::test_copy_with_recovering_after_move_faults[False]", + "test_cluster_copier/test.py::test_copy_with_recovering_after_move_faults[True]", + "test_cluster_copier/test.py::test_copy_with_recovering[False]", + "test_cluster_copier/test.py::test_copy_with_recovering[True]", + "test_cluster_copier/test.py::test_no_arg", + "test_cluster_copier/test.py::test_no_index", + "test_cluster_copier/test.py::test_non_partitioned_table", + "test_cluster_copier/test.py::test_self_copy", + "test_compression_codec_read/test.py::test_default_codec_read", + "test_compression_nested_columns/test.py::test_nested_compression_codec", + "test_concurrent_queries_for_all_users_restriction/test.py::test_exception_message", + "test_concurrent_queries_for_user_restriction/test.py::test_exception_message", + "test_concurrent_ttl_merges/test.py::test_limited_ttl_merges_in_empty_pool", + "test_concurrent_ttl_merges/test.py::test_limited_ttl_merges_in_empty_pool_replicated", + "test_concurrent_ttl_merges/test.py::test_limited_ttl_merges_two_replicas", + "test_concurrent_ttl_merges/test.py::test_no_ttl_merges_in_busy_pool", + "test_config_corresponding_root/test.py::test_work", + "test_config_substitutions/test.py::test_allow_databases", + "test_config_substitutions/test.py::test_config", + "test_consistant_parts_after_move_partition/test.py::test_consistent_part_after_move_partition", + "test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active", + "test_create_user_and_login/test.py::test_grant_create_user", + "test_create_user_and_login/test.py::test_login", + "test_cross_replication/test.py::test", + "test_custom_settings/test.py::test_custom_settings", + "test_custom_settings/test.py::test_illformed_setting", + "test_ddl_alter_query/test.py::test_alter", + "test_ddl_worker_non_leader/test.py::test_non_leader_replica", + "test_default_compression_codec/test.py::test_default_codec_for_compact_parts", + "test_default_compression_codec/test.py::test_default_codec_multiple", + "test_default_compression_codec/test.py::test_default_codec_single", + "test_default_compression_codec/test.py::test_default_codec_version_update", + "test_default_database_on_cluster/test.py::test_default_database_on_cluster", + "test_default_role/test.py::test_alter_user", + "test_default_role/test.py::test_set_default_roles", + "test_default_role/test.py::test_wrong_set_default_role", + "test_delayed_replica_failover/test.py::test", "test_dictionaries_access/test.py::test_create", - "test_dictionaries_access/test.py::test_create", - "test_dictionaries_access/test.py::test_drop", - "test_dictionaries_access/test.py::test_drop", "test_dictionaries_access/test.py::test_dictget", - "test_dictionaries_access/test.py::test_dictget", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", + "test_dictionaries_access/test.py::test_drop", "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", - "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", - "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", - "test_dictionaries_ddl/test.py::test_restricted_database", - "test_dictionaries_ddl/test.py::test_restricted_database", - "test_dictionaries_ddl/test.py::test_conflicting_name", - "test_dictionaries_ddl/test.py::test_conflicting_name", - "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_dictionary_with_where", - "test_dictionaries_ddl/test.py::test_dictionary_with_where", "test_dictionaries_ddl/test.py::test_clickhouse_remote", - "test_dictionaries_ddl/test.py::test_clickhouse_remote", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", + "test_dictionaries_ddl/test.py::test_conflicting_name", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", + "test_dictionaries_ddl/test.py::test_dictionary_with_where", + "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_restricted_database", "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", - "test_dictionaries_dependency_xml/test.py::test_get_data", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", "test_dictionaries_dependency_xml/test.py::test_get_data", "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", - "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", "test_dictionaries_null_value/test.py::test_null_value", - "test_dictionaries_null_value/test.py::test_null_value", - "test_dictionaries_postgresql/test.py::test_load_dictionaries", - "test_dictionaries_postgresql/test.py::test_load_dictionaries", - "test_dictionaries_postgresql/test.py::test_invalidate_query", - "test_dictionaries_postgresql/test.py::test_invalidate_query", "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", - "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", - "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", + "test_dictionaries_postgresql/test.py::test_invalidate_query", + "test_dictionaries_postgresql/test.py::test_load_dictionaries", + "test_dictionaries_postgresql/test.py::test_postgres_scema", "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", - "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", - "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", - "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", - "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", - "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", + "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", + "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", + "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", - "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", - "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", - "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", - "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", - "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", - "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", - "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", - "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", - "test_dictionary_custom_settings/test.py::test_work", + "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", "test_dictionary_custom_settings/test.py::test_work", "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", - "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", "test_disabled_mysql_server/test.py::test_disabled_mysql_server", - "test_disabled_mysql_server/test.py::test_disabled_mysql_server", - "test_disk_access_storage/test.py::test_create", - "test_disk_access_storage/test.py::test_create", - "test_disk_access_storage/test.py::test_alter", "test_disk_access_storage/test.py::test_alter", + "test_disk_access_storage/test.py::test_create", "test_disk_access_storage/test.py::test_drop", - "test_disk_access_storage/test.py::test_drop", - "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_disk_types/test.py::test_different_types", + "test_disk_types/test.py::test_select_by_type", + "test_distributed_backward_compatability/test.py::test_distributed_in_tuple", + "test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl", + "test_distributed_ddl_on_cross_replication/test.py::test_atomic_database", + "test_distributed_ddl_parallel/test.py::test_all_in_parallel", "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_distributed_ddl_parallel/test.py::test_smoke", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel", "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", + "test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued", + "test_distributed_ddl_password/test.py::test_alter", + "test_distributed_ddl_password/test.py::test_truncate", + "test_distributed_ddl/test.py::test_allowed_databases[configs]", + "test_distributed_ddl/test.py::test_allowed_databases[configs_secure]", + "test_distributed_ddl/test.py::test_create_as_select[configs]", + "test_distributed_ddl/test.py::test_create_as_select[configs_secure]", + "test_distributed_ddl/test.py::test_create_reserved[configs]", + "test_distributed_ddl/test.py::test_create_reserved[configs_secure]", + "test_distributed_ddl/test.py::test_create_view[configs]", + "test_distributed_ddl/test.py::test_create_view[configs_secure]", + "test_distributed_ddl/test.py::test_default_database[configs]", + "test_distributed_ddl/test.py::test_default_database[configs_secure]", + "test_distributed_ddl/test.py::test_detach_query[configs]", + "test_distributed_ddl/test.py::test_detach_query[configs_secure]", + "test_distributed_ddl/test.py::test_implicit_macros[configs]", + "test_distributed_ddl/test.py::test_implicit_macros[configs_secure]", + "test_distributed_ddl/test.py::test_kill_query[configs]", + "test_distributed_ddl/test.py::test_kill_query[configs_secure]", + "test_distributed_ddl/test.py::test_macro[configs]", + "test_distributed_ddl/test.py::test_macro[configs_secure]", + "test_distributed_ddl/test.py::test_on_connection_loss[configs]", + "test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]", + "test_distributed_ddl/test.py::test_on_server_fail[configs]", + "test_distributed_ddl/test.py::test_on_server_fail[configs_secure]", + "test_distributed_ddl/test.py::test_on_session_expired[configs]", + "test_distributed_ddl/test.py::test_on_session_expired[configs_secure]", + "test_distributed_ddl/test.py::test_optimize_query[configs]", + "test_distributed_ddl/test.py::test_optimize_query[configs_secure]", + "test_distributed_ddl/test.py::test_rename[configs]", + "test_distributed_ddl/test.py::test_rename[configs_secure]", + "test_distributed_ddl/test.py::test_replicated_without_arguments[configs]", + "test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]", + "test_distributed_ddl/test.py::test_simple_alters[configs]", + "test_distributed_ddl/test.py::test_simple_alters[configs_secure]", + "test_distributed_ddl/test.py::test_socket_timeout[configs]", + "test_distributed_ddl/test.py::test_socket_timeout[configs_secure]", + "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]", + "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]", + "test_distributed_format/test.py::test_single_file_old[test_cluster]", + "test_distributed_format/test.py::test_single_file_old[test_cluster_2]", + "test_distributed_format/test.py::test_single_file[test_cluster]", + "test_distributed_format/test.py::test_single_file[test_cluster_2]", + "test_distributed_format/test.py::test_two_files[test_cluster]", + "test_distributed_format/test.py::test_two_files[test_cluster_2]", + "test_distributed_inter_server_secret/test.py::test_insecure", + "test_distributed_inter_server_secret/test.py::test_insecure_insert_async", + "test_distributed_inter_server_secret/test.py::test_insecure_insert_sync", + "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_insecure_cluster[default-]", + "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_insecure_cluster[nopass-]", + "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_insecure_cluster[pass-foo]", + "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[default-]", + "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[nopass-]", + "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[pass-foo]", + "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_insecure_cluster[default-]", + "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_insecure_cluster[nopass-]", + "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_insecure_cluster[pass-foo]", + "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[default-]", + "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[nopass-]", + "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[pass-foo]", + "test_distributed_inter_server_secret/test.py::test_secure", + "test_distributed_inter_server_secret/test.py::test_secure_disagree", + "test_distributed_inter_server_secret/test.py::test_secure_disagree_insert", + "test_distributed_inter_server_secret/test.py::test_secure_insert_async", + "test_distributed_inter_server_secret/test.py::test_secure_insert_buffer_async", + "test_distributed_inter_server_secret/test.py::test_secure_insert_sync", + "test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[default-]", + "test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[nopass-]", + "test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[pass-foo]", + "test_distributed_inter_server_secret/test.py::test_user_secure_cluster[default-]", + "test_distributed_inter_server_secret/test.py::test_user_secure_cluster[nopass-]", + "test_distributed_inter_server_secret/test.py::test_user_secure_cluster[pass-foo]", + "test_distributed_load_balancing/test.py::test_distributed_replica_max_ignored_errors", + "test_distributed_load_balancing/test.py::test_load_balancing_default", + "test_distributed_load_balancing/test.py::test_load_balancing_first_or_random", + "test_distributed_load_balancing/test.py::test_load_balancing_in_order", + "test_distributed_load_balancing/test.py::test_load_balancing_nearest_hostname", + "test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority]", + "test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority_negative]", + "test_distributed_load_balancing/test.py::test_load_balancing_round_robin", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[cluster_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[cluster_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[dod_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[dod_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[cluster_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[cluster_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[dod_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[dod_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[cluster_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[cluster_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[dod_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[dod_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[cluster_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[cluster_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[dod_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[dod_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[cluster_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[cluster_node2]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[dod_node1]", + "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[dod_node2]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings21]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings22]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings23]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings24]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings25]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings26]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings27]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings28]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings29]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings30]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings31]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings32]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings33]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings34]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings35]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings36]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings37]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings38]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings39]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings40]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings41]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings0]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings1]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings10]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings11]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings12]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings13]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings14]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings15]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings16]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings17]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings18]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings19]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings2]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings20]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings3]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings4]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings5]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings6]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings7]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings8]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings9]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings63]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings64]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings65]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings66]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings67]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings68]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings69]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings70]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings71]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings72]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings73]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings74]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings75]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings76]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings77]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings78]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings79]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings80]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings81]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings82]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings83]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings42]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings43]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings44]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings45]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings46]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings47]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings48]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings49]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings50]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings51]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings52]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings53]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings54]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings55]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings56]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings57]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings58]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings59]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings60]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings61]", + "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings62]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]", "test_distributed_storage_configuration/test.py::test_insert", - "test_distributed_storage_configuration/test.py::test_insert", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", + "test_distributed_system_query/test.py::test_start_and_stop_replica_send", + "test_drop_replica/test.py::test_drop_replica", + "test_enabling_access_management/test.py::test_enabling_access_management", + "test_extreme_deduplication/test.py::test_deduplication_window_in_seconds", + "test_extreme_deduplication/test.py::test_deduplication_works_in_case_of_intensive_inserts SKIPPED", + "test_fetch_partition_from_auxiliary_zookeeper/test.py::test_fetch_part_from_allowed_zookeeper[PART-2020-08-28-20200828_0_0_0]", + "test_fetch_partition_from_auxiliary_zookeeper/test.py::test_fetch_part_from_allowed_zookeeper[PARTITION-2020-08-27-2020-08-27]", + "test_fetch_partition_should_reset_mutation/test.py::test_part_should_reset_mutation", + "test_fetch_partition_with_outdated_parts/test.py::test_fetch_partition_with_outdated_parts", + "test_filesystem_layout/test.py::test_file_path_escaping", + "test_filesystem_layout/test.py::test_file_path_escaping_atomic_db", + "test_force_deduplication/test.py::test_basic", + "test_force_drop_table/test.py::test_drop_materialized_view[Atomic]", + "test_force_drop_table/test.py::test_drop_materialized_view[Ordinary]", + "test_format_avro_confluent/test.py::test_select", + "test_format_schema_on_server/test.py::test_protobuf_format_input", + "test_format_schema_on_server/test.py::test_protobuf_format_output", + "test_freeze_table/test.py::test_freeze_table", + "test_globs_in_filepath/test.py::test_deep_structure", + "test_globs_in_filepath/test.py::test_linear_structure", + "test_globs_in_filepath/test.py::test_strange_filenames", + "test_globs_in_filepath/test.py::test_table_function_and_virtual_columns", + "test_grant_and_revoke/test.py::test_allowed_grantees", + "test_grant_and_revoke/test.py::test_current_database", + "test_grant_and_revoke/test.py::test_grant_all_on_table", + "test_grant_and_revoke/test.py::test_grant_option", + "test_grant_and_revoke/test.py::test_implicit_create_temporary_table_grant", + "test_grant_and_revoke/test.py::test_implicit_create_view_grant", + "test_grant_and_revoke/test.py::test_implicit_show_grants", + "test_grant_and_revoke/test.py::test_introspection", + "test_grant_and_revoke/test.py::test_revoke_requires_grant_option", + "test_grant_and_revoke/test.py::test_smoke", + "test_graphite_merge_tree/test.py::test_broken_partial_rollup", + "test_graphite_merge_tree/test.py::test_combined_rules", + "test_graphite_merge_tree/test.py::test_combined_rules_with_default", + "test_graphite_merge_tree/test.py::test_multiple_output_blocks", + "test_graphite_merge_tree/test.py::test_multiple_paths_and_versions", + "test_graphite_merge_tree/test.py::test_path_dangling_pointer", + "test_graphite_merge_tree/test.py::test_paths_not_matching_any_pattern", + "test_graphite_merge_tree/test.py::test_rollup_aggregation", + "test_graphite_merge_tree/test.py::test_rollup_aggregation_2", + "test_graphite_merge_tree/test.py::test_rollup_versions", + "test_graphite_merge_tree/test.py::test_system_graphite_retentions", + "test_graphite_merge_tree/test.py::test_wrong_rollup_config", + "test_hedged_requests_parallel/test.py::test_combination1", + "test_hedged_requests_parallel/test.py::test_combination2", + "test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample", + "test_hedged_requests_parallel/test.py::test_send_data", + "test_hedged_requests_parallel/test.py::test_send_table_status_sleep", + "test_hedged_requests/test.py::test_combination1", + "test_hedged_requests/test.py::test_combination2", + "test_hedged_requests/test.py::test_combination3", + "test_hedged_requests/test.py::test_combination4", + "test_hedged_requests/test.py::test_long_query", + "test_hedged_requests/test.py::test_receive_timeout1", + "test_hedged_requests/test.py::test_receive_timeout2", + "test_hedged_requests/test.py::test_send_data", + "test_hedged_requests/test.py::test_send_data2", + "test_hedged_requests/test.py::test_send_table_status_sleep", + "test_hedged_requests/test.py::test_send_table_status_sleep2", + "test_hedged_requests/test.py::test_stuck_replica", + "test_host_ip_change/test.py::test_dns_cache_update", + "test_host_ip_change/test.py::test_ip_change_drop_dns_cache", + "test_host_ip_change/test.py::test_ip_change_update_dns_cache", + "test_host_ip_change/test.py::test_user_access_ip_change[node0]", + "test_host_ip_change/test.py::test_user_access_ip_change[node1]", + "test_http_and_readonly/test.py::test_http_get_is_readonly", + "test_http_handlers_config/test.py::test_absolute_path_static_handler", + "test_http_handlers_config/test.py::test_config_static_handler", + "test_http_handlers_config/test.py::test_defaults_http_handlers", + "test_http_handlers_config/test.py::test_dynamic_query_handler", + "test_http_handlers_config/test.py::test_fixed_static_handler", + "test_http_handlers_config/test.py::test_predefined_query_handler", + "test_http_handlers_config/test.py::test_prometheus_handler", + "test_http_handlers_config/test.py::test_relative_path_static_handler", + "test_http_handlers_config/test.py::test_replicas_status_handler", + "test_https_replication/test.py::test_both_http", + "test_https_replication/test.py::test_both_https", + "test_https_replication/test.py::test_mixed_protocol", + "test_https_replication/test.py::test_replication_after_partition", + "test_inherit_multiple_profiles/test.py::test_combined_profile", + "test_input_format_parallel_parsing_memory_tracking/test.py::test_memory_tracking_total", + "test_insert_distributed_async_extra_dirs/test.py::test_insert_distributed_async_send_success", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_big[0]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_big[1]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_small[0]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_small[1]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_different_header[0]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_different_header[1]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_success[0]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_success[1]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_1[0]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_1[1]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_2[0]", + "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_2[1]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order[internal_replication-0]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order[no_internal_replication-1]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order_sync[internal_replication-0]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order_sync[no_internal_replication-1]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_nearest_hostname[internal_replication-0]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_nearest_hostname[no_internal_replication-1]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1[internal_replication-0]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1_load_balancing_in_order[internal_replication-0]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1_load_balancing_in_order[no_internal_replication-1]", + "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1[no_internal_replication-1]", + "test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout", + "test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout", + "test_insert_into_distributed/test.py::test_inserts_batching", + "test_insert_into_distributed/test.py::test_inserts_local", + "test_insert_into_distributed/test.py::test_inserts_low_cardinality", + "test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication", + "test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication", + "test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication", + "test_insert_into_distributed/test.py::test_prefer_localhost_replica", + "test_insert_into_distributed/test.py::test_reconnect", + "test_insert_into_distributed/test.py::test_table_function", + "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED", + "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", + "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", + "test_jbod_balancer/test.py::test_jbod_balanced_merge", + "test_jbod_balancer/test.py::test_replicated_balanced_merge_fetch", + "test_join_set_family_s3/test.py::test_join_s3", + "test_join_set_family_s3/test.py::test_set_s3", + "test_keeper_back_to_back/test.py::test_concurrent_watches", + "test_keeper_back_to_back/test.py::test_end_of_session", + "test_keeper_back_to_back/test.py::test_end_of_watches_session", + "test_keeper_back_to_back/test.py::test_multitransactions", + "test_keeper_back_to_back/test.py::test_random_requests", + "test_keeper_back_to_back/test.py::test_sequential_nodes", + "test_keeper_back_to_back/test.py::test_simple_commands", + "test_keeper_back_to_back/test.py::test_stats", + "test_keeper_back_to_back/test.py::test_watchers", + "test_keeper_internal_secure/test.py::test_secure_raft_works", + "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", + "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice", + "test_keeper_multinode_simple/test.py::test_follower_restart", + "test_keeper_multinode_simple/test.py::test_read_write_multinode", + "test_keeper_multinode_simple/test.py::test_session_expiration", + "test_keeper_multinode_simple/test.py::test_simple_replicated_table", + "test_keeper_multinode_simple/test.py::test_watch_on_follower", + "test_keeper_persistent_log_multinode/test.py::test_restart_multinode", + "test_keeper_persistent_log/test.py::test_ephemeral_after_restart", + "test_keeper_persistent_log/test.py::test_state_after_restart", + "test_keeper_restore_from_snapshot/test.py::test_recover_from_snapshot", + "test_keeper_secure_client/test.py::test_connection", + "test_keeper_snapshots_multinode/test.py::test_restart_multinode", + "test_keeper_snapshots/test.py::test_ephemeral_after_restart", + "test_keeper_snapshots/test.py::test_state_after_restart", + "test_library_bridge/test.py::test_load_all", + "test_library_bridge/test.py::test_load_all_many_rows", + "test_library_bridge/test.py::test_load_ids", + "test_library_bridge/test.py::test_load_keys", + "test_library_bridge/test.py::test_null_values", + "test_limited_replicated_fetches/test.py::test_limited_fetches", + "test_log_family_s3/test.py::test_log_family_s3[Log-2-1]", + "test_log_family_s3/test.py::test_log_family_s3[StripeLog-1-2]", + "test_log_family_s3/test.py::test_log_family_s3[TinyLog-1-1]", + "test_logs_level/test.py::test_check_client_logs_level", + "test_match_process_uid_against_data_owner/test.py::test_different_user", "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", - "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", - "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", - "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", + "test_max_http_connections_for_replication/test.py::test_keepalive_timeout", + "test_max_http_connections_for_replication/test.py::test_multiple_endpoint_connections_count", + "test_max_http_connections_for_replication/test.py::test_single_endpoint_connections_count", + "test_MemoryTracking/test.py::test_http", + "test_MemoryTracking/test.py::test_tcp_multiple_sessions", + "test_MemoryTracking/test.py::test_tcp_single_session", + "test_merge_table_over_distributed/test.py::test_filtering", + "test_merge_table_over_distributed/test.py::test_global_in", + "test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed", + "test_merge_tree_empty_parts/test.py::test_empty_parts_alter_delete", + "test_merge_tree_empty_parts/test.py::test_empty_parts_summing", + "test_merge_tree_s3_failover/test.py::test_move_failover", + "test_merge_tree_s3_failover/test.py::test_write_failover[0-13]", + "test_merge_tree_s3_failover/test.py::test_write_failover[1048576-9]", + "test_merge_tree_s3_restore/test.py::test_full_restore[False]", + "test_merge_tree_s3_restore/test.py::test_full_restore[True]", + "test_merge_tree_s3_restore/test.py::test_migrate_to_restorable_schema", + "test_merge_tree_s3_restore/test.py::test_restore_another_bucket_path", + "test_merge_tree_s3_restore/test.py::test_restore_different_revisions", + "test_merge_tree_s3_restore/test.py::test_restore_mutations", + "test_merge_tree_s3_restore/test.py::test_restore_to_detached[False]", + "test_merge_tree_s3_restore/test.py::test_restore_to_detached[True]", + "test_merge_tree_s3/test.py::test_alter_table_columns", + "test_merge_tree_s3/test.py::test_attach_detach_partition", + "test_merge_tree_s3/test.py::test_freeze_unfreeze", + "test_merge_tree_s3/test.py::test_insert_same_partition_and_merge[False]", + "test_merge_tree_s3/test.py::test_insert_same_partition_and_merge[True]", + "test_merge_tree_s3/test.py::test_move_partition_to_another_disk", + "test_merge_tree_s3/test.py::test_move_replace_partition_to_another_table", + "test_merge_tree_s3/test.py::test_s3_disk_apply_new_settings", + "test_merge_tree_s3/test.py::test_s3_disk_restart_during_load", + "test_merge_tree_s3/test.py::test_simple_insert_select[0-15]", + "test_merge_tree_s3/test.py::test_simple_insert_select[8192-11]", + "test_merge_tree_s3/test.py::test_table_manipulations", + "test_merge_tree_s3_with_cache/test.py::test_read_after_cache_is_wiped[0-4-2]", + "test_merge_tree_s3_with_cache/test.py::test_read_after_cache_is_wiped[8192-2-1]", + "test_merge_tree_s3_with_cache/test.py::test_write_is_cached[0-2]", + "test_merge_tree_s3_with_cache/test.py::test_write_is_cached[8192-1]", + "test_multiple_disks/test.py::test_alter_double_move_partition[DISK]", + "test_multiple_disks/test.py::test_alter_double_move_partition[VOLUME]", + "test_multiple_disks/test.py::test_alter_move_half_of_partition[DISK]", + "test_multiple_disks/test.py::test_alter_move_half_of_partition[VOLUME]", + "test_multiple_disks/test.py::test_alter_move[mt]", + "test_multiple_disks/test.py::test_alter_policy[mt]", + "test_multiple_disks/test.py::test_alter_policy[replicated]", + "test_multiple_disks/test.py::test_background_move[mt]", + "test_multiple_disks/test.py::test_background_move[replicated]", + "test_multiple_disks/test.py::test_concurrent_alter_modify[mt]", + "test_multiple_disks/test.py::test_concurrent_alter_modify[replicated]", + "test_multiple_disks/test.py::test_concurrent_alter_move_and_drop[mt]", + "test_multiple_disks/test.py::test_concurrent_alter_move_and_drop[replicated]", + "test_multiple_disks/test.py::test_concurrent_alter_move[mt]", + "test_multiple_disks/test.py::test_concurrent_alter_move[replicated]", + "test_multiple_disks/test.py::test_detach_attach[mt]", + "test_multiple_disks/test.py::test_detach_attach[replicated]", + "test_multiple_disks/test.py::test_download_appropriate_disk", + "test_multiple_disks/test.py::test_freeze", + "test_multiple_disks/test.py::test_jbod_overflow[mt]", + "test_multiple_disks/test.py::test_jbod_overflow[replicated]", + "test_multiple_disks/test.py::test_kill_while_insert", + "test_multiple_disks/test.py::test_max_data_part_size[mt]", + "test_multiple_disks/test.py::test_max_data_part_size[replicated]", + "test_multiple_disks/test.py::test_move_across_policies_does_not_work", + "test_multiple_disks/test.py::test_move_while_merge", + "test_multiple_disks/test.py::test_mutate_to_another_disk[mt]", + "test_multiple_disks/test.py::test_mutate_to_another_disk[replicated]", + "test_multiple_disks/test.py::test_no_merges_in_configuration_allow_from_query_without_reload", + "test_multiple_disks/test.py::test_no_merges_in_configuration_allow_from_query_with_reload", + "test_multiple_disks/test.py::test_no_warning_about_zero_max_data_part_size", + "test_multiple_disks/test.py::test_query_parser", + "test_multiple_disks/test.py::test_rename", + "test_multiple_disks/test.py::test_round_robin[mt]", + "test_multiple_disks/test.py::test_round_robin[replicated]", + "test_multiple_disks/test.py::test_simple_merge_tree_merges_are_disabled", + "test_multiple_disks/test.py::test_simple_replication_and_moves", + "test_multiple_disks/test.py::test_start_stop_moves[mt]", + "test_multiple_disks/test.py::test_start_stop_moves[replicated]", + "test_multiple_disks/test.py::test_system_tables", + "test_multiple_disks/test.py::test_yes_merges_in_configuration_disallow_from_query_without_reload", + "test_multiple_disks/test.py::test_yes_merges_in_configuration_disallow_from_query_with_reload", + "test_mutations_hardlinks/test.py::test_delete_and_drop_mutation", + "test_mutations_hardlinks/test.py::test_drop_mutation", + "test_mutations_hardlinks/test.py::test_modify_mutation", + "test_mutations_hardlinks/test.py::test_update_mutation", + "test_mutations_in_partitions_of_merge_tree/test.py::test_trivial_alter_in_partition_merge_tree_without_where", + "test_mutations_in_partitions_of_merge_tree/test.py::test_trivial_alter_in_partition_merge_tree_with_where", + "test_mutations_in_partitions_of_merge_tree/test.py::test_trivial_alter_in_partition_replicated_merge_tree", + "test_mutations_with_merge_tree/test.py::test_mutations_with_merge_background_task", + "test_mutations_with_merge_tree/test.py::test_mutations_with_truncate_table", + "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", - "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", "test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database", - "test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database", - "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", - "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", - "test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine", "test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine", + "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", "test_mysql_database_engine/test.py::test_mysql_types[common_types_1]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_1]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_10]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_10]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_11]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_11]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_12]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_12]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_13]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_13]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_14]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_14]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_15]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_15]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_16]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_16]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_17]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_17]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_18]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_18]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_19]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_19]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", "test_mysql_database_engine/test.py::test_mysql_types[common_types_20]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_20]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", - "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", - "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", - "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", - "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", + "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]", "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]", - "test_mysql_protocol/test.py::test_mysql_client", + "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", + "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", + "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", + "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", + "test_mysql_protocol/test.py::test_golang_client", + "test_mysql_protocol/test.py::test_java_client", + "test_mysql_protocol/test.py::test_mysql_affected_rows", "test_mysql_protocol/test.py::test_mysql_client", "test_mysql_protocol/test.py::test_mysql_client_exception", - "test_mysql_protocol/test.py::test_mysql_client_exception", - "test_mysql_protocol/test.py::test_mysql_affected_rows", - "test_mysql_protocol/test.py::test_mysql_affected_rows", - "test_mysql_protocol/test.py::test_mysql_replacement_query", - "test_mysql_protocol/test.py::test_mysql_replacement_query", - "test_mysql_protocol/test.py::test_mysql_explain", "test_mysql_protocol/test.py::test_mysql_explain", "test_mysql_protocol/test.py::test_mysql_federated", - "test_mysql_protocol/test.py::test_mysql_federated", - "test_mysql_protocol/test.py::test_mysql_set_variables", - "test_mysql_protocol/test.py::test_mysql_set_variables", - "test_mysql_protocol/test.py::test_python_client", - "test_mysql_protocol/test.py::test_python_client", - "test_mysql_protocol/test.py::test_golang_client", - "test_mysql_protocol/test.py::test_golang_client", - "test_mysql_protocol/test.py::test_php_client", - "test_mysql_protocol/test.py::test_php_client", "test_mysql_protocol/test.py::test_mysqljs_client", - "test_mysql_protocol/test.py::test_mysqljs_client", - "test_mysql_protocol/test.py::test_java_client", - "test_mysql_protocol/test.py::test_java_client", + "test_mysql_protocol/test.py::test_mysql_replacement_query", + "test_mysql_protocol/test.py::test_mysql_set_variables", + "test_mysql_protocol/test.py::test_php_client", + "test_mysql_protocol/test.py::test_python_client", "test_mysql_protocol/test.py::test_types", - "test_mysql_protocol/test.py::test_types", - "test_odbc_interaction/test.py::test_mysql_simple_select_works", - "test_odbc_interaction/test.py::test_mysql_simple_select_works", - "test_odbc_interaction/test.py::test_mysql_insert", - "test_odbc_interaction/test.py::test_mysql_insert", - "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", - "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", - "test_odbc_interaction/test.py::test_sqlite_table_function", - "test_odbc_interaction/test.py::test_sqlite_table_function", - "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", - "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", - "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", - "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", - "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", - "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", - "test_odbc_interaction/test.py::test_postgres_insert", - "test_odbc_interaction/test.py::test_postgres_insert", + "test_no_local_metadata_node/test.py::test_table_start_without_metadata", + "test_non_default_compression/test.py::test_preconfigured_custom_codec", + "test_non_default_compression/test.py::test_preconfigured_default_codec", + "test_non_default_compression/test.py::test_uncompressed_cache_custom_codec", + "test_non_default_compression/test.py::test_uncompressed_cache_plus_zstd_codec", "test_odbc_interaction/test.py::test_bridge_dies_with_parent", - "test_odbc_interaction/test.py::test_bridge_dies_with_parent", - "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", - "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", - "test_odbc_interaction/test.py::test_odbc_postgres_conversions", - "test_odbc_interaction/test.py::test_odbc_postgres_conversions", - "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", - "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", - "test_odbc_interaction/test.py::test_many_connections", - "test_odbc_interaction/test.py::test_many_connections", "test_odbc_interaction/test.py::test_concurrent_queries", - "test_odbc_interaction/test.py::test_concurrent_queries", - "test_odbc_interaction/test.py::test_odbc_long_column_names", + "test_odbc_interaction/test.py::test_many_connections", + "test_odbc_interaction/test.py::test_mysql_insert", + "test_odbc_interaction/test.py::test_mysql_simple_select_works", + "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", "test_odbc_interaction/test.py::test_odbc_long_column_names", "test_odbc_interaction/test.py::test_odbc_long_text", - "test_odbc_interaction/test.py::test_odbc_long_text", - "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", - "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", - "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", + "test_odbc_interaction/test.py::test_odbc_postgres_conversions", + "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", + "test_odbc_interaction/test.py::test_postgres_insert", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", + "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", + "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", + "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", + "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", + "test_odbc_interaction/test.py::test_sqlite_table_function", + "test_old_versions/test.py::test_client_is_older_than_server", + "test_old_versions/test.py::test_distributed_query_initiator_is_older_than_shard", + "test_old_versions/test.py::test_server_is_older_than_client", + "test_on_cluster_timeouts/test.py::test_long_query", + "test_optimize_on_insert/test.py::test_empty_parts_optimize", + "test_partition/test.py::test_attach_check_all_parts", + "test_partition/test.py::test_cannot_attach_active_part", + "test_partition/test.py::test_drop_detached_parts", + "test_partition/test.py::test_partition_complex", + "test_partition/test.py::test_partition_simple", + "test_part_log_table/test.py::test_config_with_non_standard_part_log", + "test_part_log_table/test.py::test_config_without_part_log", + "test_part_log_table/test.py::test_config_with_standard_part_log", + "test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper", + "test_part_uuid/test.py::test_part_uuid", + "test_part_uuid/test.py::test_part_uuid_wal", + "test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster", + "test_polymorphic_parts/test.py::test_compact_parts_only", + "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]", + "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]", + "test_polymorphic_parts/test.py::test_in_memory", + "test_polymorphic_parts/test.py::test_in_memory_alters", + "test_polymorphic_parts/test.py::test_in_memory_deduplication", + "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", + "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", + "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", + "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED", + "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED", + "test_polymorphic_parts/test.py::test_polymorphic_parts_index", + "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", + "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", "test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache", - "test_postgresql_protocol/test.py::test_psql_is_ready", - "test_postgresql_protocol/test.py::test_psql_is_ready", - "test_postgresql_protocol/test.py::test_psql_client", - "test_postgresql_protocol/test.py::test_psql_client", - "test_postgresql_protocol/test.py::test_python_client", - "test_postgresql_protocol/test.py::test_python_client", - "test_postgresql_protocol/test.py::test_java_client", + "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", "test_postgresql_protocol/test.py::test_java_client", + "test_postgresql_protocol/test.py::test_psql_client", + "test_postgresql_protocol/test.py::test_psql_is_ready", + "test_postgresql_protocol/test.py::test_python_client", + "test_profile_events_s3/test.py::test_profile_events", + "test_prometheus_endpoint/test.py::test_prometheus_endpoint", + "test_query_deduplication/test.py::test_no_merge_with_deduplication", + "test_query_deduplication/test.py::test_virtual_column", + "test_query_deduplication/test.py::test_with_deduplication", + "test_query_deduplication/test.py::test_without_deduplication", + "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", + "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum", + "test_quorum_inserts/test.py::test_drop_replica_and_achieve_quorum", + "test_quorum_inserts/test.py::test_insert_quorum_with_drop_partition[False]", + "test_quorum_inserts/test.py::test_insert_quorum_with_drop_partition[True]", + "test_quorum_inserts/test.py::test_insert_quorum_with_move_partition[False]", + "test_quorum_inserts/test.py::test_insert_quorum_with_move_partition[True]", + "test_quorum_inserts/test.py::test_insert_quorum_with_ttl", + "test_quorum_inserts/test.py::test_simple_add_replica", + "test_quota/test.py::test_add_remove_interval", + "test_quota/test.py::test_add_remove_quota", + "test_quota/test.py::test_consumption_of_show_clusters", + "test_quota/test.py::test_consumption_of_show_databases", + "test_quota/test.py::test_consumption_of_show_privileges", + "test_quota/test.py::test_consumption_of_show_processlist", + "test_quota/test.py::test_consumption_of_show_tables", + "test_quota/test.py::test_dcl_introspection", + "test_quota/test.py::test_dcl_management", + "test_quota/test.py::test_exceed_quota", + "test_quota/test.py::test_query_inserts", + "test_quota/test.py::test_quota_from_users_xml", + "test_quota/test.py::test_reload_users_xml_by_timer", + "test_quota/test.py::test_simpliest_quota", + "test_quota/test.py::test_tracking_quota", + "test_quota/test.py::test_users_xml_is_readonly", + "test_random_inserts/test.py::test_insert_multithreaded", + "test_random_inserts/test.py::test_random_inserts", "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", - "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", - "test_redirect_url_storage/test.py::test_url_without_redirect", - "test_redirect_url_storage/test.py::test_url_without_redirect", - "test_redirect_url_storage/test.py::test_url_with_globs", + "test_read_temporary_tables_on_failure/test.py::test_different_versions", + "test_recompression_ttl/test.py::test_recompression_multiple_ttls", + "test_recompression_ttl/test.py::test_recompression_replicated", + "test_recompression_ttl/test.py::test_recompression_simple", + "test_recovery_replica/test.py::test_choose_source_replica", + "test_recovery_replica/test.py::test_recovery", + "test_recovery_replica/test.py::test_update_metadata", "test_redirect_url_storage/test.py::test_url_with_globs", "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", - "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", - "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", - "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", - "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", + "test_redirect_url_storage/test.py::test_url_without_redirect", "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", + "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", + "test_relative_filepath/test.py::test_filepath", + "test_reload_auxiliary_zookeepers/test.py::test_reload_auxiliary_zookeepers", + "test_reload_clusters_config/test.py::test_add_cluster", + "test_reload_clusters_config/test.py::test_delete_cluster", + "test_reload_clusters_config/test.py::test_simple_reload", + "test_reload_clusters_config/test.py::test_update_one_cluster", + "test_reloading_settings_from_users_xml/test.py::test_force_reload", + "test_reloading_settings_from_users_xml/test.py::test_reload_on_timeout", + "test_reloading_settings_from_users_xml/test.py::test_unexpected_setting_enum", + "test_reloading_settings_from_users_xml/test.py::test_unexpected_setting_int", + "test_reloading_settings_from_users_xml/test.py::test_unknown_setting_force_reload", + "test_reloading_settings_from_users_xml/test.py::test_unknown_setting_reload_on_timeout", "test_reloading_storage_configuration/test.py::test_add_disk", - "test_reloading_storage_configuration/test.py::test_add_disk", - "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", + "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", "test_reloading_storage_configuration/test.py::test_add_policy", - "test_reloading_storage_configuration/test.py::test_add_policy", - "test_reloading_storage_configuration/test.py::test_new_policy_works", - "test_reloading_storage_configuration/test.py::test_new_policy_works", "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", - "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", - "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", - "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", + "test_reloading_storage_configuration/test.py::test_new_policy_works", "test_reloading_storage_configuration/test.py::test_remove_disk", - "test_reloading_storage_configuration/test.py::test_remove_disk", - "test_reloading_storage_configuration/test.py::test_remove_policy", + "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", "test_reloading_storage_configuration/test.py::test_remove_policy", "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", - "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", - "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", - "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", + "test_reload_max_table_size_to_drop/test.py::test_reload_max_table_size_to_drop", + "test_reload_zookeeper/test.py::test_reload_zookeeper", + "test_remote_prewhere/test.py::test_remote", + "test_rename_column/test.py::test_rename_distributed", + "test_rename_column/test.py::test_rename_distributed_parallel_insert_and_select", + "test_rename_column/test.py::test_rename_parallel", + "test_rename_column/test.py::test_rename_parallel_same_node", + "test_rename_column/test.py::test_rename_with_parallel_insert", + "test_rename_column/test.py::test_rename_with_parallel_merges", + "test_rename_column/test.py::test_rename_with_parallel_moves", + "test_rename_column/test.py::test_rename_with_parallel_select", + "test_rename_column/test.py::test_rename_with_parallel_slow_insert", + "test_rename_column/test.py::test_rename_with_parallel_slow_select", + "test_rename_column/test.py::test_rename_with_parallel_ttl_delete", + "test_rename_column/test.py::test_rename_with_parallel_ttl_move", + "test_replace_partition/test.py::test_drop_failover", + "test_replace_partition/test.py::test_normal_work", + "test_replace_partition/test.py::test_replace_after_replace_failover", + "test_replica_can_become_leader/test.py::test_can_become_leader", + "test_replicated_database/test.py::test_alters_from_different_replicas", + "test_replicated_database/test.py::test_create_replicated_table", + "test_replicated_database/test.py::test_recover_staled_replica", + "test_replicated_database/test.py::test_simple_alter_table[MergeTree]", + "test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]", + "test_replicated_database/test.py::test_startup_without_zk", + "test_replicated_fetches_timeouts/test.py::test_no_stall", + "test_replicated_merge_tree_config/test.py::test_replicated_merge_tree_settings", + "test_replicated_merge_tree_s3/test.py::test_insert_select_replicated[0-15]", + "test_replicated_merge_tree_s3/test.py::test_insert_select_replicated[8192-11]", + "test_replicated_merge_tree_s3_zero_copy/test.py::test_insert_select_replicated[0-15]", + "test_replicated_merge_tree_s3_zero_copy/test.py::test_insert_select_replicated[8192-11]", + "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_create_replicated_merge_tree_with_auxiliary_zookeeper", + "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_create_replicated_merge_tree_with_default_zookeeper", + "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_create_replicated_merge_tree_with_not_exists_auxiliary_zookeeper", + "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_drop_replicated_merge_tree_with_auxiliary_zookeeper", + "test_replicated_mutations/test.py::test_mutations", + "test_replicated_mutations/test.py::test_mutations_dont_prevent_merges[nodes0]", + "test_replicated_mutations/test.py::test_mutations_dont_prevent_merges[nodes1]", + "test_replicated_parse_zk_metadata/test.py::test_replicated_engine_parse_metadata_on_attach", + "test_replicating_constants/test.py::test_different_versions", + "test_replication_credentials/test.py::test_credentials_and_no_credentials", + "test_replication_credentials/test.py::test_different_credentials", + "test_replication_credentials/test.py::test_no_credentials", + "test_replication_credentials/test.py::test_same_credentials", + "test_replication_without_zookeeper/test.py::test_startup_without_zookeeper", + "test_role/test.py::test_admin_option", + "test_role/test.py::test_combine_privileges", + "test_role/test.py::test_create_role", + "test_role/test.py::test_grant_role_to_role", + "test_role/test.py::test_introspection", + "test_role/test.py::test_revoke_requires_admin_option", + "test_row_policy/test.py::test_cannot_trick_row_policy_with_keyword_with", + "test_row_policy/test.py::test_change_of_users_xml_changes_row_policies", + "test_row_policy/test.py::test_dcl_introspection", + "test_row_policy/test.py::test_dcl_management", + "test_row_policy/test.py::test_introspection", + "test_row_policy/test.py::test_join", + "test_row_policy/test.py::test_miscellaneous_engines", + "test_row_policy/test.py::test_policy_from_users_xml_affects_only_user_assigned", + "test_row_policy/test.py::test_reload_users_xml_by_timer", + "test_row_policy/test.py::test_smoke", + "test_row_policy/test.py::test_tags_with_db_and_table_names", + "test_row_policy/test.py::test_throwif_error_in_prewhere_with_same_condition_as_filter", + "test_row_policy/test.py::test_throwif_error_in_where_with_same_condition_as_filter", + "test_row_policy/test.py::test_throwif_in_prewhere_doesnt_expose_restricted_data", + "test_row_policy/test.py::test_throwif_in_where_doesnt_expose_restricted_data", + "test_row_policy/test.py::test_users_xml_is_readonly", + "test_row_policy/test.py::test_with_prewhere", + "test_s3_cluster/test.py::test_count", + "test_s3_cluster/test.py::test_select_all", + "test_s3_cluster/test.py::test_union_all", + "test_s3_cluster/test.py::test_wrong_cluster", + "test_s3_with_https/test.py::test_s3_with_https[s3_secure]", + "test_s3_with_https/test.py::test_s3_with_https[s3_secure_with_proxy]", + "test_s3_with_proxy/test.py::test_s3_with_proxy_list[s3]", + "test_s3_with_proxy/test.py::test_s3_with_proxy_list[s3_with_resolver]", "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", - "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", - "test_storage_hdfs/test.py::test_read_write_storage", - "test_storage_hdfs/test.py::test_read_write_storage", - "test_storage_hdfs/test.py::test_read_write_storage_with_globs", - "test_storage_hdfs/test.py::test_read_write_storage_with_globs", - "test_storage_hdfs/test.py::test_read_write_table", - "test_storage_hdfs/test.py::test_read_write_table", - "test_storage_hdfs/test.py::test_write_table", - "test_storage_hdfs/test.py::test_write_table", - "test_storage_hdfs/test.py::test_bad_hdfs_uri", + "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_replication[s3]", + "test_secure_socket/test.py::test", + "test_select_access_rights/test.py::test_alias", + "test_select_access_rights/test.py::test_alias_columns", + "test_select_access_rights/test.py::test_materialized_columns", + "test_select_access_rights/test.py::test_select_all_columns", + "test_select_access_rights/test.py::test_select_all_columns_with_table_grant", + "test_select_access_rights/test.py::test_select_count", + "test_select_access_rights/test.py::test_select_join", + "test_select_access_rights/test.py::test_select_single_column", + "test_select_access_rights/test.py::test_select_single_column_with_table_grant", + "test_select_access_rights/test.py::test_select_union", + "test_send_crash_reports/test.py::test_send_segfault", + "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case1]", + "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case2]", + "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case3]", + "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case4]", + "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case5]", + "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case6]", + "test_server_initialization/test.py::test_live_view_dependency", + "test_server_initialization/test.py::test_partially_dropped_tables", + "test_server_initialization/test.py::test_sophisticated_default", + "test_settings_constraints_distributed/test.py::test_insert_clamps_settings", + "test_settings_constraints_distributed/test.py::test_select_clamps_settings", + "test_settings_constraints/test.py::test_max_constraint", + "test_settings_constraints/test.py::test_min_constraint", + "test_settings_constraints/test.py::test_read_only_constraint", + "test_settings_constraints/test.py::test_system_constraints", + "test_settings_constraints/test.py::test_system_settings", + "test_settings_profile/test.py::test_allow_ddl", + "test_settings_profile/test.py::test_allow_introspection", + "test_settings_profile/test.py::test_alter_and_drop", + "test_settings_profile/test.py::test_inheritance", + "test_settings_profile/test.py::test_settings_from_granted_role", + "test_settings_profile/test.py::test_show_profiles", + "test_settings_profile/test.py::test_smoke", "test_storage_hdfs/test.py::test_bad_hdfs_uri", "test_storage_hdfs/test.py::test_globs_in_read_table", - "test_storage_hdfs/test.py::test_globs_in_read_table", - "test_storage_hdfs/test.py::test_read_write_gzip_table", - "test_storage_hdfs/test.py::test_read_write_gzip_table", - "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", - "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", - "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", - "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", - "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", - "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", - "test_storage_hdfs/test.py::test_write_gz_storage", - "test_storage_hdfs/test.py::test_write_gz_storage", - "test_storage_hdfs/test.py::test_write_gzip_storage", - "test_storage_hdfs/test.py::test_write_gzip_storage", - "test_storage_hdfs/test.py::test_virtual_columns", - "test_storage_hdfs/test.py::test_virtual_columns", "test_storage_hdfs/test.py::test_read_files_with_spaces", - "test_storage_hdfs/test.py::test_read_files_with_spaces", - "test_storage_kafka/test.py::test_kafka_json_as_string", - "test_storage_kafka/test.py::test_kafka_json_as_string", - "test_storage_kafka/test.py::test_kafka_formats", - "test_storage_kafka/test.py::test_kafka_formats", - "test_storage_kafka/test.py::test_kafka_settings_old_syntax", - "test_storage_kafka/test.py::test_kafka_settings_old_syntax", - "test_storage_kafka/test.py::test_kafka_settings_new_syntax", - "test_storage_kafka/test.py::test_kafka_settings_new_syntax", - "test_storage_kafka/test.py::test_kafka_issue11308", - "test_storage_kafka/test.py::test_kafka_issue11308", - "test_storage_kafka/test.py::test_kafka_issue4116", - "test_storage_kafka/test.py::test_kafka_issue4116", - "test_storage_kafka/test.py::test_kafka_consumer_hang", - "test_storage_kafka/test.py::test_kafka_consumer_hang", - "test_storage_kafka/test.py::test_kafka_consumer_hang2", - "test_storage_kafka/test.py::test_kafka_consumer_hang2", - "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_select_empty", - "test_storage_kafka/test.py::test_kafka_select_empty", - "test_storage_kafka/test.py::test_kafka_json_without_delimiter", - "test_storage_kafka/test.py::test_kafka_json_without_delimiter", - "test_storage_kafka/test.py::test_kafka_protobuf", - "test_storage_kafka/test.py::test_kafka_protobuf", - "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", - "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", - "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", - "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", - "test_storage_kafka/test.py::test_kafka_materialized_view", - "test_storage_kafka/test.py::test_kafka_materialized_view", - "test_storage_kafka/test.py::test_librdkafka_compression", - "test_storage_kafka/test.py::test_librdkafka_compression", - "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", - "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", - "test_storage_kafka/test.py::test_kafka_many_materialized_views", - "test_storage_kafka/test.py::test_kafka_many_materialized_views", - "test_storage_kafka/test.py::test_kafka_flush_on_big_message", - "test_storage_kafka/test.py::test_kafka_flush_on_big_message", - "test_storage_kafka/test.py::test_kafka_virtual_columns", - "test_storage_kafka/test.py::test_kafka_virtual_columns", - "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", - "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", - "test_storage_kafka/test.py::test_kafka_insert", - "test_storage_kafka/test.py::test_kafka_insert", - "test_storage_kafka/test.py::test_kafka_produce_consume", - "test_storage_kafka/test.py::test_kafka_produce_consume", - "test_storage_kafka/test.py::test_kafka_commit_on_block_write", - "test_storage_kafka/test.py::test_kafka_commit_on_block_write", - "test_storage_kafka/test.py::test_kafka_virtual_columns2", - "test_storage_kafka/test.py::test_kafka_virtual_columns2", - "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", - "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", - "test_storage_kafka/test.py::test_kafka_flush_by_time", - "test_storage_kafka/test.py::test_kafka_flush_by_time", - "test_storage_kafka/test.py::test_kafka_flush_by_block_size", - "test_storage_kafka/test.py::test_kafka_flush_by_block_size", - "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", - "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", - "test_storage_kafka/test.py::test_kafka_rebalance", - "test_storage_kafka/test.py::test_kafka_rebalance", - "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", - "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", - "test_storage_kafka/test.py::test_exception_from_destructor", - "test_storage_kafka/test.py::test_exception_from_destructor", - "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", - "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", - "test_storage_kafka/test.py::test_bad_reschedule", - "test_storage_kafka/test.py::test_bad_reschedule", - "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", - "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", - "test_storage_kafka/test.py::test_premature_flush_on_eof", - "test_storage_kafka/test.py::test_premature_flush_on_eof", - "test_storage_kafka/test.py::test_kafka_unavailable", - "test_storage_kafka/test.py::test_kafka_unavailable", - "test_storage_kafka/test.py::test_kafka_issue14202", - "test_storage_kafka/test.py::test_kafka_issue14202", - "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", - "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", - "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", - "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", - "test_storage_kerberized_hdfs/test.py::test_read_table", - "test_storage_kerberized_hdfs/test.py::test_read_table", - "test_storage_kerberized_hdfs/test.py::test_read_write_storage", - "test_storage_kerberized_hdfs/test.py::test_read_write_storage", - "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", - "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", - "test_storage_kerberized_hdfs/test.py::test_two_users", - "test_storage_kerberized_hdfs/test.py::test_two_users", - "test_storage_kerberized_hdfs/test.py::test_read_table_expired", - "test_storage_kerberized_hdfs/test.py::test_read_table_expired", - "test_storage_kerberized_hdfs/test.py::test_prohibited", - "test_storage_kerberized_hdfs/test.py::test_prohibited", + "test_storage_hdfs/test.py::test_read_write_gzip_table", + "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", + "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", + "test_storage_hdfs/test.py::test_read_write_storage", + "test_storage_hdfs/test.py::test_read_write_storage_with_globs", + "test_storage_hdfs/test.py::test_read_write_table", + "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", + "test_storage_hdfs/test.py::test_virtual_columns", + "test_storage_hdfs/test.py::test_write_gzip_storage", + "test_storage_hdfs/test.py::test_write_gz_storage", + "test_storage_hdfs/test.py::test_write_table", "test_storage_kerberized_hdfs/test.py::test_cache_path", - "test_storage_kerberized_hdfs/test.py::test_cache_path", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", - "test_storage_mongodb/test.py::test_simple_select", - "test_storage_mongodb/test.py::test_simple_select", - "test_storage_mongodb/test.py::test_complex_data_type", + "test_storage_kerberized_hdfs/test.py::test_prohibited", + "test_storage_kerberized_hdfs/test.py::test_read_table", + "test_storage_kerberized_hdfs/test.py::test_read_table_expired", + "test_storage_kerberized_hdfs/test.py::test_read_write_storage", + "test_storage_kerberized_hdfs/test.py::test_two_users", + "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", "test_storage_mongodb/test.py::test_complex_data_type", "test_storage_mongodb/test.py::test_incorrect_data_type", - "test_storage_mongodb/test.py::test_incorrect_data_type", - "test_storage_mysql/test.py::test_many_connections", - "test_storage_mysql/test.py::test_many_connections", - "test_storage_mysql/test.py::test_insert_select", - "test_storage_mysql/test.py::test_insert_select", - "test_storage_mysql/test.py::test_replace_select", - "test_storage_mysql/test.py::test_replace_select", - "test_storage_mysql/test.py::test_insert_on_duplicate_select", - "test_storage_mysql/test.py::test_insert_on_duplicate_select", - "test_storage_mysql/test.py::test_where", - "test_storage_mysql/test.py::test_where", - "test_storage_mysql/test.py::test_table_function", - "test_storage_mysql/test.py::test_table_function", - "test_storage_mysql/test.py::test_binary_type", + "test_storage_mongodb/test.py::test_simple_select", "test_storage_mysql/test.py::test_binary_type", "test_storage_mysql/test.py::test_enum_type", - "test_storage_mysql/test.py::test_enum_type", - "test_storage_mysql/test.py::test_mysql_distributed", - "test_storage_mysql/test.py::test_mysql_distributed", "test_storage_mysql/test.py::test_external_settings", - "test_storage_mysql/test.py::test_external_settings", - "test_storage_postgresql/test.py::test_postgres_select_insert", - "test_storage_postgresql/test.py::test_postgres_select_insert", - "test_storage_postgresql/test.py::test_postgres_conversions", - "test_storage_postgresql/test.py::test_postgres_conversions", - "test_storage_postgresql/test.py::test_non_default_scema", - "test_storage_postgresql/test.py::test_non_default_scema", - "test_storage_postgresql/test.py::test_concurrent_queries", + "test_storage_mysql/test.py::test_insert_on_duplicate_select", + "test_storage_mysql/test.py::test_insert_select", + "test_storage_mysql/test.py::test_many_connections", + "test_storage_mysql/test.py::test_mysql_distributed", + "test_storage_mysql/test.py::test_replace_select", + "test_storage_mysql/test.py::test_table_function", + "test_storage_mysql/test.py::test_where", "test_storage_postgresql/test.py::test_concurrent_queries", + "test_storage_postgresql/test.py::test_non_default_scema", + "test_storage_postgresql/test.py::test_postgres_conversions", "test_storage_postgresql/test.py::test_postgres_distributed", - "test_storage_postgresql/test.py::test_postgres_distributed", - "test_storage_rabbitmq/test.py::test_rabbitmq_select", - "test_storage_rabbitmq/test.py::test_rabbitmq_select", - "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", - "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", - "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", - "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", - "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", - "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", + "test_storage_postgresql/test.py::test_postgres_select_insert", "test_storage_rabbitmq/test.py::test_rabbitmq_big_message", - "test_storage_rabbitmq/test.py::test_rabbitmq_big_message", - "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", - "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", - "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", - "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", - "test_storage_rabbitmq/test.py::test_rabbitmq_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", - "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_direct_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", + "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_direct_exchange", "test_storage_rabbitmq/test.py::test_rabbitmq_fanout_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_fanout_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", "test_storage_rabbitmq/test.py::test_rabbitmq_hash_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_hash_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", - "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", "test_storage_rabbitmq/test.py::test_rabbitmq_headers_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_headers_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", - "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", - "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", - "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", + "test_storage_rabbitmq/test.py::test_rabbitmq_insert", + "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_many_consumers_to_each_queue", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_consumers_to_each_queue", - "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", + "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", + "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", + "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", + "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", + "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", + "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", + "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", + "test_storage_rabbitmq/test.py::test_rabbitmq_protobuf SKIPPED (clich...)", "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", - "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", - "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", - "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", - "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", - "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", - "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", - "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", + "test_storage_rabbitmq/test.py::test_rabbitmq_select", + "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", + "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", + "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", + "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", "test_storage_rabbitmq/test.py::test_rabbitmq_vhost", - "test_storage_rabbitmq/test.py::test_rabbitmq_vhost", - "test_storage_s3/test.py::test_put[positive]", - "test_storage_s3/test.py::test_put[positive]", - "test_storage_s3/test.py::test_put[auth_positive]", - "test_storage_s3/test.py::test_put[auth_positive]", - "test_storage_s3/test.py::test_put[auto]", - "test_storage_s3/test.py::test_put[auto]", - "test_storage_s3/test.py::test_put[gzip]", - "test_storage_s3/test.py::test_put[gzip]", - "test_storage_s3/test.py::test_put[deflate]", - "test_storage_s3/test.py::test_put[deflate]", - "test_storage_s3/test.py::test_put[brotli]", - "test_storage_s3/test.py::test_put[brotli]", - "test_storage_s3/test.py::test_put[xz]", - "test_storage_s3/test.py::test_put[xz]", - "test_storage_s3/test.py::test_put[zstd]", - "test_storage_s3/test.py::test_put[zstd]", - "test_storage_s3/test.py::test_get_file_with_special[space]", - "test_storage_s3/test.py::test_get_file_with_special[space]", - "test_storage_s3/test.py::test_get_file_with_special[plus]", - "test_storage_s3/test.py::test_get_file_with_special[plus]", - "test_storage_s3/test.py::test_get_path_with_special[space]", - "test_storage_s3/test.py::test_get_path_with_special[space]", - "test_storage_s3/test.py::test_get_path_with_special[plus]", - "test_storage_s3/test.py::test_get_path_with_special[plus]", - "test_storage_s3/test.py::test_get_path_with_special[plus2]", - "test_storage_s3/test.py::test_get_path_with_special[plus2]", - "test_storage_s3/test.py::test_empty_put[minio]", - "test_storage_s3/test.py::test_empty_put[minio]", - "test_storage_s3/test.py::test_put_csv[positive]", - "test_storage_s3/test.py::test_put_csv[positive]", - "test_storage_s3/test.py::test_put_csv[auth_positive]", - "test_storage_s3/test.py::test_put_csv[auth_positive]", - "test_storage_s3/test.py::test_put_csv[negative]", - "test_storage_s3/test.py::test_put_csv[negative]", - "test_storage_s3/test.py::test_put_get_with_redirect", - "test_storage_s3/test.py::test_put_get_with_redirect", - "test_storage_s3/test.py::test_put_with_zero_redirect", - "test_storage_s3/test.py::test_put_with_zero_redirect", - "test_storage_s3/test.py::test_put_get_with_globs", - "test_storage_s3/test.py::test_put_get_with_globs", - "test_storage_s3/test.py::test_multipart_put[positive]", - "test_storage_s3/test.py::test_multipart_put[positive]", - "test_storage_s3/test.py::test_multipart_put[negative]", - "test_storage_s3/test.py::test_multipart_put[negative]", - "test_storage_s3/test.py::test_remote_host_filter", - "test_storage_s3/test.py::test_remote_host_filter", - "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", - "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", - "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", - "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", - "test_storage_s3/test.py::test_s3_glob_scheherazade", - "test_storage_s3/test.py::test_s3_glob_scheherazade", - "test_storage_s3/test.py::test_custom_auth_headers", + "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", + "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", "test_storage_s3/test.py::test_custom_auth_headers", "test_storage_s3/test.py::test_custom_auth_headers_exclusion", - "test_storage_s3/test.py::test_custom_auth_headers_exclusion", + "test_storage_s3/test.py::test_empty_put[minio]", + "test_storage_s3/test.py::test_get_file_with_special[plus]", + "test_storage_s3/test.py::test_get_file_with_special[space]", + "test_storage_s3/test.py::test_get_path_with_special[plus]", + "test_storage_s3/test.py::test_get_path_with_special[plus2]", + "test_storage_s3/test.py::test_get_path_with_special[space]", "test_storage_s3/test.py::test_infinite_redirect", - "test_storage_s3/test.py::test_infinite_redirect", - "test_storage_s3/test.py::test_storage_s3_get_gzip[bin]", + "test_storage_s3/test.py::test_multipart_put[negative]", + "test_storage_s3/test.py::test_multipart_put[positive]", + "test_storage_s3/test.py::test_put[auth_positive]", + "test_storage_s3/test.py::test_put[auto]", + "test_storage_s3/test.py::test_put[brotli]", + "test_storage_s3/test.py::test_put_csv[auth_positive]", + "test_storage_s3/test.py::test_put_csv[negative]", + "test_storage_s3/test.py::test_put_csv[positive]", + "test_storage_s3/test.py::test_put[deflate]", + "test_storage_s3/test.py::test_put_get_with_globs", + "test_storage_s3/test.py::test_put_get_with_redirect", + "test_storage_s3/test.py::test_put[gzip]", + "test_storage_s3/test.py::test_put[positive]", + "test_storage_s3/test.py::test_put_with_zero_redirect", + "test_storage_s3/test.py::test_put[xz]", + "test_storage_s3/test.py::test_put[zstd]", + "test_storage_s3/test.py::test_remote_host_filter", + "test_storage_s3/test.py::test_s3_glob_scheherazade", "test_storage_s3/test.py::test_storage_s3_get_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_get_gzip[gz]", - "test_storage_s3/test.py::test_storage_s3_get_gzip[gz]", "test_storage_s3/test.py::test_storage_s3_get_unstable", - "test_storage_s3/test.py::test_storage_s3_get_unstable", - "test_storage_s3/test.py::test_storage_s3_put_uncompressed", - "test_storage_s3/test.py::test_storage_s3_put_uncompressed", - "test_storage_s3/test.py::test_storage_s3_put_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_put_gzip[bin]", "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", - "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", - "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", + "test_storage_s3/test.py::test_storage_s3_put_uncompressed", + "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", + "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", + "test_system_clusters_actual_information/test.py::test", + "test_system_ddl_worker_queue/test.py::test_distributed_ddl_queue", + "test_system_flush_logs/test.py::test_system_logs_non_empty_queue", + "test_system_flush_logs/test.py::test_system_logs[system.metric_log-1]", + "test_system_flush_logs/test.py::test_system_logs[system.part_log-0]", + "test_system_flush_logs/test.py::test_system_logs[system.query_log-1]", + "test_system_flush_logs/test.py::test_system_logs[system.query_thread_log-1]", + "test_system_flush_logs/test.py::test_system_logs[system.text_log-0]", + "test_system_flush_logs/test.py::test_system_logs[system.trace_log-1]", + "test_system_merges/test.py::test_merge_simple[]", + "test_system_merges/test.py::test_merge_simple[replicated]", + "test_system_merges/test.py::test_mutation_simple[]", + "test_system_merges/test.py::test_mutation_simple[replicated]", + "test_system_metrics/test.py::test_readonly_metrics", + "test_system_queries/test.py::test_DROP_DNS_CACHE", + "test_system_queries/test.py::test_RELOAD_CONFIG_AND_MACROS", + "test_system_queries/test.py::test_system_flush_logs", + "test_system_queries/test.py::test_SYSTEM_RELOAD_DICTIONARY", + "test_system_replicated_fetches/test.py::test_system_replicated_fetches", + "test_table_functions_access_rights/test.py::test_merge", + "test_text_log_level/test.py::test_basic", + "test_timezone_config/test.py::test_check_timezone_config", + "test_tmp_policy/test.py::test_different_versions", + "test_ttl_move/test.py::test_alter_multiple_ttls[negative]", + "test_ttl_move/test.py::test_alter_multiple_ttls[positive]", + "test_ttl_move/test.py::test_alter_multiple_ttls[replicated_negative]", + "test_ttl_move/test.py::test_alter_multiple_ttls[replicated_positive]", + "test_ttl_move/test.py::test_alter_with_merge_work[mt]", + "test_ttl_move/test.py::test_alter_with_merge_work[mt_work]", + "test_ttl_move/test.py::test_alter_with_merge_work[replicated]", + "test_ttl_move/test.py::test_alter_with_merge_work[replicated_work]", + "test_ttl_move/test.py::test_concurrent_alter_with_ttl_move[mt]", + "test_ttl_move/test.py::test_concurrent_alter_with_ttl_move[replicated_mt]", + "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[disk]", + "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[replicated_disk]", + "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[replicated_volume]", + "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[volume]", + "test_ttl_move/test.py::test_double_move_while_select[negative] SKIPPED", + "test_ttl_move/test.py::test_double_move_while_select[positive] SKIPPED", + "test_ttl_move/test.py::test_inserts_to_disk_work[mt_test_inserts_to_disk_do_not_work]", + "test_ttl_move/test.py::test_inserts_to_disk_work[mt_test_inserts_to_disk_work_1]", + "test_ttl_move/test.py::test_inserts_to_disk_work[replicated_mt_test_inserts_to_disk_do_not_work]", + "test_ttl_move/test.py::test_inserts_to_disk_work[replicated_mt_test_inserts_to_disk_work_1]", + "test_ttl_move/test.py::test_inserts_to_volume_work[mt_test_inserts_to_volume_do_not_work]", + "test_ttl_move/test.py::test_inserts_to_volume_work[mt_test_inserts_to_volume_work]", + "test_ttl_move/test.py::test_inserts_to_volume_work[replicated_mt_test_inserts_to_volume_do_not_work]", + "test_ttl_move/test.py::test_inserts_to_volume_work[replicated_mt_test_inserts_to_volume_work]", + "test_ttl_move/test.py::test_materialize_ttl_in_partition[mt]", + "test_ttl_move/test.py::test_materialize_ttl_in_partition[replicated]", + "test_ttl_move/test.py::test_merges_to_disk_work[mt_test_merges_to_disk_do_not_work0]", + "test_ttl_move/test.py::test_merges_to_disk_work[mt_test_merges_to_disk_do_not_work1]", + "test_ttl_move/test.py::test_merges_to_disk_work[mt_test_merges_to_disk_work]", + "test_ttl_move/test.py::test_merges_to_disk_work[replicated_mt_test_merges_to_disk_work]", + "test_ttl_move/test.py::test_merges_with_full_disk_work[mt_test_merges_with_full_disk_work]", + "test_ttl_move/test.py::test_merges_with_full_disk_work[replicated_mt_test_merges_with_full_disk_work]", + "test_ttl_move/test.py::test_moves_after_merges_work[mt_test_moves_after_merges_do_not_work]", + "test_ttl_move/test.py::test_moves_after_merges_work[mt_test_moves_after_merges_work]", + "test_ttl_move/test.py::test_moves_after_merges_work[replicated_mt_test_moves_after_merges_do_not_work]", + "test_ttl_move/test.py::test_moves_after_merges_work[replicated_mt_test_moves_after_merges_work]", + "test_ttl_move/test.py::test_moves_to_disk_eventually_work[mt_test_moves_to_disk_eventually_work]", + "test_ttl_move/test.py::test_moves_to_disk_eventually_work[replicated_mt_test_moves_to_disk_eventually_work]", + "test_ttl_move/test.py::test_moves_to_disk_work[mt_test_moves_to_disk_do_not_work]", + "test_ttl_move/test.py::test_moves_to_disk_work[mt_test_moves_to_disk_work]", + "test_ttl_move/test.py::test_moves_to_disk_work[replicated_mt_test_moves_to_disk_do_not_work]", + "test_ttl_move/test.py::test_moves_to_disk_work[replicated_mt_test_moves_to_disk_work]", + "test_ttl_move/test.py::test_moves_to_volume_work[mt_test_moves_to_volume_work]", + "test_ttl_move/test.py::test_moves_to_volume_work[replicated_mt_test_moves_to_volume_work]", "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]", - "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]" + "test_ttl_move/test.py::test_replicated_download_ttl_info", + "test_ttl_move/test.py::test_rule_with_invalid_destination[case0]", + "test_ttl_move/test.py::test_rule_with_invalid_destination[case1]", + "test_ttl_move/test.py::test_rule_with_invalid_destination[case2]", + "test_ttl_move/test.py::test_rule_with_invalid_destination[case3]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_external_negative]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_external_positive]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_negative]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_positive]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[repicated_negative]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[repicated_positive]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[replicated_external_negative]", + "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[replicated_external_positive]", + "test_ttl_replicated/test.py::test_merge_with_ttl_timeout", + "test_ttl_replicated/test.py::test_modify_column_ttl", + "test_ttl_replicated/test.py::test_modify_ttl", + "test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete]", + "test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete_replicated]", + "test_ttl_replicated/test.py::test_ttl_columns", + "test_ttl_replicated/test.py::test_ttl_compatibility[node_left0-node_right0-0]", + "test_ttl_replicated/test.py::test_ttl_compatibility[node_left1-node_right1-1]", + "test_ttl_replicated/test.py::test_ttl_compatibility[node_left2-node_right2-2]", + "test_ttl_replicated/test.py::test_ttl_double_delete_rule_returns_error", + "test_ttl_replicated/test.py::test_ttl_empty_parts", + "test_ttl_replicated/test.py::test_ttl_many_columns", + "test_ttl_replicated/test.py::test_ttl_table[]", + "test_ttl_replicated/test.py::test_ttl_table[DELETE]", + "test_union_header/test.py::test_read", + "test_user_directories/test.py::test_duplicates", + "test_user_directories/test.py::test_local_directories", + "test_user_directories/test.py::test_memory", + "test_user_directories/test.py::test_mixed_style", + "test_user_directories/test.py::test_old_style", + "test_user_directories/test.py::test_relative_path", + "test_user_ip_restrictions/test.py::test_ipv4", + "test_user_ip_restrictions/test.py::test_ipv6", + "test_user_zero_database_access/test_user_zero_database_access.py::test_user_zero_database_access", + "test_version_update_after_mutation/test.py::test_mutate_and_upgrade", + "test_version_update_after_mutation/test.py::test_upgrade_while_mutation", + "test_zookeeper_config/test.py::test_chroot_with_different_root", + "test_zookeeper_config/test.py::test_chroot_with_same_root", + "test_zookeeper_config/test.py::test_identity", + "test_zookeeper_config/test.py::test_secure_connection" ] diff --git a/tests/integration/parallel.readme b/tests/integration/parallel.readme index 3a8d55cf4cd..7eef2014524 100644 --- a/tests/integration/parallel.readme +++ b/tests/integration/parallel.readme @@ -1,6 +1,6 @@ # Parallel tests are currently generated manually. All tests except those in parallel.txt will be run sequentially. # Current list is generated with following commands # 1. Generate all tests list as in CI run -./runner ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' > all_tests.txt +./runner ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt # 2. Filter known tests that can be run in parallel -cat all_tests.txt | grep 'dict\|storage\|postgres\|mysql' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel.json +cat all_tests.txt | grep -v 'grpc\|test_adaptive_granularity\|test_adaptive_granularity\|test_adaptive_granularity\|kafka' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel.json From cfb55c2cf694201b7479796433c6d07765c74e83 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 May 2021 19:38:40 +0300 Subject: [PATCH 131/652] fix --- tests/integration/ci-runner.py | 5 +++-- tests/integration/parallel.json | 25 ------------------------- tests/integration/parallel.readme | 2 +- 3 files changed, 4 insertions(+), 28 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 3b4e9568116..4e2b361dcc9 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -371,8 +371,9 @@ class ClickhouseIntegrationTestsRunner: test_names.add(test_name) test_cmd = ' '.join([test for test in sorted(test_names)]) - cmd = "cd {}/tests/integration && ./runner {} --tmpfs -t {} --parallel {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( - repo_path, image_cmd, test_cmd, num_workers, _get_deselect_option(self.should_skip_tests()), output_path) + parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" + cmd = "cd {}/tests/integration && ./runner {} --tmpfs -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( + repo_path, image_cmd, test_cmd, parallel_cmd, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: logging.info("Executing cmd: %s", cmd) diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index a01518fcc23..d04cdd4708f 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -25,10 +25,6 @@ "test_backup_restore/test.py::test_attach_partition", "test_backup_restore/test.py::test_replace_partition", "test_backup_restore/test.py::test_restore", - "test_backup_with_other_granularity/test.py::test_backup_and_alter", - "test_backup_with_other_granularity/test.py::test_backup_from_old_version", - "test_backup_with_other_granularity/test.py::test_backup_from_old_version_config", - "test_backup_with_other_granularity/test.py::test_backup_from_old_version_setting", "test_backward_compatibility/test_aggregate_function_state_avg.py::test_backward_compatability", "test_backward_compatibility/test.py::test_backward_compatability1", "test_backward_compatibility/test_short_strings_aggregation.py::test_backward_compatability", @@ -440,7 +436,6 @@ "test_drop_replica/test.py::test_drop_replica", "test_enabling_access_management/test.py::test_enabling_access_management", "test_extreme_deduplication/test.py::test_deduplication_window_in_seconds", - "test_extreme_deduplication/test.py::test_deduplication_works_in_case_of_intensive_inserts SKIPPED", "test_fetch_partition_from_auxiliary_zookeeper/test.py::test_fetch_part_from_allowed_zookeeper[PART-2020-08-28-20200828_0_0_0]", "test_fetch_partition_from_auxiliary_zookeeper/test.py::test_fetch_part_from_allowed_zookeeper[PARTITION-2020-08-27-2020-08-27]", "test_fetch_partition_should_reset_mutation/test.py::test_part_should_reset_mutation", @@ -555,7 +550,6 @@ "test_insert_into_distributed/test.py::test_prefer_localhost_replica", "test_insert_into_distributed/test.py::test_reconnect", "test_insert_into_distributed/test.py::test_table_function", - "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED", "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", "test_jbod_balancer/test.py::test_jbod_balanced_merge", @@ -839,8 +833,6 @@ "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED", - "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED", "test_polymorphic_parts/test.py::test_polymorphic_parts_index", "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", @@ -1092,7 +1084,6 @@ "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_protobuf SKIPPED (clich...)", "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", "test_storage_rabbitmq/test.py::test_rabbitmq_select", @@ -1175,8 +1166,6 @@ "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[replicated_disk]", "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[replicated_volume]", "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[volume]", - "test_ttl_move/test.py::test_double_move_while_select[negative] SKIPPED", - "test_ttl_move/test.py::test_double_move_while_select[positive] SKIPPED", "test_ttl_move/test.py::test_inserts_to_disk_work[mt_test_inserts_to_disk_do_not_work]", "test_ttl_move/test.py::test_inserts_to_disk_work[mt_test_inserts_to_disk_work_1]", "test_ttl_move/test.py::test_inserts_to_disk_work[replicated_mt_test_inserts_to_disk_do_not_work]", @@ -1220,20 +1209,6 @@ "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[repicated_positive]", "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[replicated_external_negative]", "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[replicated_external_positive]", - "test_ttl_replicated/test.py::test_merge_with_ttl_timeout", - "test_ttl_replicated/test.py::test_modify_column_ttl", - "test_ttl_replicated/test.py::test_modify_ttl", - "test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete]", - "test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete_replicated]", - "test_ttl_replicated/test.py::test_ttl_columns", - "test_ttl_replicated/test.py::test_ttl_compatibility[node_left0-node_right0-0]", - "test_ttl_replicated/test.py::test_ttl_compatibility[node_left1-node_right1-1]", - "test_ttl_replicated/test.py::test_ttl_compatibility[node_left2-node_right2-2]", - "test_ttl_replicated/test.py::test_ttl_double_delete_rule_returns_error", - "test_ttl_replicated/test.py::test_ttl_empty_parts", - "test_ttl_replicated/test.py::test_ttl_many_columns", - "test_ttl_replicated/test.py::test_ttl_table[]", - "test_ttl_replicated/test.py::test_ttl_table[DELETE]", "test_union_header/test.py::test_read", "test_user_directories/test.py::test_duplicates", "test_user_directories/test.py::test_local_directories", diff --git a/tests/integration/parallel.readme b/tests/integration/parallel.readme index 7eef2014524..3b987c689f4 100644 --- a/tests/integration/parallel.readme +++ b/tests/integration/parallel.readme @@ -3,4 +3,4 @@ # 1. Generate all tests list as in CI run ./runner ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt # 2. Filter known tests that can be run in parallel -cat all_tests.txt | grep -v 'grpc\|test_adaptive_granularity\|test_adaptive_granularity\|test_adaptive_granularity\|kafka' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel.json +cat all_tests.txt | grep -v 'grpc\|SKIP\|test_backup_with_other_granularity\|test_ttl_replicated\|test_adaptive_granularity\|kafka' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel.json From bdf03387abba19e14d996a176233dc0fbdb52c80 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 22 Apr 2021 04:25:40 +0300 Subject: [PATCH 132/652] Better S3 read retries. Renamed `s3_max_single_read_retries` -> `s3_single_read_retry_attempts`, added sleeps before next attempts. --- src/Core/Settings.h | 2 +- src/Disks/S3/DiskS3.cpp | 18 ++++++++-------- src/Disks/S3/DiskS3.h | 4 ++-- src/Disks/S3/registerDiskS3.cpp | 2 +- src/IO/ReadBufferFromS3.cpp | 21 ++++++++++++------- src/IO/ReadBufferFromS3.h | 8 +++++-- src/IO/S3Common.cpp | 2 -- src/Storages/StorageS3.cpp | 17 ++++++++------- src/Storages/StorageS3.h | 8 +++---- src/TableFunctions/TableFunctionS3.cpp | 7 +++++-- src/TableFunctions/TableFunctionS3Cluster.cpp | 7 +++++-- 11 files changed, 56 insertions(+), 40 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ea2019a4ff1..9af6ba0f3cd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -70,7 +70,7 @@ class IColumn; M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_single_part_upload_size, 64*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ - M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ + M(UInt64, s3_single_read_retry_attempts, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 325e887073c..d1146c0774c 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -277,12 +277,12 @@ public: std::shared_ptr client_ptr_, const String & bucket_, DiskS3::Metadata metadata_, - size_t s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, size_t buf_size_) : client_ptr(std::move(client_ptr_)) , bucket(bucket_) , metadata(std::move(metadata_)) - , s3_max_single_read_retries(s3_max_single_read_retries_) + , single_read_retry_strategy(std::move(single_read_retry_strategy_)) , buf_size(buf_size_) { } @@ -339,7 +339,7 @@ private: const auto & [path, size] = metadata.s3_objects[i]; if (size > offset) { - auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); + auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, single_read_retry_strategy, buf_size); buf->seek(offset, SEEK_SET); return buf; } @@ -368,7 +368,7 @@ private: ++current_buf_idx; const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); + current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, single_read_retry_strategy, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); absolute_position += working_buffer.size(); @@ -379,7 +379,7 @@ private: std::shared_ptr client_ptr; const String & bucket; DiskS3::Metadata metadata; - size_t s3_max_single_read_retries; + std::shared_ptr single_read_retry_strategy; size_t buf_size; size_t absolute_position = 0; @@ -686,7 +686,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si LOG_DEBUG(log, "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.s3_objects.size()); - auto reader = std::make_unique(settings->client, bucket, metadata, settings->s3_max_single_read_retries, buf_size); + auto reader = std::make_unique(settings->client, bucket, metadata, settings->single_read_retry_strategy, buf_size); return std::make_unique(std::move(reader), settings->min_bytes_for_seek); } @@ -1000,7 +1000,7 @@ int DiskS3::readSchemaVersion(const String & source_bucket, const String & sourc settings->client, source_bucket, source_path + SCHEMA_VERSION_OBJECT, - settings->s3_max_single_read_retries); + settings->single_read_retry_strategy); readIntText(version, buffer); @@ -1559,7 +1559,7 @@ void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config, DiskS3Settings::DiskS3Settings( const std::shared_ptr & client_, - size_t s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, size_t s3_min_upload_part_size_, size_t s3_max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -1567,7 +1567,7 @@ DiskS3Settings::DiskS3Settings( int thread_pool_size_, int list_object_keys_size_) : client(client_) - , s3_max_single_read_retries(s3_max_single_read_retries_) + , single_read_retry_strategy(single_read_retry_strategy_) , s3_min_upload_part_size(s3_min_upload_part_size_) , s3_max_single_part_upload_size(s3_max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index efc7fdcf643..91113f57441 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -22,7 +22,7 @@ struct DiskS3Settings { DiskS3Settings( const std::shared_ptr & client_, - size_t s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, size_t s3_min_upload_part_size_, size_t s3_max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -31,7 +31,7 @@ struct DiskS3Settings int list_object_keys_size_); std::shared_ptr client; - size_t s3_max_single_read_retries; + std::shared_ptr single_read_retry_strategy; size_t s3_min_upload_part_size; size_t s3_max_single_part_upload_size; size_t min_bytes_for_seek; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 767e8890a01..6dcdba25478 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -150,7 +150,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigurat { return std::make_unique( getClient(config, config_prefix, context), - config.getUInt64(config_prefix + ".s3_max_single_read_retries", context->getSettingsRef().s3_max_single_read_retries), + std::make_shared(config.getUInt(config_prefix + ".single_read_retry_attempts", context->getSettingsRef().s3_single_read_retry_attempts)), config.getUInt64(config_prefix + ".s3_min_upload_part_size", context->getSettingsRef().s3_min_upload_part_size), config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size), config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 1e27b0284b7..dde4564ea53 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -6,6 +6,7 @@ # include # include +# include # include # include # include @@ -31,12 +32,12 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, const String & key_, UInt64 s3_max_single_read_retries_, size_t buffer_size_) + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, std::shared_ptr retry_strategy_, size_t buffer_size_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , key(key_) - , s3_max_single_read_retries(s3_max_single_read_retries_) + , retry_strategy(std::move(retry_strategy_)) , buffer_size(buffer_size_) { } @@ -52,11 +53,10 @@ bool ReadBufferFromS3::nextImpl() Stopwatch watch; bool next_result = false; - for (Int64 attempt = static_cast(s3_max_single_read_retries); attempt >= 0; --attempt) - { - if (!impl) - impl = initialize(); + Aws::Client::AWSError network_error; + for (int attempt = 1;; ++attempt) + { try { next_result = impl->next(); @@ -68,6 +68,8 @@ bool ReadBufferFromS3::nextImpl() } catch (const Exception & e) { + network_error = Aws::Client::AWSError(Aws::Client::CoreErrors::NETWORK_CONNECTION, e.name(), e.message(), true); + ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1); LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Remaining attempts: {}, Message: {}", @@ -75,9 +77,14 @@ bool ReadBufferFromS3::nextImpl() impl.reset(); - if (!attempt) + if (!retry_strategy->ShouldRetry(network_error, attempt)) throw; } + + std::this_thread::sleep_for(std::chrono::milliseconds(retry_strategy->CalculateDelayBeforeNextRetry(network_error, attempt))); + + if (!impl) + impl = initialize(); } watch.stop(); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 1f4124d909f..c84c20da7cd 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -15,6 +15,10 @@ namespace Aws::S3 { class S3Client; } +namespace Aws::Client +{ +class RetryStrategy; +} namespace DB { @@ -27,7 +31,7 @@ private: std::shared_ptr client_ptr; String bucket; String key; - UInt64 s3_max_single_read_retries; + std::shared_ptr retry_strategy; size_t buffer_size; off_t offset = 0; Aws::S3::Model::GetObjectResult read_result; @@ -40,7 +44,7 @@ public: std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - UInt64 s3_max_single_read_retries_, + std::shared_ptr retry_strategy_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); bool nextImpl() override; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index ff9eafc9bc7..e8406134f87 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -428,8 +428,6 @@ public: /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. aws_client_configuration.connectTimeoutMs = 1000; - - /// FIXME. Somehow this timeout does not work in docker without --net=host. aws_client_configuration.requestTimeoutMs = 1000; aws_client_configuration.retryStrategy = std::make_shared(1, 1000); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 36d078e7bf3..a4b7b541e15 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -25,6 +25,7 @@ #include #include +#include #include #include @@ -167,7 +168,7 @@ StorageS3Source::StorageS3Source( ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, - UInt64 s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, const String compression_hint_, const std::shared_ptr & client_, const String & bucket_, @@ -179,7 +180,7 @@ StorageS3Source::StorageS3Source( , format(format_) , columns_desc(columns_) , max_block_size(max_block_size_) - , s3_max_single_read_retries(s3_max_single_read_retries_) + , single_read_retry_strategy(std::move(single_read_retry_strategy_)) , compression_hint(compression_hint_) , client(client_) , sample_block(sample_block_) @@ -200,7 +201,7 @@ bool StorageS3Source::initialize() file_path = bucket + "/" + current_key; read_buf = wrapReadBufferWithCompressionMethod( - std::make_unique(client, bucket, current_key, s3_max_single_read_retries), chooseCompressionMethod(current_key, compression_hint)); + std::make_unique(client, bucket, current_key, single_read_retry_strategy), chooseCompressionMethod(current_key, compression_hint)); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size); reader = std::make_shared(input_format); @@ -324,7 +325,7 @@ StorageS3::StorageS3( const String & secret_access_key_, const StorageID & table_id_, const String & format_name_, - UInt64 s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -337,7 +338,7 @@ StorageS3::StorageS3( : IStorage(table_id_) , client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later , format_name(format_name_) - , s3_max_single_read_retries(s3_max_single_read_retries_) + , single_read_retry_strategy(std::move(single_read_retry_strategy_)) , min_upload_part_size(min_upload_part_size_) , max_single_part_upload_size(max_single_part_upload_size_) , compression_method(compression_method_) @@ -405,7 +406,7 @@ Pipe StorageS3::read( local_context, metadata_snapshot->getColumns(), max_block_size, - s3_max_single_read_retries, + single_read_retry_strategy, compression_method, client_auth.client, client_auth.uri.bucket, @@ -492,7 +493,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) secret_access_key = engine_args[2]->as().value.safeGet(); } - UInt64 s3_max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries; + auto single_read_retry_strategy = std::make_shared(args.getLocalContext()->getSettingsRef().s3_single_read_retry_attempts); UInt64 min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections; @@ -516,7 +517,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) secret_access_key, args.table_id, format_name, - s3_max_single_read_retries, + single_read_retry_strategy, min_upload_part_size, max_single_part_upload_size, max_connections, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 694dec4c0dc..7ab6d2fe259 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -55,7 +55,7 @@ public: ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, - UInt64 s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, const String compression_hint_, const std::shared_ptr & client_, const String & bucket, @@ -72,7 +72,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; - UInt64 s3_max_single_read_retries; + std::shared_ptr single_read_retry_strategy; String compression_hint; std::shared_ptr client; Block sample_block; @@ -103,7 +103,7 @@ public: const String & secret_access_key, const StorageID & table_id_, const String & format_name_, - UInt64 s3_max_single_read_retries_, + std::shared_ptr single_read_retry_strategy_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -150,7 +150,7 @@ private: ClientAuthentificaiton client_auth; String format_name; - UInt64 s3_max_single_read_retries; + std::shared_ptr single_read_retry_strategy; size_t min_upload_part_size; size_t max_single_part_upload_size; String compression_method; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 1eddbf69b7c..1e7b0588e8c 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -12,6 +12,9 @@ #include #include "registerTableFunctions.h" +#include + + namespace DB { @@ -83,7 +86,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context { Poco::URI uri (filename); S3::URI s3_uri (uri); - UInt64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; + auto single_read_retry_strategy = std::make_shared(context->getSettingsRef().s3_single_read_retry_attempts); UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; @@ -94,7 +97,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context secret_access_key, StorageID(getDatabaseName(), table_name), format, - s3_max_single_read_retries, + single_read_retry_strategy, min_upload_part_size, max_single_part_upload_size, max_connections, diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 7b0d50a6a0a..f07116736a4 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -28,6 +28,9 @@ #include #include +#include + + namespace DB { @@ -109,7 +112,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( Poco::URI uri (filename); S3::URI s3_uri (uri); /// Actually this parameters are not used - UInt64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; + auto s3_max_single_read_retry_strategy = std::make_shared(context->getSettingsRef().s3_single_read_retry_attempts); UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; @@ -119,7 +122,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( secret_access_key, StorageID(getDatabaseName(), table_name), format, - s3_max_single_read_retries, + s3_max_single_read_retry_strategy, min_upload_part_size, max_single_part_upload_size, max_connections, From 90efb9d0dbbe9bea5102c0a14f3add4e49e1ec92 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 22 Apr 2021 09:57:15 +0300 Subject: [PATCH 133/652] Minor fix. --- src/TableFunctions/TableFunctionS3Cluster.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index f07116736a4..8d16e62e297 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -112,7 +112,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( Poco::URI uri (filename); S3::URI s3_uri (uri); /// Actually this parameters are not used - auto s3_max_single_read_retry_strategy = std::make_shared(context->getSettingsRef().s3_single_read_retry_attempts); + auto single_read_retry_strategy = std::make_shared(context->getSettingsRef().s3_single_read_retry_attempts); UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; @@ -122,7 +122,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( secret_access_key, StorageID(getDatabaseName(), table_name), format, - s3_max_single_read_retry_strategy, + single_read_retry_strategy, min_upload_part_size, max_single_part_upload_size, max_connections, From fa7308d2cb8c49af6ef020ee8f3842b0187cbdd7 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 29 Apr 2021 09:29:06 +0300 Subject: [PATCH 134/652] Minor fix. --- src/IO/ReadBufferFromS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index dde4564ea53..42db3c6e842 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -72,7 +72,7 @@ bool ReadBufferFromS3::nextImpl() ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1); - LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Remaining attempts: {}, Message: {}", + LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Attempt: {}, Message: {}", bucket, key, getPosition(), attempt, e.message()); impl.reset(); From 752d0cb0b62b2bf4fa1734f0c8fdb4a8d0b836a4 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 29 Apr 2021 09:30:44 +0300 Subject: [PATCH 135/652] Fixed test. --- tests/integration/test_storage_s3/s3_mocks/unstable_server.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 4a27845ff9f..03e0d8fa288 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -40,7 +40,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.end_bytes = len(lines) self.size = self.end_bytes self.send_block_size = 256 - self.stop_at = random.randint(900000, 1200000) // self.send_block_size # Block size is 1024**2. + self.stop_at = random.randint(900000, 1300000) // self.send_block_size # Block size is 1024**2. if "Range" in self.headers: cr = self.headers["Range"] From 914116d759867a8ac9d28cd008033572e9b0cef0 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 6 May 2021 10:53:23 +0300 Subject: [PATCH 136/652] Added documentation. --- docs/en/engines/table-engines/integrations/s3.md | 3 +++ docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 ++ docs/ru/engines/table-engines/integrations/s3.md | 3 +++ docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 ++ 4 files changed, 10 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index a27308b9b3f..78fcfaa7ac4 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -130,6 +130,7 @@ The following settings can be set before query execution or placed into configur - `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`. - `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`. - `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`. +- `s3_single_read_retry_attempts` — The maximum number of attempts during single read. Default value is `4`. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. @@ -144,6 +145,7 @@ The following settings can be specified in configuration file for given endpoint - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be speficied multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. +- `single_read_retry_attempts` — The maximum number of attempts during single read. Default value is `4`. Optional. **Example:** @@ -158,6 +160,7 @@ The following settings can be specified in configuration file for given endpoint + ``` diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 8743090df41..654cc39e548 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -748,6 +748,7 @@ Configuration markup: 10000 5000 10 + 4 1000 /var/lib/clickhouse/disks/s3/ true @@ -772,6 +773,7 @@ Optional parameters: - `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`. - `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`. - `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`. +- `single_read_retry_attempts` — Number of retry attempts in case of connection drop during read. Default value is `4`. - `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`. - `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks//`. - `cache_enabled` — Allows to cache mark and index files on local FS. Default value is `true`. diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 177d69dc3e0..bf9c51c6c04 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -70,6 +70,7 @@ SELECT * FROM s3_engine_table LIMIT 2; - `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`. - `s3_min_upload_part_size` — минимальный размер объекта для загрузки при многокомпонентной загрузке в [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Значение по умолчанию — `512 Mб`. - `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`. +- `s3_single_read_retry_attempts` — максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. Соображение безопасности: если злонамеренный пользователь попробует указать произвольные URL-адреса S3, параметр `s3_max_redirects` должен быть установлен в ноль, чтобы избежать атак [SSRF] (https://en.wikipedia.org/wiki/Server-side_request_forgery). Как альтернатива, в конфигурации сервера должен быть указан `remote_host_filter`. @@ -87,6 +88,7 @@ SELECT * FROM s3_engine_table LIMIT 2; - `region` — название региона S3. - `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз. - `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. +- `single_read_retry_attempts` — Максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. **Пример** @@ -101,6 +103,7 @@ SELECT * FROM s3_engine_table LIMIT 2; + ``` diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 4cff6fcfb80..8530af66352 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -735,6 +735,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 10000 5000 10 + 4 1000 /var/lib/clickhouse/disks/s3/ true @@ -761,6 +762,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' - `connect_timeout_ms` — таймаут подключения к сокету в миллисекундах. Значение по умолчанию: 10 секунд. - `request_timeout_ms` — таймаут выполнения запроса в миллисекундах. Значение по умолчанию: 5 секунд. - `retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки. Значение по умолчанию: `10`. +- `single_read_retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки в процессе чтения. Значение по умолчанию: `4`. - `min_bytes_for_seek` — минимальное количество байтов, которые используются для операций поиска вместо последовательного чтения. Значение по умолчанию: 1 МБайт. - `metadata_path` — путь к локальному файловому хранилищу для хранения файлов с метаданными для S3. Значение по умолчанию: `/var/lib/clickhouse/disks//`. - `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`. From 40e63646ca65353926def430b1d551619238b17d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 17 May 2021 14:16:16 +0300 Subject: [PATCH 137/652] more --- .../compose/docker_compose_rabbitmq.yml | 5 + tests/integration/ci-runner.py | 2 +- tests/integration/helpers/cluster.py | 9 + .../test_attach_without_fetching/test.py | 8 +- .../test_dictionaries_postgresql/test.py | 2 +- .../config.d/zookeeper_session_timeout.xml | 2 +- .../config.d/zookeeper_session_timeout.xml | 2 +- .../integration/test_distributed_ddl/test.py | 6 +- tests/integration/test_multiple_disks/test.py | 160 +++++++++--------- 9 files changed, 105 insertions(+), 91 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index 997a346779c..ef6016761a7 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -9,3 +9,8 @@ services: environment: RABBITMQ_DEFAULT_USER: "root" RABBITMQ_DEFAULT_PASS: "clickhouse" + RABBITMQ_LOGS: /rabbitmq_logs/ + volumes: + - type: ${RABBITMQ_LOGS_FS:-tmpfs} + source: ${RABBITMQ_LOGS:-} + target: /rabbitmq_logs/ \ No newline at end of file diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4e2b361dcc9..970daa7ec46 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -84,7 +84,7 @@ def get_counters(output): else: logging.info("Strange line %s", line) else: - logging.info("Strange line %s") + logging.info("Strange line %s", line) return {k: list(v) for k, v in counters.items()} diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d3f88203eac..cfa25a613b9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -291,6 +291,9 @@ class ClickHouseCluster: self.rabbitmq_host = "rabbitmq1" self.rabbitmq_ip = None self.rabbitmq_port = 5672 + self.rabbitmq_dir = p.abspath(p.join(self.instances_dir, "rabbitmq")) + self.rabbitmq_logs_dir = os.path.join(self.rabbitmq_dir, "logs") + # available when with_redis == True self.redis_host = "redis1" @@ -535,6 +538,8 @@ class ClickHouseCluster: self.with_rabbitmq = True env_variables['RABBITMQ_HOST'] = self.rabbitmq_host env_variables['RABBITMQ_PORT'] = str(self.rabbitmq_port) + env_variables['RABBITMQ_LOGS'] = self.rabbitmq_logs_dir + env_variables['RABBITMQ_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, @@ -1208,6 +1213,9 @@ class ClickHouseCluster: self.wait_kafka_is_available(self.kerberized_kafka_docker_id, self.kerberized_kafka_port, 100) if self.with_rabbitmq and self.base_rabbitmq_cmd: + logging.debug('Setup RabbitMQ') + os.makedirs(self.rabbitmq_logs_dir) + os.chmod(self.rabbitmq_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') self.wait_rabbitmq_to_start() @@ -1282,6 +1290,7 @@ class ClickHouseCluster: logging.debug("Failed to start cluster: ") logging.debug(str(e)) logging.debug(traceback.print_exc()) + self.shutdown() raise def shutdown(self, kill=True): diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 8b0c1ffbc5c..1a7a2a1074b 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -7,9 +7,9 @@ from helpers.network import PartitionManager from helpers.corrupt_part_data_on_disk import corrupt_part_data_by_path def fill_node(node): - node.query( + node.query_with_retry( ''' - CREATE TABLE test(n UInt32) + CREATE TABLE IF NOT EXISTS test(n UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', '{replica}') ORDER BY n PARTITION BY n % 10; '''.format(replica=node.name)) @@ -39,9 +39,9 @@ def start_cluster(): def check_data(nodes, detached_parts): for node in nodes: print("> Replication queue for", node.name, "\n> table\treplica_name\tsource_replica\ttype\tposition\n", - node.query("SELECT table, replica_name, source_replica, type, position FROM system.replication_queue")) + node.query_with_retry("SELECT table, replica_name, source_replica, type, position FROM system.replication_queue")) - node.query("SYSTEM SYNC REPLICA test") + node.query_with_retry("SYSTEM SYNC REPLICA test") print("> Checking data integrity for", node.name) diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index 36cbaef6fb9..f0d18909166 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -160,7 +160,7 @@ def test_dictionary_with_replicas(started_cluster): def test_postgres_scema(started_cluster): - conn = get_postgres_conn(port=5432, database=True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') diff --git a/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml b/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml index 071725b5391..3b878bbf4de 100644 --- a/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml +++ b/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml @@ -1,6 +1,6 @@ - 3000 + 10000 diff --git a/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml b/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml index 071725b5391..3b878bbf4de 100644 --- a/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml +++ b/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml @@ -1,6 +1,6 @@ - 3000 + 10000 diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 68d8a2dab53..87e793a7acb 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -98,7 +98,7 @@ def _test_on_connection_losses(test_cluster, zk_timeout): with PartitionManager() as pm: pm.drop_instance_zk_connections(kill_instance) - request = instance.get_query_request("DROP TABLE IF EXISTS test.__nope__ ON CLUSTER 'cluster'", timeout=10) + request = instance.get_query_request("DROP TABLE IF EXISTS test.__nope__ ON CLUSTER 'cluster'", timeout=20) time.sleep(zk_timeout) pm.restore_instance_zk_connections(kill_instance) @@ -106,11 +106,11 @@ def _test_on_connection_losses(test_cluster, zk_timeout): def test_on_connection_loss(test_cluster): - _test_on_connection_losses(test_cluster, 1.5) # connection loss will occur only (3 sec ZK timeout in config) + _test_on_connection_losses(test_cluster, 5) # connection loss will occur only (3 sec ZK timeout in config) def test_on_session_expired(test_cluster): - _test_on_connection_losses(test_cluster, 4) # session should be expired (3 sec ZK timeout in config) + _test_on_connection_losses(test_cluster, 15) # session should be expired (3 sec ZK timeout in config) def test_simple_alters(test_cluster): diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 7ad7fe92b0f..e821bd4e5a8 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -244,7 +244,7 @@ def test_query_parser(start_cluster): try: with pytest.raises(QueryRuntimeException): node1.query(""" - CREATE TABLE table_with_absent_policy ( + CREATE TABLE IF NOT EXISTS table_with_absent_policy ( d UInt64 ) ENGINE = MergeTree() ORDER BY d @@ -253,7 +253,7 @@ def test_query_parser(start_cluster): with pytest.raises(QueryRuntimeException): node1.query(""" - CREATE TABLE table_with_absent_policy ( + CREATE TABLE IF NOT EXISTS table_with_absent_policy ( d UInt64 ) ENGINE = MergeTree() ORDER BY d @@ -261,7 +261,7 @@ def test_query_parser(start_cluster): """) node1.query(""" - CREATE TABLE table_with_normal_policy ( + CREATE TABLE IF NOT EXISTS table_with_normal_policy ( d UInt64 ) ENGINE = MergeTree() ORDER BY d @@ -295,8 +295,8 @@ def test_query_parser(start_cluster): ]) def test_alter_policy(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( d UInt64 ) ENGINE = {engine} ORDER BY d @@ -314,7 +314,7 @@ def test_alter_policy(start_cluster, name, engine): assert node1.query("""SELECT storage_policy FROM system.tables WHERE name = '{name}'""".format( name=name)) == "small_jbod_with_external\n" - node1.query("""ALTER TABLE {name} MODIFY SETTING storage_policy='jbods_with_external'""".format(name=name)) + node1.query_with_retry("""ALTER TABLE {name} MODIFY SETTING storage_policy='jbods_with_external'""".format(name=name)) assert node1.query("""SELECT storage_policy FROM system.tables WHERE name = '{name}'""".format( name=name)) == "jbods_with_external\n" @@ -327,7 +327,7 @@ def test_alter_policy(start_cluster, name, engine): name=name)) == "jbods_with_external\n" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") def get_random_string(length): @@ -349,13 +349,13 @@ def test_no_warning_about_zero_max_data_part_size(start_cluster): for node in (node1, node2): node.query(""" - CREATE TABLE default.test_warning_table ( + CREATE TABLE IF NOT EXISTS default.test_warning_table ( s String ) ENGINE = MergeTree ORDER BY tuple() SETTINGS storage_policy='small_jbod_with_external' """) - node.query("DROP TABLE default.test_warning_table SYNC") + node.query("DROP TABLE IF EXISTS default.test_warning_table SYNC") log = get_log(node) assert not re.search("Warning.*Volume.*special_warning_zero_volume", log) assert not re.search("Warning.*Volume.*special_warning_default_volume", log) @@ -369,8 +369,8 @@ def test_no_warning_about_zero_max_data_part_size(start_cluster): ]) def test_round_robin(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( d UInt64 ) ENGINE = {engine} ORDER BY d @@ -378,17 +378,17 @@ def test_round_robin(start_cluster, name, engine): """.format(name=name, engine=engine)) # first should go to the jbod1 - node1.query("insert into {} select * from numbers(10000)".format(name)) + node1.query_with_retry("insert into {} select * from numbers(10000)".format(name)) used_disk = get_used_disks_for_table(node1, name) assert len(used_disk) == 1, 'More than one disk used for single insert' - node1.query("insert into {} select * from numbers(10000, 10000)".format(name)) + node1.query_with_retry("insert into {} select * from numbers(10000, 10000)".format(name)) used_disks = get_used_disks_for_table(node1, name) assert len(used_disks) == 2, 'Two disks should be used for two parts' assert used_disks[0] != used_disks[1], "Should write to different disks" - node1.query("insert into {} select * from numbers(20000, 10000)".format(name)) + node1.query_with_retry("insert into {} select * from numbers(20000, 10000)".format(name)) used_disks = get_used_disks_for_table(node1, name) # jbod1 -> jbod2 -> jbod1 -> jbod2 ... etc @@ -396,7 +396,7 @@ def test_round_robin(start_cluster, name, engine): assert used_disks[0] != used_disks[1] assert used_disks[2] == used_disks[0] finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -407,8 +407,8 @@ def test_max_data_part_size(start_cluster, name, engine): try: assert int(*node1.query("""SELECT max_data_part_size FROM system.storage_policies WHERE policy_name = 'jbods_with_external' AND volume_name = 'main'""").splitlines()) == 10*1024*1024 - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -418,12 +418,12 @@ def test_max_data_part_size(start_cluster, name, engine): for i in range(10): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) assert len(used_disks) == 1 assert used_disks[0] == 'external' finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -432,8 +432,8 @@ def test_max_data_part_size(start_cluster, name, engine): ]) def test_jbod_overflow(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -447,7 +447,7 @@ def test_jbod_overflow(start_cluster, name, engine): data = [] # 5MB in total for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) assert all(disk == 'jbod1' for disk in used_disks) @@ -457,7 +457,7 @@ def test_jbod_overflow(start_cluster, name, engine): for i in range(10): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -466,7 +466,7 @@ def test_jbod_overflow(start_cluster, name, engine): node1.query(f"SYSTEM START MERGES {name}") time.sleep(1) - node1.query("OPTIMIZE TABLE {} FINAL".format(name)) + node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) time.sleep(2) disks_for_merges = node1.query( @@ -476,7 +476,7 @@ def test_jbod_overflow(start_cluster, name, engine): assert all(disk == 'external' for disk in disks_for_merges) finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -485,8 +485,8 @@ def test_jbod_overflow(start_cluster, name, engine): ]) def test_background_move(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -500,7 +500,7 @@ def test_background_move(start_cluster, name, engine): for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row # small jbod size is 40MB, so lets insert 5MB batch 5 times - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -526,7 +526,7 @@ def test_background_move(start_cluster, name, engine): node1.query(f"SYSTEM START MERGES {name}") finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -535,16 +535,16 @@ def test_background_move(start_cluster, name, engine): ]) def test_start_stop_moves(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() SETTINGS storage_policy='moving_jbod_with_external' """.format(name=name, engine=engine)) - node1.query("INSERT INTO {} VALUES ('HELLO')".format(name)) - node1.query("INSERT INTO {} VALUES ('WORLD')".format(name)) + node1.query_with_retry("INSERT INTO {} VALUES ('HELLO')".format(name)) + node1.query_with_retry("INSERT INTO {} VALUES ('WORLD')".format(name)) used_disks = get_used_disks_for_table(node1, name) assert all(d == "jbod1" for d in used_disks), "All writes shoud go to jbods" @@ -571,7 +571,7 @@ def test_start_stop_moves(start_cluster, name, engine): assert disk == "external" - node1.query("TRUNCATE TABLE {}".format(name)) + node1.query_with_retry("TRUNCATE TABLE {}".format(name)) node1.query("SYSTEM STOP MOVES {}".format(name)) node1.query("SYSTEM STOP MERGES {}".format(name)) @@ -581,7 +581,7 @@ def test_start_stop_moves(start_cluster, name, engine): for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row # jbod size is 40MB, so lets insert 5MB batch 7 times - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -613,7 +613,7 @@ def test_start_stop_moves(start_cluster, name, engine): assert used_disks[0] == 'external' finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") def get_path_for_part_from_part_log(node, table, part_name): @@ -640,7 +640,7 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): def test_alter_move(start_cluster, name, engine): try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -713,7 +713,7 @@ def test_alter_move_half_of_partition(start_cluster, volume_or_disk): engine = "MergeTree()" try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -762,7 +762,7 @@ def test_alter_double_move_partition(start_cluster, volume_or_disk): engine = "MergeTree()" try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -833,8 +833,8 @@ def produce_alter_move(node, name): ]) def test_concurrent_alter_move(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -850,7 +850,7 @@ def test_concurrent_alter_move(start_cluster, name, engine): day = random.randint(11, 30) value = values.pop() month = '0' + str(random.choice([3, 4])) - node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) + node1.query_with_retry("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) def alter_move(num): for i in range(num): @@ -862,7 +862,7 @@ def test_concurrent_alter_move(start_cluster, name, engine): def optimize_table(num): for i in range(num): - node1.query("OPTIMIZE TABLE {} FINAL".format(name)) + node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) p = Pool(15) tasks = [] @@ -889,7 +889,7 @@ def test_concurrent_alter_move(start_cluster, name, engine): def test_concurrent_alter_move_and_drop(start_cluster, name, engine): try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -905,7 +905,7 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): day = random.randint(11, 30) value = values.pop() month = '0' + str(random.choice([3, 4])) - node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) + node1.query_with_retry("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) def alter_move(num): for i in range(num): @@ -931,7 +931,7 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): assert node1.query("SELECT 1") == "1\n" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -940,8 +940,8 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): ]) def test_detach_attach(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -951,7 +951,7 @@ def test_detach_attach(start_cluster, name, engine): data = [] # 5MB in total for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) node1.query("ALTER TABLE {} DETACH PARTITION tuple()".format(name)) assert node1.query("SELECT count() FROM {}".format(name)).strip() == "0" @@ -962,7 +962,7 @@ def test_detach_attach(start_cluster, name, engine): assert node1.query("SELECT count() FROM {}".format(name)).strip() == "5" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -971,8 +971,8 @@ def test_detach_attach(start_cluster, name, engine): ]) def test_mutate_to_another_disk(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -983,7 +983,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): data = [] # 5MB in total for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name)) @@ -996,7 +996,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): assert node1.query("SELECT sum(endsWith(s1, 'x')) FROM {}".format(name)) == "25\n" else: # mutation failed, let's try on another disk print("Mutation failed") - node1.query("OPTIMIZE TABLE {} FINAL".format(name)) + node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name)) retry = 20 while node1.query("SELECT * FROM system.mutations WHERE is_done = 0") != "" and retry > 0: @@ -1008,7 +1008,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ @@ -1017,8 +1017,8 @@ def test_mutate_to_another_disk(start_cluster, name, engine): ]) def test_concurrent_alter_modify(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -1034,7 +1034,7 @@ def test_concurrent_alter_modify(start_cluster, name, engine): day = random.randint(11, 30) value = values.pop() month = '0' + str(random.choice([3, 4])) - node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) + node1.query_with_retry("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) def alter_move(num): for i in range(num): @@ -1066,14 +1066,14 @@ def test_concurrent_alter_modify(start_cluster, name, engine): assert node1.query("SELECT COUNT() FROM {}".format(name)) == "100\n" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") def test_simple_replication_and_moves(start_cluster): try: for i, node in enumerate([node1, node2]): - node.query(""" - CREATE TABLE replicated_table_for_moves ( + node.query_with_retry(""" + CREATE TABLE IF NOT EXISTS replicated_table_for_moves ( s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_moves', '{}') ORDER BY tuple() @@ -1086,13 +1086,13 @@ def test_simple_replication_and_moves(start_cluster): data = [] # 1MB in total for i in range(2): data.append(get_random_string(512 * 1024)) # 500KB value - node.query("INSERT INTO replicated_table_for_moves VALUES {}".format( + node.query_with_retry("INSERT INTO replicated_table_for_moves VALUES {}".format( ','.join(["('" + x + "')" for x in data]))) def optimize(num): for i in range(num): node = random.choice([node1, node2]) - node.query("OPTIMIZE TABLE replicated_table_for_moves FINAL") + node.query_with_retry("OPTIMIZE TABLE replicated_table_for_moves FINAL") p = Pool(60) tasks = [] @@ -1102,8 +1102,8 @@ def test_simple_replication_and_moves(start_cluster): for task in tasks: task.get(timeout=60) - node1.query("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) - node2.query("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) + node1.query_with_retry("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) + node2.query_with_retry("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) node1.query("SELECT COUNT() FROM replicated_table_for_moves") == "40\n" node2.query("SELECT COUNT() FROM replicated_table_for_moves") == "40\n" @@ -1116,9 +1116,9 @@ def test_simple_replication_and_moves(start_cluster): node1.query("SYSTEM STOP MERGES") node2.query("SYSTEM STOP MERGES") - node1.query( + node1.query_with_retry( "INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data]))) - node2.query( + node2.query_with_retry( "INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data]))) time.sleep(3) # nothing was moved @@ -1139,8 +1139,8 @@ def test_simple_replication_and_moves(start_cluster): def test_download_appropriate_disk(start_cluster): try: for i, node in enumerate([node1, node2]): - node.query(""" - CREATE TABLE replicated_table_for_download ( + node.query_with_retry(""" + CREATE TABLE IF NOT EXISTS replicated_table_for_download ( s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_download', '{}') ORDER BY tuple() @@ -1150,13 +1150,13 @@ def test_download_appropriate_disk(start_cluster): data = [] for i in range(50): data.append(get_random_string(1024 * 1024)) # 1MB value - node1.query( + node1.query_with_retry( "INSERT INTO replicated_table_for_download VALUES {}".format(','.join(["('" + x + "')" for x in data]))) for _ in range(10): try: print("Syncing replica") - node2.query("SYSTEM SYNC REPLICA replicated_table_for_download") + node2.query_with_retry("SYSTEM SYNC REPLICA replicated_table_for_download") break except: time.sleep(0.5) @@ -1167,13 +1167,13 @@ def test_download_appropriate_disk(start_cluster): finally: for node in [node1, node2]: - node.query("DROP TABLE IF EXISTS replicated_table_for_download SYNC") + node.query_with_retry("DROP TABLE IF EXISTS replicated_table_for_download SYNC") def test_rename(start_cluster): try: node1.query(""" - CREATE TABLE default.renaming_table ( + CREATE TABLE IF NOT EXISTS default.renaming_table ( s String ) ENGINE = MergeTree ORDER BY tuple() @@ -1212,7 +1212,7 @@ def test_rename(start_cluster): def test_freeze(start_cluster): try: node1.query(""" - CREATE TABLE default.freezing_table ( + CREATE TABLE IF NOT EXISTS default.freezing_table ( d Date, s String ) ENGINE = MergeTree @@ -1249,7 +1249,7 @@ def test_kill_while_insert(start_cluster): name = "test_kill_while_insert" node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( s String ) ENGINE = MergeTree ORDER BY tuple() @@ -1294,7 +1294,7 @@ def test_move_while_merge(start_cluster): name = "test_move_while_merge" node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY sleep(2) @@ -1353,7 +1353,7 @@ def test_move_across_policies_does_not_work(start_cluster): name = "test_move_across_policies_does_not_work" node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() @@ -1361,7 +1361,7 @@ def test_move_across_policies_does_not_work(start_cluster): """.format(name=name)) node1.query(""" - CREATE TABLE {name}2 ( + CREATE TABLE IF NOT EXISTS {name}2 ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() @@ -1393,7 +1393,7 @@ def test_move_across_policies_does_not_work(start_cluster): def _insert_merge_execute(node, name, policy, parts, cmds, parts_before_cmds, parts_after_cmds): try: node.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() @@ -1430,7 +1430,7 @@ def _check_merges_are_working(node, storage_policy, volume, shall_work): name = "_check_merges_are_working_{storage_policy}_{volume}".format(storage_policy=storage_policy, volume=volume) node.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() From 39f41ed9b988f4bf002286b8a7e90d4e9200144a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 May 2021 07:47:51 +0300 Subject: [PATCH 138/652] less tests to commit --- tests/integration/parallel.json | 1126 +------------------------------ 1 file changed, 2 insertions(+), 1124 deletions(-) diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index d04cdd4708f..3e6ef773c78 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -1,1030 +1,4 @@ [ - "test_access_control_on_cluster/test.py::test_access_control_on_cluster", - "test_aggregation_memory_efficient/test.py::test_remote", - "test_allowed_client_hosts/test.py::test_allowed_host", - "test_allowed_url_from_config/test.py::test_config_with_hosts", - "test_allowed_url_from_config/test.py::test_config_with_only_primary_hosts", - "test_allowed_url_from_config/test.py::test_config_with_only_regexp_hosts", - "test_allowed_url_from_config/test.py::test_config_without_allowed_hosts", - "test_allowed_url_from_config/test.py::test_config_without_allowed_hosts_section", - "test_allowed_url_from_config/test.py::test_HDFS", - "test_allowed_url_from_config/test.py::test_redirect", - "test_allowed_url_from_config/test.py::test_table_function_remote", - "test_alter_codec/test.py::test_alter_codec_index", - "test_alter_codec/test.py::test_alter_codec_pk", - "test_alter_on_mixed_type_cluster/test.py::test_alter_on_cluter_non_replicated", - "test_alter_on_mixed_type_cluster/test.py::test_alter_replicated_on_cluster", - "test_alter_settings_on_cluster/test.py::test_default_database_on_cluster", - "test_always_fetch_merged/test.py::test_replica_always_download", - "test_asynchronous_metric_log_table/test.py::test_event_time_microseconds_field", - "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", - "test_attach_without_checksums/test.py::test_attach_without_checksums", - "test_attach_without_fetching/test.py::test_attach_without_fetching", - "test_authentication/test.py::test_authentication_fail", - "test_authentication/test.py::test_authentication_pass", - "test_backup_restore/test.py::test_attach_partition", - "test_backup_restore/test.py::test_replace_partition", - "test_backup_restore/test.py::test_restore", - "test_backward_compatibility/test_aggregate_function_state_avg.py::test_backward_compatability", - "test_backward_compatibility/test.py::test_backward_compatability1", - "test_backward_compatibility/test_short_strings_aggregation.py::test_backward_compatability", - "test_block_structure_mismatch/test.py::test", - "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", - "test_buffer_profile/test.py::test_buffer_profile", - "test_buffer_profile/test.py::test_default_profile", - "test_catboost_model_config_reload/test.py::test", - "test_catboost_model_first_evaluate/test.py::test", - "test_catboost_model_reload/test.py::test_model_reload", - "test_catboost_model_reload/test.py::test_models_reload", - "test_check_table/test.py::test_check_normal_table_corruption", - "test_check_table/test.py::test_check_replicated_table_corruption", - "test_check_table/test.py::test_check_replicated_table_simple", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path", - "test_cluster_all_replicas/test.py::test_remote", - "test_cluster_copier/test.py::test_block_size", - "test_cluster_copier/test.py::test_copy_month_to_week_partition", - "test_cluster_copier/test.py::test_copy_month_to_week_partition_with_recovering", - "test_cluster_copier/test.py::test_copy_month_to_week_partition_with_recovering_after_move_faults", - "test_cluster_copier/test.py::test_copy_simple[False]", - "test_cluster_copier/test.py::test_copy_simple[True]", - "test_cluster_copier/test.py::test_copy_with_recovering_after_move_faults[False]", - "test_cluster_copier/test.py::test_copy_with_recovering_after_move_faults[True]", - "test_cluster_copier/test.py::test_copy_with_recovering[False]", - "test_cluster_copier/test.py::test_copy_with_recovering[True]", - "test_cluster_copier/test.py::test_no_arg", - "test_cluster_copier/test.py::test_no_index", - "test_cluster_copier/test.py::test_non_partitioned_table", - "test_cluster_copier/test.py::test_self_copy", - "test_compression_codec_read/test.py::test_default_codec_read", - "test_compression_nested_columns/test.py::test_nested_compression_codec", - "test_concurrent_queries_for_all_users_restriction/test.py::test_exception_message", - "test_concurrent_queries_for_user_restriction/test.py::test_exception_message", - "test_concurrent_ttl_merges/test.py::test_limited_ttl_merges_in_empty_pool", - "test_concurrent_ttl_merges/test.py::test_limited_ttl_merges_in_empty_pool_replicated", - "test_concurrent_ttl_merges/test.py::test_limited_ttl_merges_two_replicas", - "test_concurrent_ttl_merges/test.py::test_no_ttl_merges_in_busy_pool", - "test_config_corresponding_root/test.py::test_work", - "test_config_substitutions/test.py::test_allow_databases", - "test_config_substitutions/test.py::test_config", - "test_consistant_parts_after_move_partition/test.py::test_consistent_part_after_move_partition", - "test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active", - "test_create_user_and_login/test.py::test_grant_create_user", - "test_create_user_and_login/test.py::test_login", - "test_cross_replication/test.py::test", - "test_custom_settings/test.py::test_custom_settings", - "test_custom_settings/test.py::test_illformed_setting", - "test_ddl_alter_query/test.py::test_alter", - "test_ddl_worker_non_leader/test.py::test_non_leader_replica", - "test_default_compression_codec/test.py::test_default_codec_for_compact_parts", - "test_default_compression_codec/test.py::test_default_codec_multiple", - "test_default_compression_codec/test.py::test_default_codec_single", - "test_default_compression_codec/test.py::test_default_codec_version_update", - "test_default_database_on_cluster/test.py::test_default_database_on_cluster", - "test_default_role/test.py::test_alter_user", - "test_default_role/test.py::test_set_default_roles", - "test_default_role/test.py::test_wrong_set_default_role", - "test_delayed_replica_failover/test.py::test", - "test_dictionaries_access/test.py::test_create", - "test_dictionaries_access/test.py::test_dictget", - "test_dictionaries_access/test.py::test_drop", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", - "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", - "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", - "test_dictionaries_ddl/test.py::test_clickhouse_remote", - "test_dictionaries_ddl/test.py::test_conflicting_name", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", - "test_dictionaries_ddl/test.py::test_dictionary_with_where", - "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_restricted_database", - "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", - "test_dictionaries_dependency_xml/test.py::test_get_data", - "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", - "test_dictionaries_null_value/test.py::test_null_value", - "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", - "test_dictionaries_postgresql/test.py::test_invalidate_query", - "test_dictionaries_postgresql/test.py::test_load_dictionaries", - "test_dictionaries_postgresql/test.py::test_postgres_scema", - "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", - "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", - "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", - "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", - "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", - "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", - "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", - "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", - "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", - "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", - "test_dictionary_custom_settings/test.py::test_work", - "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", - "test_disabled_mysql_server/test.py::test_disabled_mysql_server", - "test_disk_access_storage/test.py::test_alter", - "test_disk_access_storage/test.py::test_create", - "test_disk_access_storage/test.py::test_drop", - "test_disk_types/test.py::test_different_types", - "test_disk_types/test.py::test_select_by_type", - "test_distributed_backward_compatability/test.py::test_distributed_in_tuple", - "test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl", - "test_distributed_ddl_on_cross_replication/test.py::test_atomic_database", - "test_distributed_ddl_parallel/test.py::test_all_in_parallel", - "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", - "test_distributed_ddl_parallel/test.py::test_smoke", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", - "test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued", - "test_distributed_ddl_password/test.py::test_alter", - "test_distributed_ddl_password/test.py::test_truncate", - "test_distributed_ddl/test.py::test_allowed_databases[configs]", - "test_distributed_ddl/test.py::test_allowed_databases[configs_secure]", - "test_distributed_ddl/test.py::test_create_as_select[configs]", - "test_distributed_ddl/test.py::test_create_as_select[configs_secure]", - "test_distributed_ddl/test.py::test_create_reserved[configs]", - "test_distributed_ddl/test.py::test_create_reserved[configs_secure]", - "test_distributed_ddl/test.py::test_create_view[configs]", - "test_distributed_ddl/test.py::test_create_view[configs_secure]", - "test_distributed_ddl/test.py::test_default_database[configs]", - "test_distributed_ddl/test.py::test_default_database[configs_secure]", - "test_distributed_ddl/test.py::test_detach_query[configs]", - "test_distributed_ddl/test.py::test_detach_query[configs_secure]", - "test_distributed_ddl/test.py::test_implicit_macros[configs]", - "test_distributed_ddl/test.py::test_implicit_macros[configs_secure]", - "test_distributed_ddl/test.py::test_kill_query[configs]", - "test_distributed_ddl/test.py::test_kill_query[configs_secure]", - "test_distributed_ddl/test.py::test_macro[configs]", - "test_distributed_ddl/test.py::test_macro[configs_secure]", - "test_distributed_ddl/test.py::test_on_connection_loss[configs]", - "test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]", - "test_distributed_ddl/test.py::test_on_server_fail[configs]", - "test_distributed_ddl/test.py::test_on_server_fail[configs_secure]", - "test_distributed_ddl/test.py::test_on_session_expired[configs]", - "test_distributed_ddl/test.py::test_on_session_expired[configs_secure]", - "test_distributed_ddl/test.py::test_optimize_query[configs]", - "test_distributed_ddl/test.py::test_optimize_query[configs_secure]", - "test_distributed_ddl/test.py::test_rename[configs]", - "test_distributed_ddl/test.py::test_rename[configs_secure]", - "test_distributed_ddl/test.py::test_replicated_without_arguments[configs]", - "test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]", - "test_distributed_ddl/test.py::test_simple_alters[configs]", - "test_distributed_ddl/test.py::test_simple_alters[configs_secure]", - "test_distributed_ddl/test.py::test_socket_timeout[configs]", - "test_distributed_ddl/test.py::test_socket_timeout[configs_secure]", - "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]", - "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]", - "test_distributed_format/test.py::test_single_file_old[test_cluster]", - "test_distributed_format/test.py::test_single_file_old[test_cluster_2]", - "test_distributed_format/test.py::test_single_file[test_cluster]", - "test_distributed_format/test.py::test_single_file[test_cluster_2]", - "test_distributed_format/test.py::test_two_files[test_cluster]", - "test_distributed_format/test.py::test_two_files[test_cluster_2]", - "test_distributed_inter_server_secret/test.py::test_insecure", - "test_distributed_inter_server_secret/test.py::test_insecure_insert_async", - "test_distributed_inter_server_secret/test.py::test_insecure_insert_sync", - "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_insecure_cluster[default-]", - "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_insecure_cluster[nopass-]", - "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_insecure_cluster[pass-foo]", - "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[default-]", - "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[nopass-]", - "test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[pass-foo]", - "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_insecure_cluster[default-]", - "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_insecure_cluster[nopass-]", - "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_insecure_cluster[pass-foo]", - "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[default-]", - "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[nopass-]", - "test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[pass-foo]", - "test_distributed_inter_server_secret/test.py::test_secure", - "test_distributed_inter_server_secret/test.py::test_secure_disagree", - "test_distributed_inter_server_secret/test.py::test_secure_disagree_insert", - "test_distributed_inter_server_secret/test.py::test_secure_insert_async", - "test_distributed_inter_server_secret/test.py::test_secure_insert_buffer_async", - "test_distributed_inter_server_secret/test.py::test_secure_insert_sync", - "test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[default-]", - "test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[nopass-]", - "test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[pass-foo]", - "test_distributed_inter_server_secret/test.py::test_user_secure_cluster[default-]", - "test_distributed_inter_server_secret/test.py::test_user_secure_cluster[nopass-]", - "test_distributed_inter_server_secret/test.py::test_user_secure_cluster[pass-foo]", - "test_distributed_load_balancing/test.py::test_distributed_replica_max_ignored_errors", - "test_distributed_load_balancing/test.py::test_load_balancing_default", - "test_distributed_load_balancing/test.py::test_load_balancing_first_or_random", - "test_distributed_load_balancing/test.py::test_load_balancing_in_order", - "test_distributed_load_balancing/test.py::test_load_balancing_nearest_hostname", - "test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority]", - "test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority_negative]", - "test_distributed_load_balancing/test.py::test_load_balancing_round_robin", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[cluster_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[cluster_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[dod_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_sum[dod_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[cluster_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[cluster_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[dod_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_key[dod_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[cluster_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[cluster_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[dod_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_group_by_node[dod_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[cluster_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[cluster_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[dod_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_key[dod_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[cluster_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[cluster_node2]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[dod_node1]", - "test_distributed_over_distributed/test.py::TestDistributedOverDistributedSuite::test_select_with_order_by_node[dod_node2]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings21]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings22]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings23]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings24]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings25]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings26]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings27]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings28]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings29]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings30]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings31]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings32]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings33]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings34]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings35]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings36]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings37]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings38]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings39]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings40]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one_over_dist-settings41]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings0]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings1]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings10]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings11]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings12]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings13]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings14]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings15]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings16]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings17]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings18]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings19]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings2]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings20]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings3]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings4]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings5]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings6]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings7]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings8]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_one-settings9]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings63]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings64]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings65]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings66]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings67]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings68]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings69]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings70]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings71]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings72]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings73]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings74]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings75]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings76]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings77]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings78]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings79]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings80]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings81]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings82]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two_over_dist-settings83]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings42]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings43]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings44]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings45]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings46]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings47]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings48]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings49]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings50]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings51]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings52]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings53]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings54]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings55]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings56]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings57]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings58]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings59]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings60]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings61]", - "test_distributed_queries_stress/test.py::test_stress_distributed[dist_two-settings62]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]", - "test_distributed_storage_configuration/test.py::test_insert", - "test_distributed_system_query/test.py::test_start_and_stop_replica_send", - "test_drop_replica/test.py::test_drop_replica", - "test_enabling_access_management/test.py::test_enabling_access_management", - "test_extreme_deduplication/test.py::test_deduplication_window_in_seconds", - "test_fetch_partition_from_auxiliary_zookeeper/test.py::test_fetch_part_from_allowed_zookeeper[PART-2020-08-28-20200828_0_0_0]", - "test_fetch_partition_from_auxiliary_zookeeper/test.py::test_fetch_part_from_allowed_zookeeper[PARTITION-2020-08-27-2020-08-27]", - "test_fetch_partition_should_reset_mutation/test.py::test_part_should_reset_mutation", - "test_fetch_partition_with_outdated_parts/test.py::test_fetch_partition_with_outdated_parts", - "test_filesystem_layout/test.py::test_file_path_escaping", - "test_filesystem_layout/test.py::test_file_path_escaping_atomic_db", - "test_force_deduplication/test.py::test_basic", - "test_force_drop_table/test.py::test_drop_materialized_view[Atomic]", - "test_force_drop_table/test.py::test_drop_materialized_view[Ordinary]", - "test_format_avro_confluent/test.py::test_select", - "test_format_schema_on_server/test.py::test_protobuf_format_input", - "test_format_schema_on_server/test.py::test_protobuf_format_output", - "test_freeze_table/test.py::test_freeze_table", - "test_globs_in_filepath/test.py::test_deep_structure", - "test_globs_in_filepath/test.py::test_linear_structure", - "test_globs_in_filepath/test.py::test_strange_filenames", - "test_globs_in_filepath/test.py::test_table_function_and_virtual_columns", - "test_grant_and_revoke/test.py::test_allowed_grantees", - "test_grant_and_revoke/test.py::test_current_database", - "test_grant_and_revoke/test.py::test_grant_all_on_table", - "test_grant_and_revoke/test.py::test_grant_option", - "test_grant_and_revoke/test.py::test_implicit_create_temporary_table_grant", - "test_grant_and_revoke/test.py::test_implicit_create_view_grant", - "test_grant_and_revoke/test.py::test_implicit_show_grants", - "test_grant_and_revoke/test.py::test_introspection", - "test_grant_and_revoke/test.py::test_revoke_requires_grant_option", - "test_grant_and_revoke/test.py::test_smoke", - "test_graphite_merge_tree/test.py::test_broken_partial_rollup", - "test_graphite_merge_tree/test.py::test_combined_rules", - "test_graphite_merge_tree/test.py::test_combined_rules_with_default", - "test_graphite_merge_tree/test.py::test_multiple_output_blocks", - "test_graphite_merge_tree/test.py::test_multiple_paths_and_versions", - "test_graphite_merge_tree/test.py::test_path_dangling_pointer", - "test_graphite_merge_tree/test.py::test_paths_not_matching_any_pattern", - "test_graphite_merge_tree/test.py::test_rollup_aggregation", - "test_graphite_merge_tree/test.py::test_rollup_aggregation_2", - "test_graphite_merge_tree/test.py::test_rollup_versions", - "test_graphite_merge_tree/test.py::test_system_graphite_retentions", - "test_graphite_merge_tree/test.py::test_wrong_rollup_config", - "test_hedged_requests_parallel/test.py::test_combination1", - "test_hedged_requests_parallel/test.py::test_combination2", - "test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample", - "test_hedged_requests_parallel/test.py::test_send_data", - "test_hedged_requests_parallel/test.py::test_send_table_status_sleep", - "test_hedged_requests/test.py::test_combination1", - "test_hedged_requests/test.py::test_combination2", - "test_hedged_requests/test.py::test_combination3", - "test_hedged_requests/test.py::test_combination4", - "test_hedged_requests/test.py::test_long_query", - "test_hedged_requests/test.py::test_receive_timeout1", - "test_hedged_requests/test.py::test_receive_timeout2", - "test_hedged_requests/test.py::test_send_data", - "test_hedged_requests/test.py::test_send_data2", - "test_hedged_requests/test.py::test_send_table_status_sleep", - "test_hedged_requests/test.py::test_send_table_status_sleep2", - "test_hedged_requests/test.py::test_stuck_replica", - "test_host_ip_change/test.py::test_dns_cache_update", - "test_host_ip_change/test.py::test_ip_change_drop_dns_cache", - "test_host_ip_change/test.py::test_ip_change_update_dns_cache", - "test_host_ip_change/test.py::test_user_access_ip_change[node0]", - "test_host_ip_change/test.py::test_user_access_ip_change[node1]", - "test_http_and_readonly/test.py::test_http_get_is_readonly", - "test_http_handlers_config/test.py::test_absolute_path_static_handler", - "test_http_handlers_config/test.py::test_config_static_handler", - "test_http_handlers_config/test.py::test_defaults_http_handlers", - "test_http_handlers_config/test.py::test_dynamic_query_handler", - "test_http_handlers_config/test.py::test_fixed_static_handler", - "test_http_handlers_config/test.py::test_predefined_query_handler", - "test_http_handlers_config/test.py::test_prometheus_handler", - "test_http_handlers_config/test.py::test_relative_path_static_handler", - "test_http_handlers_config/test.py::test_replicas_status_handler", - "test_https_replication/test.py::test_both_http", - "test_https_replication/test.py::test_both_https", - "test_https_replication/test.py::test_mixed_protocol", - "test_https_replication/test.py::test_replication_after_partition", - "test_inherit_multiple_profiles/test.py::test_combined_profile", - "test_input_format_parallel_parsing_memory_tracking/test.py::test_memory_tracking_total", - "test_insert_distributed_async_extra_dirs/test.py::test_insert_distributed_async_send_success", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_big[0]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_big[1]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_small[0]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_corrupted_small[1]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_different_header[0]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_different_header[1]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_success[0]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_success[1]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_1[0]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_1[1]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_2[0]", - "test_insert_distributed_async_send/test.py::test_insert_distributed_async_send_truncated_2[1]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order[internal_replication-0]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order[no_internal_replication-1]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order_sync[internal_replication-0]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_in_order_sync[no_internal_replication-1]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_nearest_hostname[internal_replication-0]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_0_load_balancing_nearest_hostname[no_internal_replication-1]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1[internal_replication-0]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1_load_balancing_in_order[internal_replication-0]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1_load_balancing_in_order[no_internal_replication-1]", - "test_insert_distributed_load_balancing/test.py::test_prefer_localhost_replica_1[no_internal_replication-1]", - "test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout", - "test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout", - "test_insert_into_distributed/test.py::test_inserts_batching", - "test_insert_into_distributed/test.py::test_inserts_local", - "test_insert_into_distributed/test.py::test_inserts_low_cardinality", - "test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication", - "test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication", - "test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication", - "test_insert_into_distributed/test.py::test_prefer_localhost_replica", - "test_insert_into_distributed/test.py::test_reconnect", - "test_insert_into_distributed/test.py::test_table_function", - "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", - "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", - "test_jbod_balancer/test.py::test_jbod_balanced_merge", - "test_jbod_balancer/test.py::test_replicated_balanced_merge_fetch", - "test_join_set_family_s3/test.py::test_join_s3", - "test_join_set_family_s3/test.py::test_set_s3", - "test_keeper_back_to_back/test.py::test_concurrent_watches", - "test_keeper_back_to_back/test.py::test_end_of_session", - "test_keeper_back_to_back/test.py::test_end_of_watches_session", - "test_keeper_back_to_back/test.py::test_multitransactions", - "test_keeper_back_to_back/test.py::test_random_requests", - "test_keeper_back_to_back/test.py::test_sequential_nodes", - "test_keeper_back_to_back/test.py::test_simple_commands", - "test_keeper_back_to_back/test.py::test_stats", - "test_keeper_back_to_back/test.py::test_watchers", - "test_keeper_internal_secure/test.py::test_secure_raft_works", - "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", - "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice", - "test_keeper_multinode_simple/test.py::test_follower_restart", - "test_keeper_multinode_simple/test.py::test_read_write_multinode", - "test_keeper_multinode_simple/test.py::test_session_expiration", - "test_keeper_multinode_simple/test.py::test_simple_replicated_table", - "test_keeper_multinode_simple/test.py::test_watch_on_follower", - "test_keeper_persistent_log_multinode/test.py::test_restart_multinode", - "test_keeper_persistent_log/test.py::test_ephemeral_after_restart", - "test_keeper_persistent_log/test.py::test_state_after_restart", - "test_keeper_restore_from_snapshot/test.py::test_recover_from_snapshot", - "test_keeper_secure_client/test.py::test_connection", - "test_keeper_snapshots_multinode/test.py::test_restart_multinode", - "test_keeper_snapshots/test.py::test_ephemeral_after_restart", - "test_keeper_snapshots/test.py::test_state_after_restart", - "test_library_bridge/test.py::test_load_all", - "test_library_bridge/test.py::test_load_all_many_rows", - "test_library_bridge/test.py::test_load_ids", - "test_library_bridge/test.py::test_load_keys", - "test_library_bridge/test.py::test_null_values", - "test_limited_replicated_fetches/test.py::test_limited_fetches", - "test_log_family_s3/test.py::test_log_family_s3[Log-2-1]", - "test_log_family_s3/test.py::test_log_family_s3[StripeLog-1-2]", - "test_log_family_s3/test.py::test_log_family_s3[TinyLog-1-1]", - "test_logs_level/test.py::test_check_client_logs_level", - "test_match_process_uid_against_data_owner/test.py::test_different_user", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", - "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", - "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", - "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", - "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", - "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", - "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", - "test_max_http_connections_for_replication/test.py::test_keepalive_timeout", - "test_max_http_connections_for_replication/test.py::test_multiple_endpoint_connections_count", - "test_max_http_connections_for_replication/test.py::test_single_endpoint_connections_count", - "test_MemoryTracking/test.py::test_http", - "test_MemoryTracking/test.py::test_tcp_multiple_sessions", - "test_MemoryTracking/test.py::test_tcp_single_session", - "test_merge_table_over_distributed/test.py::test_filtering", - "test_merge_table_over_distributed/test.py::test_global_in", - "test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed", - "test_merge_tree_empty_parts/test.py::test_empty_parts_alter_delete", - "test_merge_tree_empty_parts/test.py::test_empty_parts_summing", - "test_merge_tree_s3_failover/test.py::test_move_failover", - "test_merge_tree_s3_failover/test.py::test_write_failover[0-13]", - "test_merge_tree_s3_failover/test.py::test_write_failover[1048576-9]", - "test_merge_tree_s3_restore/test.py::test_full_restore[False]", - "test_merge_tree_s3_restore/test.py::test_full_restore[True]", - "test_merge_tree_s3_restore/test.py::test_migrate_to_restorable_schema", - "test_merge_tree_s3_restore/test.py::test_restore_another_bucket_path", - "test_merge_tree_s3_restore/test.py::test_restore_different_revisions", - "test_merge_tree_s3_restore/test.py::test_restore_mutations", - "test_merge_tree_s3_restore/test.py::test_restore_to_detached[False]", - "test_merge_tree_s3_restore/test.py::test_restore_to_detached[True]", - "test_merge_tree_s3/test.py::test_alter_table_columns", - "test_merge_tree_s3/test.py::test_attach_detach_partition", - "test_merge_tree_s3/test.py::test_freeze_unfreeze", - "test_merge_tree_s3/test.py::test_insert_same_partition_and_merge[False]", - "test_merge_tree_s3/test.py::test_insert_same_partition_and_merge[True]", - "test_merge_tree_s3/test.py::test_move_partition_to_another_disk", - "test_merge_tree_s3/test.py::test_move_replace_partition_to_another_table", - "test_merge_tree_s3/test.py::test_s3_disk_apply_new_settings", - "test_merge_tree_s3/test.py::test_s3_disk_restart_during_load", - "test_merge_tree_s3/test.py::test_simple_insert_select[0-15]", - "test_merge_tree_s3/test.py::test_simple_insert_select[8192-11]", - "test_merge_tree_s3/test.py::test_table_manipulations", - "test_merge_tree_s3_with_cache/test.py::test_read_after_cache_is_wiped[0-4-2]", - "test_merge_tree_s3_with_cache/test.py::test_read_after_cache_is_wiped[8192-2-1]", - "test_merge_tree_s3_with_cache/test.py::test_write_is_cached[0-2]", - "test_merge_tree_s3_with_cache/test.py::test_write_is_cached[8192-1]", - "test_multiple_disks/test.py::test_alter_double_move_partition[DISK]", - "test_multiple_disks/test.py::test_alter_double_move_partition[VOLUME]", - "test_multiple_disks/test.py::test_alter_move_half_of_partition[DISK]", - "test_multiple_disks/test.py::test_alter_move_half_of_partition[VOLUME]", - "test_multiple_disks/test.py::test_alter_move[mt]", - "test_multiple_disks/test.py::test_alter_policy[mt]", - "test_multiple_disks/test.py::test_alter_policy[replicated]", - "test_multiple_disks/test.py::test_background_move[mt]", - "test_multiple_disks/test.py::test_background_move[replicated]", - "test_multiple_disks/test.py::test_concurrent_alter_modify[mt]", - "test_multiple_disks/test.py::test_concurrent_alter_modify[replicated]", - "test_multiple_disks/test.py::test_concurrent_alter_move_and_drop[mt]", - "test_multiple_disks/test.py::test_concurrent_alter_move_and_drop[replicated]", - "test_multiple_disks/test.py::test_concurrent_alter_move[mt]", - "test_multiple_disks/test.py::test_concurrent_alter_move[replicated]", - "test_multiple_disks/test.py::test_detach_attach[mt]", - "test_multiple_disks/test.py::test_detach_attach[replicated]", - "test_multiple_disks/test.py::test_download_appropriate_disk", - "test_multiple_disks/test.py::test_freeze", - "test_multiple_disks/test.py::test_jbod_overflow[mt]", - "test_multiple_disks/test.py::test_jbod_overflow[replicated]", - "test_multiple_disks/test.py::test_kill_while_insert", - "test_multiple_disks/test.py::test_max_data_part_size[mt]", - "test_multiple_disks/test.py::test_max_data_part_size[replicated]", - "test_multiple_disks/test.py::test_move_across_policies_does_not_work", - "test_multiple_disks/test.py::test_move_while_merge", - "test_multiple_disks/test.py::test_mutate_to_another_disk[mt]", - "test_multiple_disks/test.py::test_mutate_to_another_disk[replicated]", - "test_multiple_disks/test.py::test_no_merges_in_configuration_allow_from_query_without_reload", - "test_multiple_disks/test.py::test_no_merges_in_configuration_allow_from_query_with_reload", - "test_multiple_disks/test.py::test_no_warning_about_zero_max_data_part_size", - "test_multiple_disks/test.py::test_query_parser", - "test_multiple_disks/test.py::test_rename", - "test_multiple_disks/test.py::test_round_robin[mt]", - "test_multiple_disks/test.py::test_round_robin[replicated]", - "test_multiple_disks/test.py::test_simple_merge_tree_merges_are_disabled", - "test_multiple_disks/test.py::test_simple_replication_and_moves", - "test_multiple_disks/test.py::test_start_stop_moves[mt]", - "test_multiple_disks/test.py::test_start_stop_moves[replicated]", - "test_multiple_disks/test.py::test_system_tables", - "test_multiple_disks/test.py::test_yes_merges_in_configuration_disallow_from_query_without_reload", - "test_multiple_disks/test.py::test_yes_merges_in_configuration_disallow_from_query_with_reload", - "test_mutations_hardlinks/test.py::test_delete_and_drop_mutation", - "test_mutations_hardlinks/test.py::test_drop_mutation", - "test_mutations_hardlinks/test.py::test_modify_mutation", - "test_mutations_hardlinks/test.py::test_update_mutation", - "test_mutations_in_partitions_of_merge_tree/test.py::test_trivial_alter_in_partition_merge_tree_without_where", - "test_mutations_in_partitions_of_merge_tree/test.py::test_trivial_alter_in_partition_merge_tree_with_where", - "test_mutations_in_partitions_of_merge_tree/test.py::test_trivial_alter_in_partition_replicated_merge_tree", - "test_mutations_with_merge_tree/test.py::test_mutations_with_merge_background_task", - "test_mutations_with_merge_tree/test.py::test_mutations_with_truncate_table", - "test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine", - "test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database", - "test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database", - "test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database", - "test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine", - "test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_1]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_10]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_11]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_12]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_13]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_14]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_15]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_16]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_17]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_18]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_19]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_2]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_20]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_3]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_4]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_5]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_6]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_7]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_80]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_81]", - "test_mysql_database_engine/test.py::test_mysql_types[common_types_9]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]", - "test_mysql_database_engine/test.py::test_mysql_types[datetime_default]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_2]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_3]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_4]", - "test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]", - "test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]", - "test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]", - "test_mysql_protocol/test.py::test_golang_client", - "test_mysql_protocol/test.py::test_java_client", - "test_mysql_protocol/test.py::test_mysql_affected_rows", - "test_mysql_protocol/test.py::test_mysql_client", - "test_mysql_protocol/test.py::test_mysql_client_exception", - "test_mysql_protocol/test.py::test_mysql_explain", - "test_mysql_protocol/test.py::test_mysql_federated", - "test_mysql_protocol/test.py::test_mysqljs_client", - "test_mysql_protocol/test.py::test_mysql_replacement_query", - "test_mysql_protocol/test.py::test_mysql_set_variables", - "test_mysql_protocol/test.py::test_php_client", - "test_mysql_protocol/test.py::test_python_client", - "test_mysql_protocol/test.py::test_types", - "test_no_local_metadata_node/test.py::test_table_start_without_metadata", - "test_non_default_compression/test.py::test_preconfigured_custom_codec", - "test_non_default_compression/test.py::test_preconfigured_default_codec", - "test_non_default_compression/test.py::test_uncompressed_cache_custom_codec", - "test_non_default_compression/test.py::test_uncompressed_cache_plus_zstd_codec", - "test_odbc_interaction/test.py::test_bridge_dies_with_parent", - "test_odbc_interaction/test.py::test_concurrent_queries", - "test_odbc_interaction/test.py::test_many_connections", - "test_odbc_interaction/test.py::test_mysql_insert", - "test_odbc_interaction/test.py::test_mysql_simple_select_works", - "test_odbc_interaction/test.py::test_odbc_cyrillic_with_varchar", - "test_odbc_interaction/test.py::test_odbc_long_column_names", - "test_odbc_interaction/test.py::test_odbc_long_text", - "test_odbc_interaction/test.py::test_odbc_postgres_conversions", - "test_odbc_interaction/test.py::test_odbc_postgres_date_data_type", - "test_odbc_interaction/test.py::test_postgres_insert", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", - "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", - "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", - "test_odbc_interaction/test.py::test_sqlite_simple_select_function_works", - "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", - "test_odbc_interaction/test.py::test_sqlite_table_function", - "test_old_versions/test.py::test_client_is_older_than_server", - "test_old_versions/test.py::test_distributed_query_initiator_is_older_than_shard", - "test_old_versions/test.py::test_server_is_older_than_client", - "test_on_cluster_timeouts/test.py::test_long_query", - "test_optimize_on_insert/test.py::test_empty_parts_optimize", - "test_partition/test.py::test_attach_check_all_parts", - "test_partition/test.py::test_cannot_attach_active_part", - "test_partition/test.py::test_drop_detached_parts", - "test_partition/test.py::test_partition_complex", - "test_partition/test.py::test_partition_simple", - "test_part_log_table/test.py::test_config_with_non_standard_part_log", - "test_part_log_table/test.py::test_config_without_part_log", - "test_part_log_table/test.py::test_config_with_standard_part_log", - "test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper", - "test_part_uuid/test.py::test_part_uuid", - "test_part_uuid/test.py::test_part_uuid_wal", - "test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster", - "test_polymorphic_parts/test.py::test_compact_parts_only", - "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]", - "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]", - "test_polymorphic_parts/test.py::test_in_memory", - "test_polymorphic_parts/test.py::test_in_memory_alters", - "test_polymorphic_parts/test.py::test_in_memory_deduplication", - "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", - "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_index", - "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", - "test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays", - "test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache", - "test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl", - "test_postgresql_protocol/test.py::test_java_client", - "test_postgresql_protocol/test.py::test_psql_client", - "test_postgresql_protocol/test.py::test_psql_is_ready", - "test_postgresql_protocol/test.py::test_python_client", - "test_profile_events_s3/test.py::test_profile_events", - "test_prometheus_endpoint/test.py::test_prometheus_endpoint", - "test_query_deduplication/test.py::test_no_merge_with_deduplication", - "test_query_deduplication/test.py::test_virtual_column", - "test_query_deduplication/test.py::test_with_deduplication", - "test_query_deduplication/test.py::test_without_deduplication", - "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", - "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum", - "test_quorum_inserts/test.py::test_drop_replica_and_achieve_quorum", - "test_quorum_inserts/test.py::test_insert_quorum_with_drop_partition[False]", - "test_quorum_inserts/test.py::test_insert_quorum_with_drop_partition[True]", - "test_quorum_inserts/test.py::test_insert_quorum_with_move_partition[False]", - "test_quorum_inserts/test.py::test_insert_quorum_with_move_partition[True]", - "test_quorum_inserts/test.py::test_insert_quorum_with_ttl", - "test_quorum_inserts/test.py::test_simple_add_replica", - "test_quota/test.py::test_add_remove_interval", - "test_quota/test.py::test_add_remove_quota", - "test_quota/test.py::test_consumption_of_show_clusters", - "test_quota/test.py::test_consumption_of_show_databases", - "test_quota/test.py::test_consumption_of_show_privileges", - "test_quota/test.py::test_consumption_of_show_processlist", - "test_quota/test.py::test_consumption_of_show_tables", - "test_quota/test.py::test_dcl_introspection", - "test_quota/test.py::test_dcl_management", - "test_quota/test.py::test_exceed_quota", - "test_quota/test.py::test_query_inserts", - "test_quota/test.py::test_quota_from_users_xml", - "test_quota/test.py::test_reload_users_xml_by_timer", - "test_quota/test.py::test_simpliest_quota", - "test_quota/test.py::test_tracking_quota", - "test_quota/test.py::test_users_xml_is_readonly", - "test_random_inserts/test.py::test_insert_multithreaded", - "test_random_inserts/test.py::test_random_inserts", - "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", - "test_read_temporary_tables_on_failure/test.py::test_different_versions", - "test_recompression_ttl/test.py::test_recompression_multiple_ttls", - "test_recompression_ttl/test.py::test_recompression_replicated", - "test_recompression_ttl/test.py::test_recompression_simple", - "test_recovery_replica/test.py::test_choose_source_replica", - "test_recovery_replica/test.py::test_recovery", - "test_recovery_replica/test.py::test_update_metadata", - "test_redirect_url_storage/test.py::test_url_with_globs", - "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", - "test_redirect_url_storage/test.py::test_url_without_redirect", - "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", - "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", - "test_relative_filepath/test.py::test_filepath", - "test_reload_auxiliary_zookeepers/test.py::test_reload_auxiliary_zookeepers", - "test_reload_clusters_config/test.py::test_add_cluster", - "test_reload_clusters_config/test.py::test_delete_cluster", - "test_reload_clusters_config/test.py::test_simple_reload", - "test_reload_clusters_config/test.py::test_update_one_cluster", - "test_reloading_settings_from_users_xml/test.py::test_force_reload", - "test_reloading_settings_from_users_xml/test.py::test_reload_on_timeout", - "test_reloading_settings_from_users_xml/test.py::test_unexpected_setting_enum", - "test_reloading_settings_from_users_xml/test.py::test_unexpected_setting_int", - "test_reloading_settings_from_users_xml/test.py::test_unknown_setting_force_reload", - "test_reloading_settings_from_users_xml/test.py::test_unknown_setting_reload_on_timeout", - "test_reloading_storage_configuration/test.py::test_add_disk", - "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", - "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", - "test_reloading_storage_configuration/test.py::test_add_policy", - "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", - "test_reloading_storage_configuration/test.py::test_new_policy_works", - "test_reloading_storage_configuration/test.py::test_remove_disk", - "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", - "test_reloading_storage_configuration/test.py::test_remove_policy", - "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", - "test_reload_max_table_size_to_drop/test.py::test_reload_max_table_size_to_drop", - "test_reload_zookeeper/test.py::test_reload_zookeeper", - "test_remote_prewhere/test.py::test_remote", - "test_rename_column/test.py::test_rename_distributed", - "test_rename_column/test.py::test_rename_distributed_parallel_insert_and_select", - "test_rename_column/test.py::test_rename_parallel", - "test_rename_column/test.py::test_rename_parallel_same_node", - "test_rename_column/test.py::test_rename_with_parallel_insert", - "test_rename_column/test.py::test_rename_with_parallel_merges", - "test_rename_column/test.py::test_rename_with_parallel_moves", - "test_rename_column/test.py::test_rename_with_parallel_select", - "test_rename_column/test.py::test_rename_with_parallel_slow_insert", - "test_rename_column/test.py::test_rename_with_parallel_slow_select", - "test_rename_column/test.py::test_rename_with_parallel_ttl_delete", - "test_rename_column/test.py::test_rename_with_parallel_ttl_move", - "test_replace_partition/test.py::test_drop_failover", - "test_replace_partition/test.py::test_normal_work", - "test_replace_partition/test.py::test_replace_after_replace_failover", - "test_replica_can_become_leader/test.py::test_can_become_leader", - "test_replicated_database/test.py::test_alters_from_different_replicas", - "test_replicated_database/test.py::test_create_replicated_table", - "test_replicated_database/test.py::test_recover_staled_replica", - "test_replicated_database/test.py::test_simple_alter_table[MergeTree]", - "test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]", - "test_replicated_database/test.py::test_startup_without_zk", - "test_replicated_fetches_timeouts/test.py::test_no_stall", - "test_replicated_merge_tree_config/test.py::test_replicated_merge_tree_settings", - "test_replicated_merge_tree_s3/test.py::test_insert_select_replicated[0-15]", - "test_replicated_merge_tree_s3/test.py::test_insert_select_replicated[8192-11]", - "test_replicated_merge_tree_s3_zero_copy/test.py::test_insert_select_replicated[0-15]", - "test_replicated_merge_tree_s3_zero_copy/test.py::test_insert_select_replicated[8192-11]", - "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_create_replicated_merge_tree_with_auxiliary_zookeeper", - "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_create_replicated_merge_tree_with_default_zookeeper", - "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_create_replicated_merge_tree_with_not_exists_auxiliary_zookeeper", - "test_replicated_merge_tree_with_auxiliary_zookeepers/test.py::test_drop_replicated_merge_tree_with_auxiliary_zookeeper", - "test_replicated_mutations/test.py::test_mutations", - "test_replicated_mutations/test.py::test_mutations_dont_prevent_merges[nodes0]", - "test_replicated_mutations/test.py::test_mutations_dont_prevent_merges[nodes1]", - "test_replicated_parse_zk_metadata/test.py::test_replicated_engine_parse_metadata_on_attach", - "test_replicating_constants/test.py::test_different_versions", - "test_replication_credentials/test.py::test_credentials_and_no_credentials", - "test_replication_credentials/test.py::test_different_credentials", - "test_replication_credentials/test.py::test_no_credentials", - "test_replication_credentials/test.py::test_same_credentials", - "test_replication_without_zookeeper/test.py::test_startup_without_zookeeper", - "test_role/test.py::test_admin_option", - "test_role/test.py::test_combine_privileges", - "test_role/test.py::test_create_role", - "test_role/test.py::test_grant_role_to_role", - "test_role/test.py::test_introspection", - "test_role/test.py::test_revoke_requires_admin_option", - "test_row_policy/test.py::test_cannot_trick_row_policy_with_keyword_with", - "test_row_policy/test.py::test_change_of_users_xml_changes_row_policies", - "test_row_policy/test.py::test_dcl_introspection", - "test_row_policy/test.py::test_dcl_management", - "test_row_policy/test.py::test_introspection", - "test_row_policy/test.py::test_join", - "test_row_policy/test.py::test_miscellaneous_engines", - "test_row_policy/test.py::test_policy_from_users_xml_affects_only_user_assigned", - "test_row_policy/test.py::test_reload_users_xml_by_timer", - "test_row_policy/test.py::test_smoke", - "test_row_policy/test.py::test_tags_with_db_and_table_names", - "test_row_policy/test.py::test_throwif_error_in_prewhere_with_same_condition_as_filter", - "test_row_policy/test.py::test_throwif_error_in_where_with_same_condition_as_filter", - "test_row_policy/test.py::test_throwif_in_prewhere_doesnt_expose_restricted_data", - "test_row_policy/test.py::test_throwif_in_where_doesnt_expose_restricted_data", - "test_row_policy/test.py::test_users_xml_is_readonly", - "test_row_policy/test.py::test_with_prewhere", - "test_s3_cluster/test.py::test_count", - "test_s3_cluster/test.py::test_select_all", - "test_s3_cluster/test.py::test_union_all", - "test_s3_cluster/test.py::test_wrong_cluster", - "test_s3_with_https/test.py::test_s3_with_https[s3_secure]", - "test_s3_with_https/test.py::test_s3_with_https[s3_secure_with_proxy]", - "test_s3_with_proxy/test.py::test_s3_with_proxy_list[s3]", - "test_s3_with_proxy/test.py::test_s3_with_proxy_list[s3_with_resolver]", - "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", - "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_replication[s3]", - "test_secure_socket/test.py::test", - "test_select_access_rights/test.py::test_alias", - "test_select_access_rights/test.py::test_alias_columns", - "test_select_access_rights/test.py::test_materialized_columns", - "test_select_access_rights/test.py::test_select_all_columns", - "test_select_access_rights/test.py::test_select_all_columns_with_table_grant", - "test_select_access_rights/test.py::test_select_count", - "test_select_access_rights/test.py::test_select_join", - "test_select_access_rights/test.py::test_select_single_column", - "test_select_access_rights/test.py::test_select_single_column_with_table_grant", - "test_select_access_rights/test.py::test_select_union", - "test_send_crash_reports/test.py::test_send_segfault", - "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case1]", - "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case2]", - "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case3]", - "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case4]", - "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case5]", - "test_send_request_to_leader_replica/test.py::test_alter_table_drop_partition[case6]", - "test_server_initialization/test.py::test_live_view_dependency", - "test_server_initialization/test.py::test_partially_dropped_tables", - "test_server_initialization/test.py::test_sophisticated_default", - "test_settings_constraints_distributed/test.py::test_insert_clamps_settings", - "test_settings_constraints_distributed/test.py::test_select_clamps_settings", - "test_settings_constraints/test.py::test_max_constraint", - "test_settings_constraints/test.py::test_min_constraint", - "test_settings_constraints/test.py::test_read_only_constraint", - "test_settings_constraints/test.py::test_system_constraints", - "test_settings_constraints/test.py::test_system_settings", - "test_settings_profile/test.py::test_allow_ddl", - "test_settings_profile/test.py::test_allow_introspection", - "test_settings_profile/test.py::test_alter_and_drop", - "test_settings_profile/test.py::test_inheritance", - "test_settings_profile/test.py::test_settings_from_granted_role", - "test_settings_profile/test.py::test_show_profiles", - "test_settings_profile/test.py::test_smoke", "test_storage_hdfs/test.py::test_bad_hdfs_uri", "test_storage_hdfs/test.py::test_globs_in_read_table", "test_storage_hdfs/test.py::test_read_files_with_spaces", @@ -1128,101 +102,5 @@ "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", "test_storage_s3/test.py::test_storage_s3_put_uncompressed", "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", - "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", - "test_system_clusters_actual_information/test.py::test", - "test_system_ddl_worker_queue/test.py::test_distributed_ddl_queue", - "test_system_flush_logs/test.py::test_system_logs_non_empty_queue", - "test_system_flush_logs/test.py::test_system_logs[system.metric_log-1]", - "test_system_flush_logs/test.py::test_system_logs[system.part_log-0]", - "test_system_flush_logs/test.py::test_system_logs[system.query_log-1]", - "test_system_flush_logs/test.py::test_system_logs[system.query_thread_log-1]", - "test_system_flush_logs/test.py::test_system_logs[system.text_log-0]", - "test_system_flush_logs/test.py::test_system_logs[system.trace_log-1]", - "test_system_merges/test.py::test_merge_simple[]", - "test_system_merges/test.py::test_merge_simple[replicated]", - "test_system_merges/test.py::test_mutation_simple[]", - "test_system_merges/test.py::test_mutation_simple[replicated]", - "test_system_metrics/test.py::test_readonly_metrics", - "test_system_queries/test.py::test_DROP_DNS_CACHE", - "test_system_queries/test.py::test_RELOAD_CONFIG_AND_MACROS", - "test_system_queries/test.py::test_system_flush_logs", - "test_system_queries/test.py::test_SYSTEM_RELOAD_DICTIONARY", - "test_system_replicated_fetches/test.py::test_system_replicated_fetches", - "test_table_functions_access_rights/test.py::test_merge", - "test_text_log_level/test.py::test_basic", - "test_timezone_config/test.py::test_check_timezone_config", - "test_tmp_policy/test.py::test_different_versions", - "test_ttl_move/test.py::test_alter_multiple_ttls[negative]", - "test_ttl_move/test.py::test_alter_multiple_ttls[positive]", - "test_ttl_move/test.py::test_alter_multiple_ttls[replicated_negative]", - "test_ttl_move/test.py::test_alter_multiple_ttls[replicated_positive]", - "test_ttl_move/test.py::test_alter_with_merge_work[mt]", - "test_ttl_move/test.py::test_alter_with_merge_work[mt_work]", - "test_ttl_move/test.py::test_alter_with_merge_work[replicated]", - "test_ttl_move/test.py::test_alter_with_merge_work[replicated_work]", - "test_ttl_move/test.py::test_concurrent_alter_with_ttl_move[mt]", - "test_ttl_move/test.py::test_concurrent_alter_with_ttl_move[replicated_mt]", - "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[disk]", - "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[replicated_disk]", - "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[replicated_volume]", - "test_ttl_move/test.py::test_disabled_ttl_move_on_insert[volume]", - "test_ttl_move/test.py::test_inserts_to_disk_work[mt_test_inserts_to_disk_do_not_work]", - "test_ttl_move/test.py::test_inserts_to_disk_work[mt_test_inserts_to_disk_work_1]", - "test_ttl_move/test.py::test_inserts_to_disk_work[replicated_mt_test_inserts_to_disk_do_not_work]", - "test_ttl_move/test.py::test_inserts_to_disk_work[replicated_mt_test_inserts_to_disk_work_1]", - "test_ttl_move/test.py::test_inserts_to_volume_work[mt_test_inserts_to_volume_do_not_work]", - "test_ttl_move/test.py::test_inserts_to_volume_work[mt_test_inserts_to_volume_work]", - "test_ttl_move/test.py::test_inserts_to_volume_work[replicated_mt_test_inserts_to_volume_do_not_work]", - "test_ttl_move/test.py::test_inserts_to_volume_work[replicated_mt_test_inserts_to_volume_work]", - "test_ttl_move/test.py::test_materialize_ttl_in_partition[mt]", - "test_ttl_move/test.py::test_materialize_ttl_in_partition[replicated]", - "test_ttl_move/test.py::test_merges_to_disk_work[mt_test_merges_to_disk_do_not_work0]", - "test_ttl_move/test.py::test_merges_to_disk_work[mt_test_merges_to_disk_do_not_work1]", - "test_ttl_move/test.py::test_merges_to_disk_work[mt_test_merges_to_disk_work]", - "test_ttl_move/test.py::test_merges_to_disk_work[replicated_mt_test_merges_to_disk_work]", - "test_ttl_move/test.py::test_merges_with_full_disk_work[mt_test_merges_with_full_disk_work]", - "test_ttl_move/test.py::test_merges_with_full_disk_work[replicated_mt_test_merges_with_full_disk_work]", - "test_ttl_move/test.py::test_moves_after_merges_work[mt_test_moves_after_merges_do_not_work]", - "test_ttl_move/test.py::test_moves_after_merges_work[mt_test_moves_after_merges_work]", - "test_ttl_move/test.py::test_moves_after_merges_work[replicated_mt_test_moves_after_merges_do_not_work]", - "test_ttl_move/test.py::test_moves_after_merges_work[replicated_mt_test_moves_after_merges_work]", - "test_ttl_move/test.py::test_moves_to_disk_eventually_work[mt_test_moves_to_disk_eventually_work]", - "test_ttl_move/test.py::test_moves_to_disk_eventually_work[replicated_mt_test_moves_to_disk_eventually_work]", - "test_ttl_move/test.py::test_moves_to_disk_work[mt_test_moves_to_disk_do_not_work]", - "test_ttl_move/test.py::test_moves_to_disk_work[mt_test_moves_to_disk_work]", - "test_ttl_move/test.py::test_moves_to_disk_work[replicated_mt_test_moves_to_disk_do_not_work]", - "test_ttl_move/test.py::test_moves_to_disk_work[replicated_mt_test_moves_to_disk_work]", - "test_ttl_move/test.py::test_moves_to_volume_work[mt_test_moves_to_volume_work]", - "test_ttl_move/test.py::test_moves_to_volume_work[replicated_mt_test_moves_to_volume_work]", - "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", - "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]", - "test_ttl_move/test.py::test_replicated_download_ttl_info", - "test_ttl_move/test.py::test_rule_with_invalid_destination[case0]", - "test_ttl_move/test.py::test_rule_with_invalid_destination[case1]", - "test_ttl_move/test.py::test_rule_with_invalid_destination[case2]", - "test_ttl_move/test.py::test_rule_with_invalid_destination[case3]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_external_negative]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_external_positive]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_negative]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[mt_positive]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[repicated_negative]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[repicated_positive]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[replicated_external_negative]", - "test_ttl_move/test.py::test_ttls_do_not_work_after_alter[replicated_external_positive]", - "test_union_header/test.py::test_read", - "test_user_directories/test.py::test_duplicates", - "test_user_directories/test.py::test_local_directories", - "test_user_directories/test.py::test_memory", - "test_user_directories/test.py::test_mixed_style", - "test_user_directories/test.py::test_old_style", - "test_user_directories/test.py::test_relative_path", - "test_user_ip_restrictions/test.py::test_ipv4", - "test_user_ip_restrictions/test.py::test_ipv6", - "test_user_zero_database_access/test_user_zero_database_access.py::test_user_zero_database_access", - "test_version_update_after_mutation/test.py::test_mutate_and_upgrade", - "test_version_update_after_mutation/test.py::test_upgrade_while_mutation", - "test_zookeeper_config/test.py::test_chroot_with_different_root", - "test_zookeeper_config/test.py::test_chroot_with_same_root", - "test_zookeeper_config/test.py::test_identity", - "test_zookeeper_config/test.py::test_secure_connection" -] + "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]" + ] From 3c804556fb3822d0b8d4516bf5b259b5cc10638a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 May 2021 14:02:23 +0300 Subject: [PATCH 139/652] fix --- .../integration/runner/compose/docker_compose_rabbitmq.yml | 6 +++--- tests/integration/test_storage_rabbitmq/test.py | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index ef6016761a7..99e0ea8e280 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -2,15 +2,15 @@ version: '2.3' services: rabbitmq1: - image: rabbitmq:3-management + image: rabbitmq:3-management-alpine hostname: rabbitmq1 expose: - ${RABBITMQ_PORT} environment: RABBITMQ_DEFAULT_USER: "root" RABBITMQ_DEFAULT_PASS: "clickhouse" - RABBITMQ_LOGS: /rabbitmq_logs/ + RABBITMQ_LOG_BASE: /rabbitmq_logs/ volumes: - type: ${RABBITMQ_LOGS_FS:-tmpfs} source: ${RABBITMQ_LOGS:-} - target: /rabbitmq_logs/ \ No newline at end of file + target: /rabbitmq_logs/ diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 21fd30faca9..02df75f0f04 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1942,7 +1942,7 @@ def test_rabbitmq_drop_table_properly(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1979,7 +1979,7 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -2012,7 +2012,7 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): @pytest.mark.timeout(120) def test_rabbitmq_queue_consume(rabbitmq_cluster): credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() channel.queue_declare(queue='rabbit_queue', durable=True) From 3ec8e779ba3dc45155d9003b2565de23a17bc3a5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 May 2021 08:54:48 +0300 Subject: [PATCH 140/652] one more timeout --- tests/integration/helpers/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index cfa25a613b9..71d8b512cb1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -51,8 +51,8 @@ def _create_env_file(path, variables): f.write("=".join([var, value]) + "\n") return path -def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE): - res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell) +def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE, timeout=120): + res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell, timeout=timeout) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually logging.debug(f"Stderr:\n{res.stderr.decode('utf-8')}\n") From 7a5895815b681e1b817d466ab53760a888d126e1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 May 2021 13:23:51 +0300 Subject: [PATCH 141/652] remove tmpfs for now --- tests/integration/ci-runner.py | 2 +- tests/integration/helpers/cluster.py | 7 +- tests/integration/parallel.json | 209 +++++++++++++++++- .../configs/.gitignore | 2 + .../configs/dictionaries/.gitignore | 2 - 5 files changed, 213 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_dictionaries_redis/configs/.gitignore delete mode 100644 tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 970daa7ec46..8ad804fbe00 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -372,7 +372,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && ./runner {} --tmpfs -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && ./runner {} -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, parallel_cmd, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 71d8b512cb1..e02aa69fefd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -51,7 +51,7 @@ def _create_env_file(path, variables): f.write("=".join([var, value]) + "\n") return path -def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE, timeout=120): +def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE, timeout=180): res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell, timeout=timeout) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually @@ -760,7 +760,7 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - start_deadline = time.time() + 120.0 # seconds + start_deadline = time.time() + 180.0 # seconds node.wait_for_start(start_deadline) return node @@ -1521,7 +1521,6 @@ class ClickHouseInstance: self.docker_client = None self.ip_address = None self.client = None - self.default_timeout = 20.0 # 20 sec self.image = image self.tag = tag self.stay_alive = stay_alive @@ -1718,7 +1717,7 @@ class ClickHouseInstance: return None return None - def restart_with_latest_version(self, stop_start_wait_sec=60, callback_onstop=None, signal=60): + def restart_with_latest_version(self, stop_start_wait_sec=120, callback_onstop=None, signal=60): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root') diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index 3e6ef773c78..58ca2209131 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -1,4 +1,162 @@ [ + "test_dictionaries_access/test.py::test_create", + "test_dictionaries_access/test.py::test_dictget", + "test_dictionaries_access/test.py::test_drop", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", + "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", + "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", + "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", + "test_dictionaries_ddl/test.py::test_clickhouse_remote", + "test_dictionaries_ddl/test.py::test_conflicting_name", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", + "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", + "test_dictionaries_ddl/test.py::test_dictionary_with_where", + "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", + "test_dictionaries_ddl/test.py::test_restricted_database", + "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", + "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", + "test_dictionaries_dependency_xml/test.py::test_get_data", + "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", + "test_dictionaries_null_value/test.py::test_null_value", + "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", + "test_dictionaries_postgresql/test.py::test_invalidate_query", + "test_dictionaries_postgresql/test.py::test_load_dictionaries", + "test_dictionaries_postgresql/test.py::test_postgres_scema", + "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", + "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", + "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", + "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", + "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", + "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", + "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", + "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", + "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", + "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", + "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", + "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", + "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", + "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", + "test_dictionary_custom_settings/test.py::test_work", + "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", + "test_disk_access_storage/test.py::test_alter", + "test_disk_access_storage/test.py::test_create", + "test_disk_access_storage/test.py::test_drop", + "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", + "test_distributed_storage_configuration/test.py::test_insert", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", + "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", + "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", + "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", + "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", + "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", + "test_redirect_url_storage/test.py::test_url_with_globs", + "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", + "test_redirect_url_storage/test.py::test_url_without_redirect", + "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", + "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", + "test_reloading_storage_configuration/test.py::test_add_disk", + "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", + "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", + "test_reloading_storage_configuration/test.py::test_add_policy", + "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", + "test_reloading_storage_configuration/test.py::test_new_policy_works", + "test_reloading_storage_configuration/test.py::test_remove_disk", + "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", + "test_reloading_storage_configuration/test.py::test_remove_policy", + "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", + "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", "test_storage_hdfs/test.py::test_bad_hdfs_uri", "test_storage_hdfs/test.py::test_globs_in_read_table", "test_storage_hdfs/test.py::test_read_files_with_spaces", @@ -13,6 +171,49 @@ "test_storage_hdfs/test.py::test_write_gzip_storage", "test_storage_hdfs/test.py::test_write_gz_storage", "test_storage_hdfs/test.py::test_write_table", + "test_storage_kafka/test.py::test_bad_reschedule", + "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", + "test_storage_kafka/test.py::test_exception_from_destructor", + "test_storage_kafka/test.py::test_kafka_commit_on_block_write", + "test_storage_kafka/test.py::test_kafka_consumer_hang", + "test_storage_kafka/test.py::test_kafka_consumer_hang2", + "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", + "test_storage_kafka/test.py::test_kafka_flush_by_block_size", + "test_storage_kafka/test.py::test_kafka_flush_by_time", + "test_storage_kafka/test.py::test_kafka_flush_on_big_message", + "test_storage_kafka/test.py::test_kafka_formats", + "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", + "test_storage_kafka/test.py::test_kafka_insert", + "test_storage_kafka/test.py::test_kafka_issue11308", + "test_storage_kafka/test.py::test_kafka_issue14202", + "test_storage_kafka/test.py::test_kafka_issue4116", + "test_storage_kafka/test.py::test_kafka_json_as_string", + "test_storage_kafka/test.py::test_kafka_json_without_delimiter", + "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", + "test_storage_kafka/test.py::test_kafka_many_materialized_views", + "test_storage_kafka/test.py::test_kafka_materialized_view", + "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", + "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", + "test_storage_kafka/test.py::test_kafka_produce_consume", + "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", + "test_storage_kafka/test.py::test_kafka_protobuf", + "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", + "test_storage_kafka/test.py::test_kafka_rebalance", + "test_storage_kafka/test.py::test_kafka_select_empty", + "test_storage_kafka/test.py::test_kafka_settings_new_syntax", + "test_storage_kafka/test.py::test_kafka_settings_old_syntax", + "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", + "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_unavailable", + "test_storage_kafka/test.py::test_kafka_virtual_columns", + "test_storage_kafka/test.py::test_kafka_virtual_columns2", + "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", + "test_storage_kafka/test.py::test_librdkafka_compression", + "test_storage_kafka/test.py::test_premature_flush_on_eof", "test_storage_kerberized_hdfs/test.py::test_cache_path", "test_storage_kerberized_hdfs/test.py::test_prohibited", "test_storage_kerberized_hdfs/test.py::test_read_table", @@ -20,6 +221,8 @@ "test_storage_kerberized_hdfs/test.py::test_read_write_storage", "test_storage_kerberized_hdfs/test.py::test_two_users", "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", "test_storage_mongodb/test.py::test_complex_data_type", "test_storage_mongodb/test.py::test_incorrect_data_type", "test_storage_mongodb/test.py::test_simple_select", @@ -102,5 +305,7 @@ "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", "test_storage_s3/test.py::test_storage_s3_put_uncompressed", "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", - "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]" - ] + "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", + "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", + "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]" +] diff --git a/tests/integration/test_dictionaries_redis/configs/.gitignore b/tests/integration/test_dictionaries_redis/configs/.gitignore new file mode 100644 index 00000000000..12657916b22 --- /dev/null +++ b/tests/integration/test_dictionaries_redis/configs/.gitignore @@ -0,0 +1,2 @@ +dictionaries/* +!.gitignore diff --git a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore deleted file mode 100644 index d6b7ef32c84..00000000000 --- a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -* -!.gitignore From 31d1b92fdc6a18d57c7e53575014619e2e913039 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 19 May 2021 16:19:49 +0300 Subject: [PATCH 142/652] Create dictionary.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал табличную функцию dictionary. --- .../table-functions/dictionary.md | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 docs/en/sql-reference/table-functions/dictionary.md diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md new file mode 100644 index 00000000000..e9d78a6fab5 --- /dev/null +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -0,0 +1,60 @@ +--- +toc_priority: 54 +toc_title: dictionary function +--- + +# dictionary {#dictionary-function} + +Displays the [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. + +**Syntax** + +``` sql +dictionary('dict') +``` + +**Arguments** + +- `dict` — A dictionary name. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +A ClickHouse table. + +**Example** + +Input table: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +Create a dictionary: + +``` sql +CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) +LAYOUT(DIRECT()); +``` + +Query: + +``` sql +SELECT * FROM dictionary('table_function_dictionary_test_dictionary'); +``` + +Result: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +**See Also** + +- [Dictionary engine](../../engines/table-engines/special/dictionary.md#dictionary) From b5421209e3707067554e84f298d21d6030b51dba Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 19 May 2021 18:20:52 +0300 Subject: [PATCH 143/652] Fix link MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылку. --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index e9d78a6fab5..608dba01bea 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -5,7 +5,7 @@ toc_title: dictionary function # dictionary {#dictionary-function} -Displays the [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. +Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. **Syntax** From f33b70c54a591697827df7ac675e68518122e5f8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 May 2021 00:42:25 +0300 Subject: [PATCH 144/652] better --- .../engines/table-engines/integrations/s3.md | 4 ++-- src/Core/Settings.h | 2 +- src/Disks/S3/DiskS3.cpp | 18 +++++++-------- src/Disks/S3/DiskS3.h | 4 ++-- src/Disks/S3/registerDiskS3.cpp | 2 +- src/IO/ReadBufferFromS3.cpp | 22 ++++++------------- src/IO/ReadBufferFromS3.h | 8 ++----- src/Storages/StorageS3.cpp | 19 ++++++++-------- src/Storages/StorageS3.h | 10 ++++----- src/TableFunctions/TableFunctionS3.cpp | 6 ++--- src/TableFunctions/TableFunctionS3Cluster.cpp | 6 ++--- 11 files changed, 42 insertions(+), 59 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 78fcfaa7ac4..28ba40e281c 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -145,7 +145,7 @@ The following settings can be specified in configuration file for given endpoint - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be speficied multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. -- `single_read_retry_attempts` — The maximum number of attempts during single read. Default value is `4`. Optional. +- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. **Example:** @@ -160,7 +160,7 @@ The following settings can be specified in configuration file for given endpoint - + ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9af6ba0f3cd..ea2019a4ff1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -70,7 +70,7 @@ class IColumn; M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_single_part_upload_size, 64*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ - M(UInt64, s3_single_read_retry_attempts, 4, "The maximum number of retries during single S3 read.", 0) \ + M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index d1146c0774c..1a2c6444e2a 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -277,12 +277,12 @@ public: std::shared_ptr client_ptr_, const String & bucket_, DiskS3::Metadata metadata_, - std::shared_ptr single_read_retry_strategy_, + size_t max_single_read_retries_, size_t buf_size_) : client_ptr(std::move(client_ptr_)) , bucket(bucket_) , metadata(std::move(metadata_)) - , single_read_retry_strategy(std::move(single_read_retry_strategy_)) + , max_single_read_retries(max_single_read_retries_) , buf_size(buf_size_) { } @@ -339,7 +339,7 @@ private: const auto & [path, size] = metadata.s3_objects[i]; if (size > offset) { - auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, single_read_retry_strategy, buf_size); + auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, max_single_read_retries, buf_size); buf->seek(offset, SEEK_SET); return buf; } @@ -368,7 +368,7 @@ private: ++current_buf_idx; const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, single_read_retry_strategy, buf_size); + current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, max_single_read_retries, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); absolute_position += working_buffer.size(); @@ -379,7 +379,7 @@ private: std::shared_ptr client_ptr; const String & bucket; DiskS3::Metadata metadata; - std::shared_ptr single_read_retry_strategy; + UInt64 max_single_read_retries; size_t buf_size; size_t absolute_position = 0; @@ -686,7 +686,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si LOG_DEBUG(log, "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.s3_objects.size()); - auto reader = std::make_unique(settings->client, bucket, metadata, settings->single_read_retry_strategy, buf_size); + auto reader = std::make_unique(settings->client, bucket, metadata, settings->s3_max_single_read_retries, buf_size); return std::make_unique(std::move(reader), settings->min_bytes_for_seek); } @@ -1000,7 +1000,7 @@ int DiskS3::readSchemaVersion(const String & source_bucket, const String & sourc settings->client, source_bucket, source_path + SCHEMA_VERSION_OBJECT, - settings->single_read_retry_strategy); + settings->s3_max_single_read_retries); readIntText(version, buffer); @@ -1559,7 +1559,7 @@ void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config, DiskS3Settings::DiskS3Settings( const std::shared_ptr & client_, - std::shared_ptr single_read_retry_strategy_, + UInt64 s3_max_single_read_retries_, size_t s3_min_upload_part_size_, size_t s3_max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -1567,7 +1567,7 @@ DiskS3Settings::DiskS3Settings( int thread_pool_size_, int list_object_keys_size_) : client(client_) - , single_read_retry_strategy(single_read_retry_strategy_) + , s3_max_single_read_retries(s3_max_single_read_retries_) , s3_min_upload_part_size(s3_min_upload_part_size_) , s3_max_single_part_upload_size(s3_max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 91113f57441..efc7fdcf643 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -22,7 +22,7 @@ struct DiskS3Settings { DiskS3Settings( const std::shared_ptr & client_, - std::shared_ptr single_read_retry_strategy_, + size_t s3_max_single_read_retries_, size_t s3_min_upload_part_size_, size_t s3_max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -31,7 +31,7 @@ struct DiskS3Settings int list_object_keys_size_); std::shared_ptr client; - std::shared_ptr single_read_retry_strategy; + size_t s3_max_single_read_retries; size_t s3_min_upload_part_size; size_t s3_max_single_part_upload_size; size_t min_bytes_for_seek; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 6dcdba25478..767e8890a01 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -150,7 +150,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigurat { return std::make_unique( getClient(config, config_prefix, context), - std::make_shared(config.getUInt(config_prefix + ".single_read_retry_attempts", context->getSettingsRef().s3_single_read_retry_attempts)), + config.getUInt64(config_prefix + ".s3_max_single_read_retries", context->getSettingsRef().s3_max_single_read_retries), config.getUInt64(config_prefix + ".s3_min_upload_part_size", context->getSettingsRef().s3_min_upload_part_size), config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size), config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 42db3c6e842..5fb5e7f82d7 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -6,7 +6,6 @@ # include # include -# include # include # include # include @@ -32,12 +31,12 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, const String & key_, std::shared_ptr retry_strategy_, size_t buffer_size_) + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, UInt64 max_single_read_retries_, size_t buffer_size_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , key(key_) - , retry_strategy(std::move(retry_strategy_)) + , max_single_read_retries(max_single_read_retries_) , buffer_size(buffer_size_) { } @@ -52,10 +51,9 @@ bool ReadBufferFromS3::nextImpl() Stopwatch watch; bool next_result = false; + auto sleep_time_with_backoff_milliseconds = std::chrono::milliseconds(100); - Aws::Client::AWSError network_error; - - for (int attempt = 1;; ++attempt) + for (size_t attempt = 0; attempt < max_single_read_retries; ++attempt) { try { @@ -68,23 +66,17 @@ bool ReadBufferFromS3::nextImpl() } catch (const Exception & e) { - network_error = Aws::Client::AWSError(Aws::Client::CoreErrors::NETWORK_CONNECTION, e.name(), e.message(), true); - ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1); LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Attempt: {}, Message: {}", bucket, key, getPosition(), attempt, e.message()); impl.reset(); - - if (!retry_strategy->ShouldRetry(network_error, attempt)) - throw; + impl = initialize(); } - std::this_thread::sleep_for(std::chrono::milliseconds(retry_strategy->CalculateDelayBeforeNextRetry(network_error, attempt))); - - if (!impl) - impl = initialize(); + std::this_thread::sleep_for(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; } watch.stop(); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index c84c20da7cd..7e3624dfccd 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -15,10 +15,6 @@ namespace Aws::S3 { class S3Client; } -namespace Aws::Client -{ -class RetryStrategy; -} namespace DB { @@ -31,7 +27,7 @@ private: std::shared_ptr client_ptr; String bucket; String key; - std::shared_ptr retry_strategy; + UInt64 max_single_read_retries; size_t buffer_size; off_t offset = 0; Aws::S3::Model::GetObjectResult read_result; @@ -44,7 +40,7 @@ public: std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - std::shared_ptr retry_strategy_, + UInt64 max_single_read_retries_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); bool nextImpl() override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a4b7b541e15..3ebbd0d0e81 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -25,7 +25,6 @@ #include #include -#include #include #include @@ -168,8 +167,8 @@ StorageS3Source::StorageS3Source( ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, - std::shared_ptr single_read_retry_strategy_, - const String compression_hint_, + UInt64 max_single_read_retries_, + const String & compression_hint_, const std::shared_ptr & client_, const String & bucket_, std::shared_ptr file_iterator_) @@ -180,7 +179,7 @@ StorageS3Source::StorageS3Source( , format(format_) , columns_desc(columns_) , max_block_size(max_block_size_) - , single_read_retry_strategy(std::move(single_read_retry_strategy_)) + , max_single_read_retries(max_single_read_retries_) , compression_hint(compression_hint_) , client(client_) , sample_block(sample_block_) @@ -201,7 +200,7 @@ bool StorageS3Source::initialize() file_path = bucket + "/" + current_key; read_buf = wrapReadBufferWithCompressionMethod( - std::make_unique(client, bucket, current_key, single_read_retry_strategy), chooseCompressionMethod(current_key, compression_hint)); + std::make_unique(client, bucket, current_key, max_single_read_retries), chooseCompressionMethod(current_key, compression_hint)); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size); reader = std::make_shared(input_format); @@ -325,7 +324,7 @@ StorageS3::StorageS3( const String & secret_access_key_, const StorageID & table_id_, const String & format_name_, - std::shared_ptr single_read_retry_strategy_, + UInt64 max_single_read_retries_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -338,7 +337,7 @@ StorageS3::StorageS3( : IStorage(table_id_) , client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later , format_name(format_name_) - , single_read_retry_strategy(std::move(single_read_retry_strategy_)) + , max_single_read_retries(max_single_read_retries_) , min_upload_part_size(min_upload_part_size_) , max_single_part_upload_size(max_single_part_upload_size_) , compression_method(compression_method_) @@ -406,7 +405,7 @@ Pipe StorageS3::read( local_context, metadata_snapshot->getColumns(), max_block_size, - single_read_retry_strategy, + max_single_read_retries, compression_method, client_auth.client, client_auth.uri.bucket, @@ -493,7 +492,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) secret_access_key = engine_args[2]->as().value.safeGet(); } - auto single_read_retry_strategy = std::make_shared(args.getLocalContext()->getSettingsRef().s3_single_read_retry_attempts); + UInt64 max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections; @@ -517,7 +516,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) secret_access_key, args.table_id, format_name, - single_read_retry_strategy, + max_single_read_retries, min_upload_part_size, max_single_part_upload_size, max_connections, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 7ab6d2fe259..177e31975c6 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -55,8 +55,8 @@ public: ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, - std::shared_ptr single_read_retry_strategy_, - const String compression_hint_, + UInt64 max_single_read_retries_, + const String & compression_hint_, const std::shared_ptr & client_, const String & bucket, std::shared_ptr file_iterator_); @@ -72,7 +72,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; - std::shared_ptr single_read_retry_strategy; + UInt64 max_single_read_retries; String compression_hint; std::shared_ptr client; Block sample_block; @@ -103,7 +103,7 @@ public: const String & secret_access_key, const StorageID & table_id_, const String & format_name_, - std::shared_ptr single_read_retry_strategy_, + UInt64 max_single_read_retries_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -150,7 +150,7 @@ private: ClientAuthentificaiton client_auth; String format_name; - std::shared_ptr single_read_retry_strategy; + UInt64 max_single_read_retries; size_t min_upload_part_size; size_t max_single_part_upload_size; String compression_method; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 1e7b0588e8c..83149445a7d 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -12,8 +12,6 @@ #include #include "registerTableFunctions.h" -#include - namespace DB { @@ -86,7 +84,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context { Poco::URI uri (filename); S3::URI s3_uri (uri); - auto single_read_retry_strategy = std::make_shared(context->getSettingsRef().s3_single_read_retry_attempts); + UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; @@ -97,7 +95,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context secret_access_key, StorageID(getDatabaseName(), table_name), format, - single_read_retry_strategy, + max_single_read_retries, min_upload_part_size, max_single_part_upload_size, max_connections, diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 8d16e62e297..93f90bca75a 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -28,8 +28,6 @@ #include #include -#include - namespace DB { @@ -112,7 +110,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( Poco::URI uri (filename); S3::URI s3_uri (uri); /// Actually this parameters are not used - auto single_read_retry_strategy = std::make_shared(context->getSettingsRef().s3_single_read_retry_attempts); + UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; @@ -122,7 +120,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( secret_access_key, StorageID(getDatabaseName(), table_name), format, - single_read_retry_strategy, + max_single_read_retries, min_upload_part_size, max_single_part_upload_size, max_connections, From c3e65c0d277115e48007405d0bc5a65c06f2c6b2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 20 Apr 2021 09:59:30 +0300 Subject: [PATCH 145/652] Async INSERT into Distributed() does support settings Since #4852 --- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index a4aa2779771..c21e62f3612 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -607,7 +607,6 @@ void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, size_t sh void DistributedBlockOutputStream::writeToLocal(const Block & block, size_t repeats) { - /// Async insert does not support settings forwarding yet whereas sync one supports InterpreterInsertQuery interp(query_ast, context); auto block_io = interp.execute(); From 4d737a5481554aa9bd12019de7ec705a7c95b9c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 20 Apr 2021 09:48:42 +0300 Subject: [PATCH 146/652] Respect insert_allow_materialized_columns for INSERT into Distributed() --- .../DistributedBlockOutputStream.cpp | 28 +++++++---- .../DistributedBlockOutputStream.h | 1 + src/Storages/StorageDistributed.cpp | 13 +++-- ...ributed_with_materialized_column.reference | 16 ++++++ ...o_distributed_with_materialized_column.sql | 50 +++++++++++++++++++ 5 files changed, 93 insertions(+), 15 deletions(-) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index c21e62f3612..4f9a610e316 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -102,6 +102,7 @@ DistributedBlockOutputStream::DistributedBlockOutputStream( , query_string(queryToString(query_ast_)) , cluster(cluster_) , insert_sync(insert_sync_) + , allow_materialized(context->getSettingsRef().insert_allow_materialized_columns) , insert_timeout(insert_timeout_) , main_table(main_table_) , log(&Poco::Logger::get("DistributedBlockOutputStream")) @@ -115,7 +116,10 @@ DistributedBlockOutputStream::DistributedBlockOutputStream( Block DistributedBlockOutputStream::getHeader() const { - return metadata_snapshot->getSampleBlock(); + if (!allow_materialized) + return metadata_snapshot->getSampleBlockNonMaterialized(); + else + return metadata_snapshot->getSampleBlock(); } @@ -129,19 +133,21 @@ void DistributedBlockOutputStream::write(const Block & block) { Block ordinary_block{ block }; - /* They are added by the AddingDefaultBlockOutputStream, and we will get - * different number of columns eventually */ - for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) + if (!allow_materialized) { - if (ordinary_block.has(col.name)) + /* They are added by the AddingDefaultBlockOutputStream, and we will get + * different number of columns eventually */ + for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) { - ordinary_block.erase(col.name); - LOG_DEBUG(log, "{}: column {} will be removed, because it is MATERIALIZED", - storage.getStorageID().getNameForLogs(), col.name); + if (ordinary_block.has(col.name)) + { + ordinary_block.erase(col.name); + LOG_DEBUG(log, "{}: column {} will be removed, because it is MATERIALIZED", + storage.getStorageID().getNameForLogs(), col.name); + } } } - if (insert_sync) writeSync(ordinary_block); else @@ -375,7 +381,7 @@ DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobRep /// to resolve tables (in InterpreterInsertQuery::getTable()) auto copy_query_ast = query_ast->clone(); - InterpreterInsertQuery interp(copy_query_ast, job.local_context); + InterpreterInsertQuery interp(copy_query_ast, job.local_context, allow_materialized); auto block_io = interp.execute(); job.stream = block_io.out; @@ -607,7 +613,7 @@ void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, size_t sh void DistributedBlockOutputStream::writeToLocal(const Block & block, size_t repeats) { - InterpreterInsertQuery interp(query_ast, context); + InterpreterInsertQuery interp(query_ast, context, allow_materialized); auto block_io = interp.execute(); diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.h b/src/Storages/Distributed/DistributedBlockOutputStream.h index f574702f35f..6d7ef94b39b 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -94,6 +94,7 @@ private: size_t inserted_rows = 0; bool insert_sync; + bool allow_materialized; /// Sync-related stuff UInt64 insert_timeout; // in seconds diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3a3291c6c48..731ff5835cd 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -158,7 +158,7 @@ ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, co /// The columns list in the original INSERT query is incorrect because inserted blocks are transformed /// to the form of the sample block of the Distributed table. So we rewrite it and add all columns from /// the sample block instead. -ASTPtr createInsertToRemoteTableQuery(const std::string & database, const std::string & table, const Block & sample_block_non_materialized) +ASTPtr createInsertToRemoteTableQuery(const std::string & database, const std::string & table, const Block & sample_block) { auto query = std::make_shared(); query->table_id = StorageID(database, table); @@ -166,7 +166,7 @@ ASTPtr createInsertToRemoteTableQuery(const std::string & database, const std::s auto columns = std::make_shared(); query->columns = columns; query->children.push_back(columns); - for (const auto & col : sample_block_non_materialized) + for (const auto & col : sample_block) columns->children.push_back(std::make_shared(col.name)); return query; @@ -646,11 +646,16 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta bool insert_sync = settings.insert_distributed_sync || settings.insert_shard_id || owned_cluster; auto timeout = settings.insert_distributed_timeout; + Block sample_block; + if (!settings.insert_allow_materialized_columns) + sample_block = metadata_snapshot->getSampleBlockNonMaterialized(); + else + sample_block = metadata_snapshot->getSampleBlock(); + /// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster return std::make_shared( local_context, *this, metadata_snapshot, - createInsertToRemoteTableQuery( - remote_database, remote_table, metadata_snapshot->getSampleBlockNonMaterialized()), + createInsertToRemoteTableQuery(remote_database, remote_table, sample_block), cluster, insert_sync, timeout, StorageID{remote_database, remote_table}); } diff --git a/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference b/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference index 11b42f40c7a..db6946e3b89 100644 --- a/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference +++ b/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference @@ -1,3 +1,4 @@ +insert_allow_materialized_columns=0 insert_distributed_sync=0 2018-08-01 2018-08-01 @@ -12,3 +13,18 @@ insert_distributed_sync=1 2018-08-01 2017-08-01 2018-08-01 2018-08-01 2017-08-01 +insert_allow_materialized_columns=1 +insert_distributed_sync=0 +2018-08-01 +2018-08-01 +2018-08-01 2019-08-01 +2018-08-01 2019-08-01 +2018-08-01 +2018-08-01 2019-08-01 +insert_distributed_sync=1 +2018-08-01 +2018-08-01 +2018-08-01 2019-08-01 +2018-08-01 2019-08-01 +2018-08-01 +2018-08-01 2019-08-01 diff --git a/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql b/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql index 6b70d927204..641e942bb8f 100644 --- a/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql +++ b/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql @@ -1,6 +1,12 @@ DROP TABLE IF EXISTS local_00952; DROP TABLE IF EXISTS distributed_00952; +-- +-- insert_allow_materialized_columns=0 +-- +SELECT 'insert_allow_materialized_columns=0'; +SET insert_allow_materialized_columns=0; + -- -- insert_distributed_sync=0 -- @@ -40,3 +46,47 @@ SELECT date, value FROM local_00952; DROP TABLE distributed_00952; DROP TABLE local_00952; +-- +-- insert_allow_materialized_columns=1 +-- +SELECT 'insert_allow_materialized_columns=1'; +SET insert_allow_materialized_columns=1; + +-- +-- insert_distributed_sync=0 +-- +SELECT 'insert_distributed_sync=0'; +SET insert_distributed_sync=0; + +CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192); +CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand()); + +INSERT INTO distributed_00952 (date, value) VALUES ('2018-08-01', '2019-08-01'); +SYSTEM FLUSH DISTRIBUTED distributed_00952; + +SELECT * FROM distributed_00952; +SELECT date, value FROM distributed_00952; +SELECT * FROM local_00952; +SELECT date, value FROM local_00952; + +DROP TABLE distributed_00952; +DROP TABLE local_00952; + +-- +-- insert_distributed_sync=1 +-- +SELECT 'insert_distributed_sync=1'; +SET insert_distributed_sync=1; + +CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192); +CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand()); + +INSERT INTO distributed_00952 (date, value) VALUES ('2018-08-01', '2019-08-01'); + +SELECT * FROM distributed_00952; +SELECT date, value FROM distributed_00952; +SELECT * FROM local_00952; +SELECT date, value FROM local_00952; + +DROP TABLE distributed_00952; +DROP TABLE local_00952; From 1f8d43235f374edf34c2a6f7e5222503ab46067b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 20 May 2021 21:19:45 +0300 Subject: [PATCH 147/652] better --- tests/integration/helpers/cluster.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f596756cb9e..ff4aba8eba5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1186,6 +1186,15 @@ class ClickHouseCluster: if self.with_zookeeper and self.base_zookeeper_cmd: logging.debug('Setup ZooKeeper') logging.debug(f'Creating internal ZooKeeper dirs: {self.zookeeper_dirs_to_create}') + if self.use_keeper: + for i in range(1,4): + if os.path.exists(self.keeper_instance_dir_prefix + f"{i}"): + shutil.rmtree(self.keeper_instance_dir_prefix + f"{i}") + else: + for i in range(1,3): + if os.path.exists(self.zookeeper_instance_dir_prefix + f"{i}"): + shutil.rmtree(self.zookeeper_instance_dir_prefix + f"{i}") + for dir in self.zookeeper_dirs_to_create: os.makedirs(dir) From e9f7246d2a0c8f480e4b7452f9b1bd6b9df26b6d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 21 May 2021 07:59:16 +0300 Subject: [PATCH 148/652] fixes --- tests/integration/test_dictionaries_redis/test.py | 7 ++++--- tests/integration/test_zookeeper_config/test.py | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index 7fcbd0d0909..e5a51bcb88a 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -1,5 +1,5 @@ import os - +import shutil import pytest from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout @@ -95,8 +95,9 @@ def setup_module(module): cluster = ClickHouseCluster(__file__) - for f in os.listdir(dict_configs_path): - os.remove(os.path.join(dict_configs_path, f)) + if os.path.exists(dict_configs_path): + shutil.rmtree(dict_configs_path) + os.mkdir(dict_configs_path) for i, field in enumerate(FIELDS): DICTIONARIES.append([]) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 86d94f88e8a..9d6018fff2c 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -162,12 +162,12 @@ def test_secure_connection(): node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, + with_zookeeper=True, with_keeper=False, zookeeper_docker_compose_path=docker_compose.name, ) node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, + with_zookeeper=True, with_keeper=False, zookeeper_docker_compose_path=docker_compose.name, ) try: From b079ce5ee46bad2d45d9188b9ccfb6dda9a1fe3e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 21 May 2021 08:01:51 +0300 Subject: [PATCH 149/652] typo --- tests/integration/test_zookeeper_config/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 9d6018fff2c..b5082215e22 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -162,12 +162,12 @@ def test_secure_connection(): node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, with_keeper=False, zookeeper_docker_compose_path=docker_compose.name, + with_zookeeper=True, use_keeper=False, zookeeper_docker_compose_path=docker_compose.name, ) node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, with_keeper=False, zookeeper_docker_compose_path=docker_compose.name, + with_zookeeper=True, use_keeper=False, zookeeper_docker_compose_path=docker_compose.name, ) try: From 70cb85a5d0cfe59eb723fd815b85751ea8cf34b9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 21 May 2021 12:09:10 +0300 Subject: [PATCH 150/652] better --- .../configs/zookeeper.xml | 17 ----------------- .../test_reload_auxiliary_zookeepers/test.py | 6 +++--- 2 files changed, 3 insertions(+), 20 deletions(-) delete mode 100644 tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml deleted file mode 100644 index 8157cc1d30b..00000000000 --- a/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml +++ /dev/null @@ -1,17 +0,0 @@ - - - - zoo1 - 2181 - - - zoo2 - 2181 - - - zoo3 - 2181 - - 2000 - - diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index 1b14408bc12..232964931c4 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -6,7 +6,7 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException from helpers.test_tools import assert_eq_with_retry -cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", with_zookeeper=True) @@ -60,7 +60,7 @@ def test_reload_auxiliary_zookeepers(start_cluster): """ - node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper_config.xml", new_config) node.query("SYSTEM RELOAD CONFIG") @@ -79,7 +79,7 @@ def test_reload_auxiliary_zookeepers(start_cluster): 2000 """ - node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper_config.xml", new_config) node.query("SYSTEM RELOAD CONFIG") with pytest.raises(QueryRuntimeException): node.query( From daf0b5ad2bb9196e76d7fae329b83953a708c1b7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 21 May 2021 16:29:43 +0300 Subject: [PATCH 151/652] rewrite zookeeper test --- docker/test/integration/runner/Dockerfile | 3 + .../docker_compose_zookeeper_secure.yml | 75 ++++++ .../integration/runner/dockerd-entrypoint.sh | 11 +- .../test/integration/runner/misc/client.crt | 19 ++ .../runner/misc}/zookeeper-ssl-entrypoint.sh | 0 tests/integration/helpers/cluster.py | 95 ++++++-- .../helpers/zookeeper_secure_config.xml | 17 ++ .../configs/remote_servers.xml | 6 + .../integration/test_zookeeper_config/test.py | 224 ++++-------------- .../test_zookeeper_config/test_password.py | 34 +++ .../test_zookeeper_config/test_secure.py | 52 ++++ 11 files changed, 334 insertions(+), 202 deletions(-) create mode 100644 docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml create mode 100644 docker/test/integration/runner/misc/client.crt rename {tests/integration/helpers => docker/test/integration/runner/misc}/zookeeper-ssl-entrypoint.sh (100%) create mode 100644 tests/integration/helpers/zookeeper_secure_config.xml create mode 100644 tests/integration/test_zookeeper_config/test_password.py create mode 100644 tests/integration/test_zookeeper_config/test_secure.py diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 1c538058243..164058eb72e 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -31,6 +31,8 @@ RUN apt-get update \ libkrb5-dev \ krb5-user \ g++ \ + dbus \ + firewalld \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ @@ -84,6 +86,7 @@ RUN python3 -m pip install \ COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ COPY compose/ /compose/ +COPY misc/ /misc/ RUN set -x \ && addgroup --system dockremap \ diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml new file mode 100644 index 00000000000..7a1c32e0023 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml @@ -0,0 +1,75 @@ +version: '2.3' +services: + zoo1: + image: zookeeper:3.6.2 + restart: always + environment: + ZOO_TICK_TIME: 500 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 + ZOO_MY_ID: 1 + JVMFLAGS: -Dzookeeper.forceSync=no + ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT + command: ["zkServer.sh", "start-foreground"] + entrypoint: /zookeeper-ssl-entrypoint.sh + volumes: + - type: bind + source: /misc/zookeeper-ssl-entrypoint.sh + target: /zookeeper-ssl-entrypoint.sh + - type: bind + source: /misc/client.crt + target: /clickhouse-config/client.crt + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA1:-} + target: /data + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA_LOG1:-} + target: /datalog + zoo2: + image: zookeeper:3.6.2 + restart: always + environment: + ZOO_TICK_TIME: 500 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888 + ZOO_MY_ID: 2 + JVMFLAGS: -Dzookeeper.forceSync=no + ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT + + command: ["zkServer.sh", "start-foreground"] + entrypoint: /zookeeper-ssl-entrypoint.sh + volumes: + - type: bind + source: /misc/zookeeper-ssl-entrypoint.sh + target: /zookeeper-ssl-entrypoint.sh + - type: bind + source: /misc/client.crt + target: /clickhouse-config/client.crt + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA2:-} + target: /data + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA_LOG2:-} + target: /datalog + zoo3: + image: zookeeper:3.6.2 + restart: always + environment: + ZOO_TICK_TIME: 500 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 + ZOO_MY_ID: 3 + JVMFLAGS: -Dzookeeper.forceSync=no + ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT + command: ["zkServer.sh", "start-foreground"] + entrypoint: /zookeeper-ssl-entrypoint.sh + volumes: + - type: bind + source: /misc/zookeeper-ssl-entrypoint.sh + target: /zookeeper-ssl-entrypoint.sh + - type: bind + source: /misc/client.crt + target: /clickhouse-config/client.crt + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA3:-} + target: /data + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA_LOG3:-} + target: /datalog diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index a04a88f541d..2ae51fa1a04 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -2,15 +2,18 @@ set -e mkdir -p /etc/docker/ -cat > /etc/docker/daemon.json << EOF -{ +echo '{ "ipv6": true, "fixed-cidr-v6": "fd00::/8", "ip-forward": true, + "log-level": "debug", + "storage-driver": "overlay2", "insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"], "registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"] -} -EOF +}' | dd of=/etc/docker/daemon.json + +/etc/init.d/dbus start +firewalld --debug --log-file /ClickHouse/tests/integration/firewalld.log dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & diff --git a/docker/test/integration/runner/misc/client.crt b/docker/test/integration/runner/misc/client.crt new file mode 100644 index 00000000000..7ade2d96273 --- /dev/null +++ b/docker/test/integration/runner/misc/client.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow +FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB +CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI +4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T +4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU +7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj +sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg +pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC +kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw +DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j +4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr +85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C +L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD ++UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L +P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp +0g== +-----END CERTIFICATE----- diff --git a/tests/integration/helpers/zookeeper-ssl-entrypoint.sh b/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh similarity index 100% rename from tests/integration/helpers/zookeeper-ssl-entrypoint.sh rename to docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ff4aba8eba5..e0d8b01eda0 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -98,7 +98,7 @@ def subprocess_check_call(args): def subprocess_call(args): # Uncomment for debugging..; # logging.debug('run:', ' ' . join(args)) - subprocess.call(args) + run_and_check(args) def get_odbc_bridge_path(): path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH') @@ -174,7 +174,8 @@ class ClickHouseCluster: """ def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, - odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): + odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None, + zookeeper_keyfile=None, zookeeper_certfile=None): for param in list(os.environ.keys()): logging.debug("ENV %40s %s" % (param, os.environ[param])) self.base_dir = p.dirname(base_path) @@ -220,6 +221,7 @@ class ClickHouseCluster: self.pre_zookeeper_commands = [] self.instances = {} self.with_zookeeper = False + self.with_zookeeper_secure = False self.with_mysql_client = False self.with_mysql = False self.with_mysql8 = False @@ -344,8 +346,14 @@ class ClickHouseCluster: self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8")) self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs") + # available when with_zookeper_secure == True + self.zookeeper_secure_port = 2281 + self.zookeeper_keyfile = zookeeper_keyfile + self.zookeeper_certfile = zookeeper_certfile + # available when with_zookeper == True self.use_keeper = True + self.zookeeper_port = 2181 self.keeper_instance_dir_prefix = p.join(p.abspath(self.instances_dir), "keeper") # if use_keeper = True self.zookeeper_instance_dir_prefix = p.join(self.instances_dir, "zk") self.zookeeper_dirs_to_create = [] @@ -406,6 +414,25 @@ class ClickHouseCluster: cmd += " client" return cmd + def setup_zookeeper_secure_cmd(self, instance, env_variables, docker_compose_yml_dir): + logging.debug('Setup ZooKeeper Secure') + zookeeper_docker_compose_path = p.join(docker_compose_yml_dir, 'docker_compose_zookeeper_secure.yml') + env_variables['ZOO_SECURE_CLIENT_PORT'] = str(self.zookeeper_secure_port) + env_variables['ZK_FS'] = 'bind' + for i in range(1, 4): + zk_data_path = os.path.join(self.zookeeper_instance_dir_prefix + str(i), "data") + zk_log_path = os.path.join(self.zookeeper_instance_dir_prefix + str(i), "log") + env_variables['ZK_DATA' + str(i)] = zk_data_path + env_variables['ZK_DATA_LOG' + str(i)] = zk_log_path + self.zookeeper_dirs_to_create += [zk_data_path, zk_log_path] + logging.debug(f"DEBUG ZK: {self.zookeeper_dirs_to_create}") + + self.with_zookeeper_secure = True + self.base_cmd.extend(['--file', zookeeper_docker_compose_path]) + self.base_zookeeper_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', zookeeper_docker_compose_path] + return self.base_zookeeper_cmd + def setup_zookeeper_cmd(self, instance, env_variables, docker_compose_yml_dir): logging.debug('Setup ZooKeeper') zookeeper_docker_compose_path = p.join(docker_compose_yml_dir, 'docker_compose_zookeeper.yml') @@ -631,8 +658,8 @@ class ClickHouseCluster: def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, - macros=None, - with_zookeeper=False, with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, + macros=None, with_zookeeper=False, with_zookeeper_secure=False, + with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_postgres_cluster=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -646,6 +673,7 @@ class ClickHouseCluster: main_configs - a list of config files that will be added to config.d/ directory user_configs - a list of config files that will be added to users.d/ directory with_zookeeper - if True, add ZooKeeper configuration to configs and ZooKeeper instances to the cluster. + with_zookeeper_secure - if True, add ZooKeeper Secure configuration to configs and ZooKeeper instances to the cluster. """ if self.is_up: @@ -715,6 +743,9 @@ class ClickHouseCluster: self.base_cmd.extend(['--file', instance.docker_compose_path]) cmds = [] + if with_zookeeper_secure and not self.with_zookeeper_secure: + cmds.append(self.setup_zookeeper_secure_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_zookeeper and not self.with_zookeeper: if self.use_keeper: cmds.append(self.setup_keeper_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -824,8 +855,8 @@ class ClickHouseCluster: def get_instance_ip(self, instance_name): logging.debug("get_instance_ip instance_name={}".format(instance_name)) docker_id = self.get_instance_docker_id(instance_name) - for cont in self.docker_client.containers.list(): - logging.debug("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) + # for cont in self.docker_client.containers.list(): + # logging.debug("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) handle = self.docker_client.containers.get(docker_id) return list(handle.attrs['NetworkSettings']['Networks'].values())[0]['IPAddress'] @@ -1000,12 +1031,29 @@ class ClickHouseCluster: raise Exception("Cannot wait RabbitMQ container") - def wait_zookeeper_to_start(self, timeout=180): + def wait_zookeeper_secure_to_start(self, timeout=20): + logging.debug("Wait ZooKeeper Secure to start") start = time.time() while time.time() - start < timeout: try: for instance in ['zoo1', 'zoo2', 'zoo3']: - conn = self.get_kazoo_client(instance) + conn = self.get_kazoo_client(instance, self.zookeeper_secure_port, use_ssl=True) + conn.get_children('/') + logging.debug("All instances of ZooKeeper Secure started") + return + except Exception as ex: + logging.debug("Can't connect to ZooKeeper secure " + str(ex)) + time.sleep(0.5) + + raise Exception("Cannot wait ZooKeeper secure container") + + def wait_zookeeper_to_start(self, timeout=180): + logging.debug("Wait ZooKeeper to start") + start = time.time() + while time.time() - start < timeout: + try: + for instance in ['zoo1', 'zoo2', 'zoo3']: + conn = self.get_kazoo_client(instance, self.zookeeper_port) conn.get_children('/') logging.debug("All instances of ZooKeeper started") return @@ -1183,6 +1231,20 @@ class ClickHouseCluster: common_opts = ['up', '-d'] + if self.with_zookeeper_secure and self.base_zookeeper_cmd: + logging.debug('Setup ZooKeeper Secure') + logging.debug(f'Creating internal ZooKeeper dirs: {self.zookeeper_dirs_to_create}') + for i in range(1,3): + if os.path.exists(self.zookeeper_instance_dir_prefix + f"{i}"): + shutil.rmtree(self.zookeeper_instance_dir_prefix + f"{i}") + for dir in self.zookeeper_dirs_to_create: + os.makedirs(dir) + run_and_check(self.base_zookeeper_cmd + common_opts, env=self.env) + + self.wait_zookeeper_secure_to_start() + for command in self.pre_zookeeper_commands: + self.run_kazoo_commands_with_retries(command, repeats=5, secure=True) + if self.with_zookeeper and self.base_zookeeper_cmd: logging.debug('Setup ZooKeeper') logging.debug(f'Creating internal ZooKeeper dirs: {self.zookeeper_dirs_to_create}') @@ -1204,9 +1266,9 @@ class ClickHouseCluster: run_and_check(self.base_zookeeper_cmd + common_opts, env=self.env) + self.wait_zookeeper_to_start() for command in self.pre_zookeeper_commands: self.run_kazoo_commands_with_retries(command, repeats=5) - self.wait_zookeeper_to_start() if self.with_mysql_client and self.base_mysql_client_cmd: logging.debug('Setup MySQL Client') @@ -1423,23 +1485,24 @@ class ClickHouseCluster: def open_bash_shell(self, instance_name): os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash'])) - def get_kazoo_client(self, zoo_instance_name): + def get_kazoo_client(self, zoo_instance_name, port, use_ssl=False): ip = self.get_instance_ip(zoo_instance_name) - logging.debug(f"get_kazoo_client: {zoo_instance_name}, ip:{ip}") - zk = KazooClient(hosts=ip) + logging.debug(f"get_kazoo_client: {zoo_instance_name}, ip:{ip}, port:{port}, use_ssl:{use_ssl}") + zk = KazooClient(hosts=f"{ip}:{port}", use_ssl=use_ssl, verify_certs=False, certfile=self.zookeeper_certfile, + keyfile=self.zookeeper_keyfile) zk.start() return zk - def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1): + def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1, secure=False): + logging.debug(f"run_kazoo_commands_with_retries: {zoo_instance_name}, {secure}, {kazoo_callback}") for i in range(repeats - 1): try: - kazoo_callback(self.get_kazoo_client(zoo_instance_name)) + kazoo_callback(self.get_kazoo_client(zoo_instance_name, self.zookeeper_secure_port if secure else self.zookeeper_port)) return except KazooException as e: logging.debug(repr(e)) time.sleep(sleep_for) - - kazoo_callback(self.get_kazoo_client(zoo_instance_name)) + kazoo_callback(self.get_kazoo_client(zoo_instance_name, self.zookeeper_secure_port if secure else self.zookeeper_port)) def add_zookeeper_startup_command(self, command): self.pre_zookeeper_commands.append(command) diff --git a/tests/integration/helpers/zookeeper_secure_config.xml b/tests/integration/helpers/zookeeper_secure_config.xml new file mode 100644 index 00000000000..5dbb91d42c6 --- /dev/null +++ b/tests/integration/helpers/zookeeper_secure_config.xml @@ -0,0 +1,17 @@ + + + + zoo1 + 2281 + + + zoo2 + 2281 + + + zoo3 + 2281 + + 3000 + + diff --git a/tests/integration/test_zookeeper_config/configs/remote_servers.xml b/tests/integration/test_zookeeper_config/configs/remote_servers.xml index 01865e33a85..8e972b31017 100644 --- a/tests/integration/test_zookeeper_config/configs/remote_servers.xml +++ b/tests/integration/test_zookeeper_config/configs/remote_servers.xml @@ -11,6 +11,12 @@ node2 9000 + + + node3 + 9000 + + diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index b5082215e22..732816ea224 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -1,195 +1,55 @@ - - import time -import threading -from os import path as p, unlink -from tempfile import NamedTemporaryFile - -import helpers import pytest +import logging from helpers.cluster import ClickHouseCluster +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') -def test_chroot_with_same_root(): - cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') - cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) +node3 = cluster.add_instance('node3', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"]) - node1 = cluster_1.add_instance('node1', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], - with_zookeeper=True) - node2 = cluster_2.add_instance('node2', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], - with_zookeeper=True) - nodes = [node1, node2] - - def create_zk_root(zk): - zk.ensure_path('/root_a') - print(zk.get_children('/')) - - cluster_1.add_zookeeper_startup_command(create_zk_root) - - try: - cluster_1.start() - - try: - cluster_2.start(destroy_dirs=False) - for i, node in enumerate(nodes): - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - for j in range(2): # Second insert to test deduplication - node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - - time.sleep(1) - - assert node1.query('select count() from simple').strip() == '2' - assert node2.query('select count() from simple').strip() == '2' - - finally: - cluster_2.shutdown() - - finally: - cluster_1.shutdown() - - -def test_chroot_with_different_root(): - cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') - cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_b.xml') - - node1 = cluster_1.add_instance('node1', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], - with_zookeeper=True) - node2 = cluster_2.add_instance('node2', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"], - with_zookeeper=True) - nodes = [node1, node2] - - def create_zk_roots(zk): - zk.ensure_path('/root_a') - zk.ensure_path('/root_b') - print(zk.get_children('/')) - - cluster_1.add_zookeeper_startup_command(create_zk_roots) - - try: - cluster_1.start() - - try: - cluster_2.start(destroy_dirs=False) - - for i, node in enumerate(nodes): - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - for j in range(2): # Second insert to test deduplication - node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - - assert node1.query('select count() from simple').strip() == '1' - assert node2.query('select count() from simple').strip() == '1' - - finally: - cluster_2.shutdown() - - finally: - cluster_1.shutdown() - - -def test_identity(): - cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_password.xml') - cluster_2 = ClickHouseCluster(__file__) - - # TODO ACL not implemented in Keeper. - node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", - "configs/zookeeper_config_with_password.xml"], - with_zookeeper=True, use_keeper=False) - node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml"], with_zookeeper=True, use_keeper=False) - - try: - cluster_1.start() - - node1.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node1.name)) - - with pytest.raises(Exception): - cluster_2.start(destroy_dirs=False) - node2.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192); - ''') - - finally: - cluster_1.shutdown() - cluster_2.shutdown() - - -# NOTE this test have to be ported to Keeper -def test_secure_connection(): - # We need absolute path in zookeeper volumes. Generate it dynamically. - TEMPLATE = ''' - zoo{zoo_id}: - image: zookeeper:3.6.2 - restart: always - environment: - ZOO_TICK_TIME: 500 - ZOO_MY_ID: {zoo_id} - ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 - ZOO_SECURE_CLIENT_PORT: 2281 - volumes: - - {helpers_dir}/zookeeper-ssl-entrypoint.sh:/zookeeper-ssl-entrypoint.sh - - {configs_dir}:/clickhouse-config - command: ["zkServer.sh", "start-foreground"] - entrypoint: /zookeeper-ssl-entrypoint.sh - ''' - configs_dir = p.abspath(p.join(p.dirname(__file__), 'configs_secure')) - helpers_dir = p.abspath(p.dirname(helpers.__file__)) - - cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_ssl.xml') - - docker_compose = NamedTemporaryFile(mode='w+', delete=False) - - docker_compose.write( - "version: '2.3'\nservices:\n" + - TEMPLATE.format(zoo_id=1, configs_dir=configs_dir, helpers_dir=helpers_dir) + - TEMPLATE.format(zoo_id=2, configs_dir=configs_dir, helpers_dir=helpers_dir) + - TEMPLATE.format(zoo_id=3, configs_dir=configs_dir, helpers_dir=helpers_dir) - ) - docker_compose.close() - - node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", - "configs_secure/conf.d/remote_servers.xml", - "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, use_keeper=False, zookeeper_docker_compose_path=docker_compose.name, - ) - node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", - "configs_secure/conf.d/remote_servers.xml", - "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, use_keeper=False, zookeeper_docker_compose_path=docker_compose.name, - ) +def create_zk_roots(zk): + zk.ensure_path('/root_a') + zk.ensure_path('/root_b') + logging.debug(f"Create ZK roots:{zk.get_children('/')}") +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): try: + cluster.add_zookeeper_startup_command(create_zk_roots) cluster.start() - assert node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' - assert node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' - - - kThreadsNumber = 16 - kIterations = 100 - threads = [] - for _ in range(kThreadsNumber): - threads.append(threading.Thread(target=(lambda: - [node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") for _ in range(kIterations)]))) - - for thread in threads: - thread.start() - - for thread in threads: - thread.join() + yield cluster finally: cluster.shutdown() - unlink(docker_compose.name) + +def test_chroot_with_same_root(started_cluster): + for i, node in enumerate([node1, node2]): + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + for j in range(2): # Second insert to test deduplication + node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) + + time.sleep(1) + + assert node1.query('select count() from simple').strip() == '2' + assert node2.query('select count() from simple').strip() == '2' + +def test_chroot_with_different_root(started_cluster): + for i, node in [(1, node1), (3, node3)]: + node.query(''' + CREATE TABLE simple_different (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + for j in range(2): # Second insert to test deduplication + node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) + + assert node1.query('select count() from simple_different').strip() == '1' + assert node3.query('select count() from simple_different').strip() == '1' diff --git a/tests/integration/test_zookeeper_config/test_password.py b/tests/integration/test_zookeeper_config/test_password.py new file mode 100644 index 00000000000..b99007e67f2 --- /dev/null +++ b/tests/integration/test_zookeeper_config/test_password.py @@ -0,0 +1,34 @@ + + +import time +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, name="password") + +# TODO ACL not implemented in Keeper. +node1 = cluster.add_instance('node1', with_zookeeper=True, use_keeper=False, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_with_password.xml"]) + +node2 = cluster.add_instance('node2', with_zookeeper=True, use_keeper=False, main_configs=["configs/remote_servers.xml"]) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +def test_identity(started_cluster): + node1.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node1.name)) + + with pytest.raises(Exception): + node2.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192); + ''') diff --git a/tests/integration/test_zookeeper_config/test_secure.py b/tests/integration/test_zookeeper_config/test_secure.py new file mode 100644 index 00000000000..f57b8d8a80a --- /dev/null +++ b/tests/integration/test_zookeeper_config/test_secure.py @@ -0,0 +1,52 @@ + + +import threading +import os +from tempfile import NamedTemporaryFile + +import pytest +from helpers.cluster import ClickHouseCluster + +TEST_DIR = os.path.dirname(__file__) + +cluster = ClickHouseCluster(__file__, name="secure", + zookeeper_certfile=os.path.join(TEST_DIR, "configs_secure", "client.crt"), + zookeeper_keyfile=os.path.join(TEST_DIR, "configs_secure", "client.key")) + +node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", + "configs_secure/conf.d/ssl_conf.xml", + "configs/zookeeper_config_with_ssl.xml"], + with_zookeeper_secure=True) +node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", + "configs_secure/conf.d/ssl_conf.xml", + "configs/zookeeper_config_with_ssl.xml"], + with_zookeeper_secure=True) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +# NOTE this test have to be ported to Keeper +def test_secure_connection(started_cluster): + assert node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' + assert node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' + + kThreadsNumber = 16 + kIterations = 100 + threads = [] + for _ in range(kThreadsNumber): + threads.append(threading.Thread(target=(lambda: + [node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") for _ in range(kIterations)]))) + + for thread in threads: + thread.start() + + for thread in threads: + thread.join() From c6c180c35df386e4027c1eca2b87dd8515e8bc02 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 21 May 2021 17:33:56 +0300 Subject: [PATCH 152/652] fix shellcheck --- .../test/integration/runner/misc/zookeeper-ssl-entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh b/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh index 3ddb21881d6..9748a5e8ce9 100755 --- a/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh +++ b/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh @@ -81,8 +81,8 @@ if [[ ! -f "$ZOO_DATA_DIR/myid" ]]; then echo "${ZOO_MY_ID:-1}" > "$ZOO_DATA_DIR/myid" fi -mkdir -p $(dirname $ZOO_SSL_KEYSTORE_LOCATION) -mkdir -p $(dirname $ZOO_SSL_TRUSTSTORE_LOCATION) +mkdir -p "$(dirname $ZOO_SSL_KEYSTORE_LOCATION)" +mkdir -p "$(dirname $ZOO_SSL_TRUSTSTORE_LOCATION)" if [[ ! -f "$ZOO_SSL_KEYSTORE_LOCATION" ]]; then keytool -genkeypair -alias zookeeper -keyalg RSA -validity 365 -keysize 2048 -dname "cn=zookeeper" -keypass password -keystore $ZOO_SSL_KEYSTORE_LOCATION -storepass password -deststoretype pkcs12 From d73d92ed7e67c20a1f58922a2004dae95e77580b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 21 May 2021 18:48:14 +0300 Subject: [PATCH 153/652] revert some changes --- docker/test/integration/runner/Dockerfile | 2 -- docker/test/integration/runner/dockerd-entrypoint.sh | 3 --- tests/integration/helpers/cluster.py | 12 ++++++------ 3 files changed, 6 insertions(+), 11 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 164058eb72e..772a6a013ab 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -31,8 +31,6 @@ RUN apt-get update \ libkrb5-dev \ krb5-user \ g++ \ - dbus \ - firewalld \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 2ae51fa1a04..2321c938990 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -12,9 +12,6 @@ echo '{ "registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"] }' | dd of=/etc/docker/daemon.json -/etc/init.d/dbus start -firewalld --debug --log-file /ClickHouse/tests/integration/firewalld.log - dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e0d8b01eda0..e3669e9826b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -388,13 +388,13 @@ class ClickHouseCluster: # pass # Remove unused images - try: - logging.debug("Trying to prune unused images...") + # try: + # logging.debug("Trying to prune unused images...") - subprocess_call(['docker', 'image', 'prune', '-f']) - logging.debug("Images pruned") - except: - pass + # subprocess_call(['docker', 'image', 'prune', '-f']) + # logging.debug("Images pruned") + # except: + # pass # Remove unused volumes try: From 13cfedd1881024c751a75f07d8a8795dc701f5b1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 21 May 2021 21:48:19 +0300 Subject: [PATCH 154/652] optimize some functions to subcolumns --- src/Core/Settings.h | 1 + .../RewriteFunctionToSubcolumnVisitor.cpp | 52 +++++++++++++++++++ .../RewriteFunctionToSubcolumnVisitor.h | 23 ++++++++ src/Interpreters/TreeOptimizer.cpp | 27 ++++++---- src/Interpreters/TreeOptimizer.h | 10 ++-- src/Interpreters/TreeRewriter.cpp | 3 +- .../01872_functions_to_subcolumns.reference | 27 ++++++++++ .../01872_functions_to_subcolumns.sql | 18 +++++++ 8 files changed, 143 insertions(+), 18 deletions(-) create mode 100644 src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp create mode 100644 src/Interpreters/RewriteFunctionToSubcolumnVisitor.h create mode 100644 tests/queries/0_stateless/01872_functions_to_subcolumns.reference create mode 100644 tests/queries/0_stateless/01872_functions_to_subcolumns.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 125879486ab..fc885fe38cc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -400,6 +400,7 @@ class IColumn; M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \ + M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp new file mode 100644 index 00000000000..7616529dff7 --- /dev/null +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp @@ -0,0 +1,52 @@ +#include +#include +#include + +namespace DB +{ + +namespace +{ + +ASTPtr transformToSubcolumn(const ASTIdentifier & identifier, const String & subcolumn_name) +{ + return std::make_shared(Nested::concatenateName(identifier.name(), subcolumn_name)); +} + +ASTPtr transformIsNotNullToSubcolumn(const ASTIdentifier & identifier, const String & subcolumn_name) +{ + auto ast = transformToSubcolumn(identifier, subcolumn_name); + ast = makeASTFunction("NOT", ast); + return ast; +} + +const std::unordered_map> function_to_subcolumn = +{ + {"length", {"size0", transformToSubcolumn}}, + {"isNull", {"null", transformToSubcolumn}}, + {"isNotNull", {"null", transformIsNotNullToSubcolumn}}, + {"mapKeys", {"keys", transformToSubcolumn}}, + {"mapValues", {"values", transformToSubcolumn}} +}; + +} + +void RewriteFunctionToSubcolumnData::visit(ASTFunction & function, ASTPtr & ast) +{ + const auto & arguments = function.arguments->children; + if (arguments.size() != 1) + return; + + const auto * identifier = arguments[0]->as(); + if (!identifier || !columns_to_rewrite.count(identifier->name())) + return; + + auto it = function_to_subcolumn.find(function.name); + if (it == function_to_subcolumn.end()) + return; + + const auto & [subcolumn_name, transformer] = it->second; + ast = transformer(*identifier, subcolumn_name); +} + +} diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h new file mode 100644 index 00000000000..74f0041613c --- /dev/null +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Rewrites functions to subcolumns, if possible, to reduce amount of read data. +/// E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' +class RewriteFunctionToSubcolumnData +{ +public: + using TypeToVisit = ASTFunction; + void visit(ASTFunction & function, ASTPtr & ast); + + const NameSet & columns_to_rewrite; +}; + +using RewriteFunctionToSubcolumnMatcher = OneTypeMatcher; +using RewriteFunctionToSubcolumnVisitor = InDepthNodeVisitor; + +} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 5b06c00435a..b07501ec862 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include @@ -27,7 +29,7 @@ #include #include -#include +#include #include @@ -579,6 +581,12 @@ void transformIfStringsIntoEnum(ASTPtr & query) ConvertStringsToEnumVisitor(convert_data).visit(query); } +void optimizeFunctionsToSubcolumns(ASTPtr & query, const NameSet & source_columns) +{ + RewriteFunctionToSubcolumnVisitor::Data data{source_columns}; + RewriteFunctionToSubcolumnVisitor(data).visit(query); +} + } void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif) @@ -590,10 +598,8 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ OptimizeIfChainsVisitor().visit(query); } -void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, - const std::vector & tables_with_columns, - ContextPtr context, const StorageMetadataPtr & metadata_snapshot, - bool & rewrite_subqueries) +void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, + const std::vector & tables_with_columns, ContextPtr context) { const auto & settings = context->getSettingsRef(); @@ -601,17 +607,20 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou if (!select_query) throw Exception("Select analyze for not select asts.", ErrorCodes::LOGICAL_ERROR); - optimizeIf(query, aliases, settings.optimize_if_chain_to_multiif); + if (result.storage && result.storage->supportsSubcolumns() && settings.optimize_functions_to_subcolumns) + optimizeFunctionsToSubcolumns(query, result.source_columns_set); + + optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); /// Move arithmetic operations out of aggregation functions if (settings.optimize_arithmetic_operations_in_aggregate_functions) optimizeAggregationFunctions(query); /// Push the predicate expression down to the subqueries. - rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); + result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); /// GROUP BY injective function elimination. - optimizeGroupBy(select_query, source_columns_set, context); + optimizeGroupBy(select_query, result.source_columns_set, context); /// GROUP BY functions of other keys elimination. if (settings.optimize_group_by_function_keys) @@ -658,7 +667,7 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou /// Replace monotonous functions with its argument if (settings.optimize_monotonous_functions_in_order_by) optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns, - metadata_snapshot ? metadata_snapshot->getSortingKeyColumns() : Names{}); + result.metadata_snapshot ? result.metadata_snapshot->getSortingKeyColumns() : Names{}); /// Remove duplicate items from ORDER BY. /// Execute it after all order by optimizations, diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index b268b230f4e..b8a98a6f9ac 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -8,8 +8,7 @@ namespace DB { -struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; +struct TreeRewriterResult; /// Part of of Tree Rewriter (SyntaxAnalyzer) that optimizes AST. /// Query should be ready to execute either before either after it. But resulting query could be faster. @@ -18,12 +17,9 @@ class TreeOptimizer public: static void apply( ASTPtr & query, - Aliases & aliases, - const NameSet & source_columns_set, + TreeRewriterResult & result, const std::vector & tables_with_columns, - ContextPtr context, - const StorageMetadataPtr & metadata_snapshot, - bool & rewrite_subqueries); + ContextPtr context); static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif); }; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 92cfba1bcb1..64ddf4cb750 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -924,8 +924,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, getContext(), subquery_depth, result.scalars, select_options.only_analyze); - TreeOptimizer::apply( - query, result.aliases, source_columns_set, tables_with_columns, getContext(), result.metadata_snapshot, result.rewrite_subqueries); + TreeOptimizer::apply(query, result, tables_with_columns, getContext()); /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference new file mode 100644 index 00000000000..0e666fc6bb0 --- /dev/null +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference @@ -0,0 +1,27 @@ +3 0 1 ['foo','bar'] [1,2] +1 1 0 [] [] +SELECT + `arr.size0`, + `n.null`, + NOT `n.null`, + `m.keys`, + `m.values` +FROM t_func_to_subcolumns +1 0 0 +2 1 0 +3 0 0 +SELECT + id, + `n.null`, + isNull(right.n) +FROM t_func_to_subcolumns AS left +ALL FULL OUTER JOIN +( + SELECT + 1 AS id, + \'qqq\' AS n + UNION ALL + SELECT + 3 AS id, + \'www\' +) AS right USING (id) diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.sql b/tests/queries/0_stateless/01872_functions_to_subcolumns.sql new file mode 100644 index 00000000000..5b45254d328 --- /dev/null +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS t_func_to_subcolumns; + +SET allow_experimental_map_type = 1; +SET optimize_functions_to_subcolumns = 1; + +CREATE TABLE t_func_to_subcolumns (id UInt64, arr Array(UInt64), n Nullable(String), m Map(String, UInt64)) +ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_func_to_subcolumns VALUES (1, [1, 2, 3], 'abc', map('foo', 1, 'bar', 2)) (2, [22], NULL, map()); + +SELECT length(arr), n IS NULL, n IS NOT NULL, mapKeys(m), mapValues(m) FROM t_func_to_subcolumns; +EXPLAIN SYNTAX SELECT length(arr), n IS NULL, n IS NOT NULL, mapKeys(m), mapValues(m) FROM t_func_to_subcolumns; + +SELECT id, left.n IS NULL, right.n IS NULL FROM t_func_to_subcolumns AS left +FULL JOIN (SELECT 1 AS id, 'qqq' AS n UNION ALL SELECT 3 AS id, 'www') AS right USING(id); + +EXPLAIN SYNTAX SELECT id, left.n IS NULL, right.n IS NULL FROM t_func_to_subcolumns AS left +FULL JOIN (SELECT 1 AS id, 'qqq' AS n UNION ALL SELECT 3 AS id, 'www') AS right USING(id); From 75adbd8e2357cf41da6fafb3f510cfd9cacabc02 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 22 May 2021 02:22:22 +0300 Subject: [PATCH 155/652] optimize some functions to subcolumns --- .../RewriteFunctionToSubcolumnVisitor.cpp | 58 ++++++++++++++----- .../RewriteFunctionToSubcolumnVisitor.h | 3 +- src/Interpreters/TreeOptimizer.cpp | 9 +-- .../01872_functions_to_subcolumns.reference | 28 +++++++-- .../01872_functions_to_subcolumns.sql | 30 +++++++++- 5 files changed, 101 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp index 7616529dff7..2086d2dc229 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -8,25 +9,45 @@ namespace DB namespace { -ASTPtr transformToSubcolumn(const ASTIdentifier & identifier, const String & subcolumn_name) +ASTPtr transformToSubcolumn(const String & name_in_storage, const String & subcolumn_name) { - return std::make_shared(Nested::concatenateName(identifier.name(), subcolumn_name)); + return std::make_shared(Nested::concatenateName(name_in_storage, subcolumn_name)); } -ASTPtr transformIsNotNullToSubcolumn(const ASTIdentifier & identifier, const String & subcolumn_name) +ASTPtr transformEmptyToSubcolumn(const String & name_in_storage, const String & subcolumn_name) { - auto ast = transformToSubcolumn(identifier, subcolumn_name); - ast = makeASTFunction("NOT", ast); - return ast; + auto ast = transformToSubcolumn(name_in_storage, subcolumn_name); + return makeASTFunction("equals", ast, std::make_shared(0u)); } -const std::unordered_map> function_to_subcolumn = +ASTPtr transformNotEmptyToSubcolumn(const String & name_in_storage, const String & subcolumn_name) { - {"length", {"size0", transformToSubcolumn}}, - {"isNull", {"null", transformToSubcolumn}}, - {"isNotNull", {"null", transformIsNotNullToSubcolumn}}, - {"mapKeys", {"keys", transformToSubcolumn}}, - {"mapValues", {"values", transformToSubcolumn}} + auto ast = transformToSubcolumn(name_in_storage, subcolumn_name); + return makeASTFunction("notEquals", ast, std::make_shared(0u)); +} + +ASTPtr transformIsNotNullToSubcolumn(const String & name_in_storage, const String & subcolumn_name) +{ + auto ast = transformToSubcolumn(name_in_storage, subcolumn_name); + return makeASTFunction("not", ast); +} + +ASTPtr transformCountNullableToSubcolumn(const String & name_in_storage, const String & subcolumn_name) +{ + auto ast = transformToSubcolumn(name_in_storage, subcolumn_name); + return makeASTFunction("sum", makeASTFunction("not", ast)); +} + +const std::unordered_map> function_to_subcolumn = +{ + {"length", {TypeIndex::Array, "size0", transformToSubcolumn}}, + {"empty", {TypeIndex::Array, "size0", transformEmptyToSubcolumn}}, + {"notEmpty", {TypeIndex::Array, "size0", transformNotEmptyToSubcolumn}}, + {"isNull", {TypeIndex::Nullable, "null", transformToSubcolumn}}, + {"isNotNull", {TypeIndex::Nullable, "null", transformIsNotNullToSubcolumn}}, + {"count", {TypeIndex::Nullable, "null", transformCountNullableToSubcolumn}}, + {"mapKeys", {TypeIndex::Map, "keys", transformToSubcolumn}}, + {"mapValues", {TypeIndex::Map, "values", transformToSubcolumn}}, }; } @@ -38,15 +59,22 @@ void RewriteFunctionToSubcolumnData::visit(ASTFunction & function, ASTPtr & ast) return; const auto * identifier = arguments[0]->as(); - if (!identifier || !columns_to_rewrite.count(identifier->name())) + if (!identifier) return; auto it = function_to_subcolumn.find(function.name); if (it == function_to_subcolumn.end()) return; - const auto & [subcolumn_name, transformer] = it->second; - ast = transformer(*identifier, subcolumn_name); + const auto & [type_id, subcolumn_name, transformer] = it->second; + const auto & columns = metadata_snapshot->getColumns(); + const auto & name_in_storage = identifier->name(); + + if (columns.has(name_in_storage) + && columns.get(name_in_storage).type->getTypeId() == type_id) + { + ast = transformer(name_in_storage, subcolumn_name); + } } } diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h index 74f0041613c..cfd87d8449e 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -14,7 +15,7 @@ public: using TypeToVisit = ASTFunction; void visit(ASTFunction & function, ASTPtr & ast); - const NameSet & columns_to_rewrite; + StorageMetadataPtr metadata_snapshot; }; using RewriteFunctionToSubcolumnMatcher = OneTypeMatcher; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index b07501ec862..c1a265d9a06 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -581,9 +581,9 @@ void transformIfStringsIntoEnum(ASTPtr & query) ConvertStringsToEnumVisitor(convert_data).visit(query); } -void optimizeFunctionsToSubcolumns(ASTPtr & query, const NameSet & source_columns) +void optimizeFunctionsToSubcolumns(ASTPtr & query, const StorageMetadataPtr & metadata_snapshot) { - RewriteFunctionToSubcolumnVisitor::Data data{source_columns}; + RewriteFunctionToSubcolumnVisitor::Data data{metadata_snapshot}; RewriteFunctionToSubcolumnVisitor(data).visit(query); } @@ -607,8 +607,9 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, if (!select_query) throw Exception("Select analyze for not select asts.", ErrorCodes::LOGICAL_ERROR); - if (result.storage && result.storage->supportsSubcolumns() && settings.optimize_functions_to_subcolumns) - optimizeFunctionsToSubcolumns(query, result.source_columns_set); + if (settings.optimize_functions_to_subcolumns && result.storage + && result.storage->supportsSubcolumns() && result.metadata_snapshot) + optimizeFunctionsToSubcolumns(query, result.metadata_snapshot); optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference index 0e666fc6bb0..ff929fb0f6d 100644 --- a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference @@ -1,12 +1,30 @@ -3 0 1 ['foo','bar'] [1,2] -1 1 0 [] [] +0 0 1 +0 1 0 +SELECT + isNull(id), + `n.null`, + NOT `n.null` +FROM t_func_to_subcolumns +3 0 1 0 +0 1 0 \N SELECT `arr.size0`, - `n.null`, - NOT `n.null`, + `arr.size0` = 0, + `arr.size0` != 0, + empty(n) +FROM t_func_to_subcolumns +['foo','bar'] [1,2] +[] [] +SELECT `m.keys`, `m.values` FROM t_func_to_subcolumns +1 +SELECT sum(NOT `n.null`) +FROM t_func_to_subcolumns +2 +SELECT count(id) +FROM t_func_to_subcolumns 1 0 0 2 1 0 3 0 0 @@ -25,3 +43,5 @@ ALL FULL OUTER JOIN 3 AS id, \'www\' ) AS right USING (id) +0 10 +0 20 diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.sql b/tests/queries/0_stateless/01872_functions_to_subcolumns.sql index 5b45254d328..eb0165f4e13 100644 --- a/tests/queries/0_stateless/01872_functions_to_subcolumns.sql +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.sql @@ -6,13 +6,37 @@ SET optimize_functions_to_subcolumns = 1; CREATE TABLE t_func_to_subcolumns (id UInt64, arr Array(UInt64), n Nullable(String), m Map(String, UInt64)) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO t_func_to_subcolumns VALUES (1, [1, 2, 3], 'abc', map('foo', 1, 'bar', 2)) (2, [22], NULL, map()); +INSERT INTO t_func_to_subcolumns VALUES (1, [1, 2, 3], 'abc', map('foo', 1, 'bar', 2)) (2, [], NULL, map()); -SELECT length(arr), n IS NULL, n IS NOT NULL, mapKeys(m), mapValues(m) FROM t_func_to_subcolumns; -EXPLAIN SYNTAX SELECT length(arr), n IS NULL, n IS NOT NULL, mapKeys(m), mapValues(m) FROM t_func_to_subcolumns; +SELECT id IS NULL, n IS NULL, n IS NOT NULL FROM t_func_to_subcolumns; +EXPLAIN SYNTAX SELECT id IS NULL, n IS NULL, n IS NOT NULL FROM t_func_to_subcolumns; + +SELECT length(arr), empty(arr), notEmpty(arr), empty(n) FROM t_func_to_subcolumns; +EXPLAIN SYNTAX SELECT length(arr), empty(arr), notEmpty(arr), empty(n) FROM t_func_to_subcolumns; + +SELECT mapKeys(m), mapValues(m) FROM t_func_to_subcolumns; +EXPLAIN SYNTAX SELECT mapKeys(m), mapValues(m) FROM t_func_to_subcolumns; + +SELECT count(n) FROM t_func_to_subcolumns; +EXPLAIN SYNTAX SELECT count(n) FROM t_func_to_subcolumns; + +SELECT count(id) FROM t_func_to_subcolumns; +EXPLAIN SYNTAX SELECT count(id) FROM t_func_to_subcolumns; SELECT id, left.n IS NULL, right.n IS NULL FROM t_func_to_subcolumns AS left FULL JOIN (SELECT 1 AS id, 'qqq' AS n UNION ALL SELECT 3 AS id, 'www') AS right USING(id); EXPLAIN SYNTAX SELECT id, left.n IS NULL, right.n IS NULL FROM t_func_to_subcolumns AS left FULL JOIN (SELECT 1 AS id, 'qqq' AS n UNION ALL SELECT 3 AS id, 'www') AS right USING(id); + +DROP TABLE t_func_to_subcolumns; + +DROP TABLE IF EXISTS t_tuple_null; + +CREATE TABLE t_tuple_null (t Tuple(null UInt32)) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_tuple_null VALUES ((10)), ((20)); + +SELECT t IS NULL, t.null FROM t_tuple_null; + +DROP TABLE t_tuple_null; From d016f60c1d3a256989db73825370cb4a2f8ccec5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 22 May 2021 03:29:01 +0300 Subject: [PATCH 156/652] fix test --- .../queries/0_stateless/01872_functions_to_subcolumns.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference index ff929fb0f6d..8c5ca4df602 100644 --- a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference @@ -33,7 +33,7 @@ SELECT `n.null`, isNull(right.n) FROM t_func_to_subcolumns AS left -ALL FULL OUTER JOIN +ALL FULL OUTER JOIN ( SELECT 1 AS id, From de69a9f3afbe5258ce79e80c6ddff0908320be32 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:04:25 +0300 Subject: [PATCH 157/652] Fix warning in LZSSE --- contrib/lzsse-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/lzsse-cmake/CMakeLists.txt b/contrib/lzsse-cmake/CMakeLists.txt index a4c0d9dd8e7..5ec5b312009 100644 --- a/contrib/lzsse-cmake/CMakeLists.txt +++ b/contrib/lzsse-cmake/CMakeLists.txt @@ -18,4 +18,4 @@ SET (Headers ADD_LIBRARY(lzsse ${Sources} ${Headers}) -target_include_directories (lzsse PUBLIC ${LIBRARY_DIR}) +target_include_directories (lzsse SYSTEM PUBLIC ${LIBRARY_DIR}) From 7afc052a11d7690e11ec772d017e504f972430d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:12:50 +0300 Subject: [PATCH 158/652] Revert changes in compressor tool, because everything is already available in --codec argument --- programs/compressor/Compressor.cpp | 34 +----------------------------- 1 file changed, 1 insertion(+), 33 deletions(-) diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index d5471a62afe..9f99f8eb95e 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -23,6 +23,7 @@ #include #include + namespace DB { namespace ErrorCodes @@ -80,11 +81,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") - ("lizard", "use Lizard instread of LZ4") - ("lzsse2", "use lzsse2 instread of LZ4") - ("lzsse4", "use lzsse4 instread of LZ4") - ("lzsse8", "use lzsse8 instread of LZ4") - ("density", "use Density instread of LZ4") ("param", po::value(), "extra params for compresion algorithm") ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") ("level", po::value(), "compression level for codecs specified via flags") @@ -112,11 +108,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool decompress = options.count("decompress"); bool use_lz4hc = options.count("hc"); bool use_zstd = options.count("zstd"); - bool use_lizard = options.count("lizard"); - bool use_lzsse2 = options.count("lzsse2"); - bool use_lzsse4 = options.count("lzsse4"); - bool use_lzsse8 = options.count("lzsse8"); - bool use_density = options.count("density"); bool stat_mode = options.count("stat"); bool use_none = options.count("none"); unsigned block_size = options["block-size"].as(); @@ -124,10 +115,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) if (options.count("codec")) codecs = options["codec"].as>(); - std::optional param; - if (options.count("param")) - param = options["param"].as(); - if ((use_lz4hc || use_zstd || use_none) && !codecs.empty()) throw Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", ErrorCodes::BAD_ARGUMENTS); @@ -140,16 +127,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) method_family = "LZ4HC"; else if (use_zstd) method_family = "ZSTD"; - else if (use_lizard) - method_family = "Lizard"; - else if (use_lzsse2) - method_family = "LZSSE2"; - else if (use_lzsse4) - method_family = "LZSSE4"; - else if (use_lzsse8) - method_family = "LZSSE8"; - else if (use_density) - method_family = "Density"; else if (use_none) method_family = "NONE"; @@ -165,8 +142,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) std::string codecs_line = boost::algorithm::join(codecs, ","); auto ast = parseQuery(codec_parser, "(" + codecs_line + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); codec = CompressionCodecFactory::instance().get(ast, nullptr); - } else if (param.has_value()) { - codec = CompressionCodecFactory::instance().get(method_family, level, param); } else codec = CompressionCodecFactory::instance().get(method_family, level); @@ -185,9 +160,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else wb = std::make_unique(STDOUT_FILENO); - - struct tms tv1, tv2; - times(&tv1); if (stat_mode) { /// Output statistic for compressed file. @@ -218,10 +190,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) CompressedWriteBuffer to(*wb, codec, block_size); copyData(*rb, to); } - times(&tv2); - - int tics_per_second = sysconf(_SC_CLK_TCK); - std::cerr << static_cast(tv2.tms_utime - tv1.tms_utime) / tics_per_second << std::endl; } catch (...) { From e330fe4bb5b68a7f8af4e4944c693afd619bcd53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:15:11 +0300 Subject: [PATCH 159/652] Fix extra whitespace in exception messages --- src/Compression/CompressionCodecDensity.cpp | 4 ++-- src/Compression/CompressionCodecLZSSE2.cpp | 8 ++++---- src/Compression/CompressionCodecLZSSE4.cpp | 8 ++++---- src/Compression/CompressionCodecLZSSE8.cpp | 8 ++++---- src/Compression/CompressionCodecLizard.cpp | 4 ++-- 5 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index 841638ac6ed..8aec1f0098c 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -39,7 +39,7 @@ UInt32 CompressionCodecDensity::doCompressData(const char * source, UInt32 sourc { density_processing_result res = density_compress(reinterpret_cast(source), source_size, reinterpret_cast(dest), density_compress_safe_size(source_size), algo); if (res.state != DENSITY_STATE_OK) - throw Exception("Cannot compress block with Density; ", ErrorCodes::CANNOT_COMPRESS); + throw Exception("Cannot compress block with Density", ErrorCodes::CANNOT_COMPRESS); return res.bytesWritten; } @@ -47,7 +47,7 @@ void CompressionCodecDensity::doDecompressData(const char * source, UInt32 sourc { density_processing_result res = density_decompress(reinterpret_cast(source), source_size, reinterpret_cast(dest), density_decompress_safe_size(uncompressed_size)); if (res.state != DENSITY_STATE_OK) - throw Exception("Cannot decompress block with Density; ", ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Cannot decompress block with Density", ErrorCodes::CANNOT_DECOMPRESS); } void registerCodecDensity(CompressionCodecFactory & factory) diff --git a/src/Compression/CompressionCodecLZSSE2.cpp b/src/Compression/CompressionCodecLZSSE2.cpp index 215b93e6721..bbbe63690f0 100644 --- a/src/Compression/CompressionCodecLZSSE2.cpp +++ b/src/Compression/CompressionCodecLZSSE2.cpp @@ -40,12 +40,12 @@ UInt32 CompressionCodecLZSSE2::getMaxCompressedDataSize(UInt32 uncompressed_size UInt32 CompressionCodecLZSSE2::doCompressData(const char * source, UInt32 source_size, char * dest) const { UInt32 res; - LZSSE2_OptimalParseState* state = LZSSE2_MakeOptimalParseState(source_size); + LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); LZSSE2_FreeOptimalParseState(state); if (res == 0) - throw Exception("Cannot compress block with LZSSE2; ", ErrorCodes::CANNOT_COMPRESS); + throw Exception("Cannot compress block with LZSSE2", ErrorCodes::CANNOT_COMPRESS); return res; } @@ -54,9 +54,9 @@ void CompressionCodecLZSSE2::doDecompressData(const char * source, UInt32 source { UInt32 res; res = LZSSE2_Decompress(source, source_size, dest, uncompressed_size); - + if (res < uncompressed_size) - throw Exception("Cannot decompress block with LZSSE2; ", ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Cannot decompress block with LZSSE2", ErrorCodes::CANNOT_DECOMPRESS); } void registerCodecLZSSE2(CompressionCodecFactory & factory) diff --git a/src/Compression/CompressionCodecLZSSE4.cpp b/src/Compression/CompressionCodecLZSSE4.cpp index 9b687d866a1..1f5f00cdf91 100644 --- a/src/Compression/CompressionCodecLZSSE4.cpp +++ b/src/Compression/CompressionCodecLZSSE4.cpp @@ -40,12 +40,12 @@ UInt32 CompressionCodecLZSSE4::getMaxCompressedDataSize(UInt32 uncompressed_size UInt32 CompressionCodecLZSSE4::doCompressData(const char * source, UInt32 source_size, char * dest) const { UInt32 res; - LZSSE4_OptimalParseState* state = LZSSE4_MakeOptimalParseState(source_size); + LZSSE4_OptimalParseState * state = LZSSE4_MakeOptimalParseState(source_size); res = LZSSE4_CompressOptimalParse(state, source, source_size, dest, source_size, level); LZSSE4_FreeOptimalParseState(state); if (res == 0) - throw Exception("Cannot compress block with LZSSE4; ", ErrorCodes::CANNOT_COMPRESS); + throw Exception("Cannot compress block with LZSSE4", ErrorCodes::CANNOT_COMPRESS); return res; } @@ -54,9 +54,9 @@ void CompressionCodecLZSSE4::doDecompressData(const char * source, UInt32 source { UInt32 res; res = LZSSE4_Decompress(source, source_size, dest, uncompressed_size); - + if (res < uncompressed_size) - throw Exception("Cannot decompress block with LZSSE4; ", ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Cannot decompress block with LZSSE4", ErrorCodes::CANNOT_DECOMPRESS); } void registerCodecLZSSE4(CompressionCodecFactory & factory) diff --git a/src/Compression/CompressionCodecLZSSE8.cpp b/src/Compression/CompressionCodecLZSSE8.cpp index 04e73121fc6..e1e86c75f4e 100644 --- a/src/Compression/CompressionCodecLZSSE8.cpp +++ b/src/Compression/CompressionCodecLZSSE8.cpp @@ -39,12 +39,12 @@ UInt32 CompressionCodecLZSSE8::getMaxCompressedDataSize(UInt32 uncompressed_size UInt32 CompressionCodecLZSSE8::doCompressData(const char * source, UInt32 source_size, char * dest) const { UInt32 res; - LZSSE8_OptimalParseState* state = LZSSE8_MakeOptimalParseState(source_size); + LZSSE8_OptimalParseState * state = LZSSE8_MakeOptimalParseState(source_size); res = LZSSE8_CompressOptimalParse(state, source, source_size, dest, source_size, level); LZSSE8_FreeOptimalParseState(state); if (res == 0) - throw Exception("Cannot compress block with LZSSE; ", ErrorCodes::CANNOT_COMPRESS); + throw Exception("Cannot compress block with LZSSE", ErrorCodes::CANNOT_COMPRESS); return res; } @@ -53,9 +53,9 @@ void CompressionCodecLZSSE8::doDecompressData(const char * source, UInt32 source { UInt32 res; res = LZSSE8_Decompress(source, source_size, dest, uncompressed_size); - + if (res < uncompressed_size) - throw Exception("Cannot decompress block with LZSSE; ", ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Cannot decompress block with LZSSE", ErrorCodes::CANNOT_DECOMPRESS); } void registerCodecLZSSE8(CompressionCodecFactory & factory) diff --git a/src/Compression/CompressionCodecLizard.cpp b/src/Compression/CompressionCodecLizard.cpp index a4106965bd5..2006a708493 100644 --- a/src/Compression/CompressionCodecLizard.cpp +++ b/src/Compression/CompressionCodecLizard.cpp @@ -42,7 +42,7 @@ UInt32 CompressionCodecLizard::doCompressData(const char * source, UInt32 source int res = Lizard_compress(source, dest, source_size, Lizard_compressBound(source_size), level); if (res == 0) - throw Exception("Cannot compress block with Lizard; ", ErrorCodes::CANNOT_COMPRESS); + throw Exception("Cannot compress block with Lizard", ErrorCodes::CANNOT_COMPRESS); return res; } @@ -51,7 +51,7 @@ void CompressionCodecLizard::doDecompressData(const char * source, UInt32 source int res = Lizard_decompress_safe(source, dest, source_size, uncompressed_size); if (res < 0) - throw Exception("Cannot compress block with Lizard; ", ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Cannot compress block with Lizard", ErrorCodes::CANNOT_DECOMPRESS); } void registerCodecLizard(CompressionCodecFactory & factory) From 0da00f6497bb91b014c2811ced5a83743eef83c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:18:24 +0300 Subject: [PATCH 160/652] Fix style, fix typo and remove extra code --- src/Compression/CompressionCodecDensity.cpp | 42 +++++++-------------- 1 file changed, 14 insertions(+), 28 deletions(-) diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index 8aec1f0098c..5a738c355c1 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -59,44 +59,30 @@ void registerCodecDensity(CompressionCodecFactory & factory) [&](const ASTPtr & arguments) -> CompressionCodecPtr { DENSITY_ALGORITHM algo = CompressionCodecDensity::DENSITY_DEFAULT_ALGO; - //std::cerr << arguments << std::endl; + if (arguments && !arguments->children.empty()) { if (arguments->children.size() != 1) throw Exception( - "Deisnty codec must have 1 parameter, given " + std::to_string(arguments->children.size()), + "Density codec must have only one parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); const auto children = arguments->children; const auto * algo_literal = children[0]->as(); - if (!algo_literal) - throw Exception("Density codec argument must be string (algorithm)", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + if (!algo_literal || algo_literal->value.getType() != Field::Types::String) + throw Exception("Density codec argument must be string (algorithm), one of: 'lion', 'chameleon', 'cheetah'", + ErrorCodes::ILLEGAL_CODEC_PARAMETER); - - if (algo_literal->value.getType() == Field::Types::Which::UInt64) { - const auto algorithm = algo_literal->value.safeGet(); - if (algorithm == 3) { - algo = DENSITY_ALGORITHM_LION; - } else if (algorithm == 2) { - algo = DENSITY_ALGORITHM_CHEETAH; - } else if (algorithm == 1) { - algo = DENSITY_ALGORITHM_CHAMELEON; - } else { - throw Exception("Density codec argument may be LION, CHAMELEON, CHEETAH", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - } - } else { - const auto algorithm = algo_literal->value.safeGet(); - if (algorithm == "LION") { - algo = DENSITY_ALGORITHM_LION; - } else if (algorithm == "CHAMELEON") { - algo = DENSITY_ALGORITHM_CHAMELEON; - } else if (algorithm == "CHEETAH") { - algo = DENSITY_ALGORITHM_CHEETAH; - } else { - throw Exception("Density codec argument may be LION, CHAMELEON, CHEETAH", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - } - } + const auto algorithm = algo_literal->value.safeGet(); + if (algorithm == "lion") + algo = DENSITY_ALGORITHM_LION; + else if (algorithm == "chameleon") + algo = DENSITY_ALGORITHM_CHAMELEON; + else if (algorithm == "cheetah") + algo = DENSITY_ALGORITHM_CHEETAH; + else + throw Exception("Density codec argument may be one of: 'lion', 'chameleon', 'cheetah'", ErrorCodes::ILLEGAL_CODEC_PARAMETER); } return std::make_shared(algo); From fff2f7b2be4f42c9212a1735364bac0e6808c11e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:21:21 +0300 Subject: [PATCH 161/652] Fix style, fix typo and remove extra code --- tests/integration/test_non_default_compression/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index 678b2167595..9689b4110a2 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -143,7 +143,7 @@ def test_experimental_codecs(start_cluster): node6.query(""" CREATE TABLE compression_experimental_codecs ( somedate Date CODEC(Lizard(12)), - id UInt64 CODEC(Density('LION')), + id UInt64 CODEC(Density('lion')), data String CODEC(LZSSE4(3)) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; """) From 4cfabd2675666039cd1c2b45cafc3eabd21c2107 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:21:48 +0300 Subject: [PATCH 162/652] Fix style, fix typo and remove extra code --- src/Compression/CompressionCodecDensity.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index 5a738c355c1..4404b890899 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -63,9 +63,8 @@ void registerCodecDensity(CompressionCodecFactory & factory) if (arguments && !arguments->children.empty()) { if (arguments->children.size() != 1) - throw Exception( - "Density codec must have only one parameter, given " + std::to_string(arguments->children.size()), - ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, + "Density codec must have only one parameter, given {}", arguments->children.size()); const auto children = arguments->children; From 1431679f516010160ae45fbd0a16c1119c95eba6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:23:11 +0300 Subject: [PATCH 163/652] Remove Lizard framing format because we do not need it --- contrib/lizard-cmake/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/contrib/lizard-cmake/CMakeLists.txt b/contrib/lizard-cmake/CMakeLists.txt index 8aa40418ec1..ad5f5bf132d 100644 --- a/contrib/lizard-cmake/CMakeLists.txt +++ b/contrib/lizard-cmake/CMakeLists.txt @@ -4,14 +4,11 @@ SET (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/lizard") SET (Sources "${LIBRARY_DIR}/lib/lizard_decompress.c" "${LIBRARY_DIR}/lib/lizard_compress.c" - "${LIBRARY_DIR}/lib/lizard_frame.c" - "${LIBRARY_DIR}/lib/xxhash/xxhash.c" ) SET (Headers "${LIBRARY_DIR}/lib/lizard_compress.h" "${LIBRARY_DIR}/lib/lizard_common.h" - "${LIBRARY_DIR}/lib/lizard_frame.h" ) ADD_LIBRARY(lizard ${Sources} ${Headers}) From e000b994e5e2431c177e4dc858944ed9ef9ba795 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:26:31 +0300 Subject: [PATCH 164/652] Add missing whitespace --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 41d5705187f..d005966ed6d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -238,7 +238,7 @@ class IColumn; M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ - M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs(LZSSE*, Lizard, Density).", 0) \ + M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (LZSSE2/4/8, Lizard, Density). These codecs are provided for evaluation purposes.", 0) \ M(UInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \ M(UInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ From ecc2e98265351001c5f66049a4626d3be4c3def9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:30:17 +0300 Subject: [PATCH 165/652] Fix wrong intent in exception message --- src/Compression/CompressionFactory.cpp | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 9ca64e36468..6e7444ca508 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -78,7 +78,8 @@ void CompressionCodecFactory::validateCodec(const String & family_name, std::opt } } -ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const +ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( + const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const { if (const auto * func = ast->as()) { @@ -107,16 +108,21 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr else throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (sanity_check && !allow_experimental_codecs) { - if (codec_family_name == "Lizard" || + if (!allow_experimental_codecs) + { + if (codec_family_name == "Lizard" || codec_family_name == "Density" || codec_family_name == "LZSSE2" || codec_family_name == "LZSSE4" || - codec_family_name == "LZSSE8") { + codec_family_name == "LZSSE8") + { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Experimental codecs Lizard, Density and LZSSE* are not allowed, please enable allow_experimental_codecs flag."); + "Codec {} is experimental and not meant to be used in production." + " You can enable it with the 'allow_experimental_codecs' setting.", + codec_family_name); } } + /// Default codec replaced with current default codec which may depend on different /// settings (and properties of data) in runtime. CompressionCodecPtr result_codec; From b0df9286f0c88e6939f5a9119ca4f63c8120a36f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:30:53 +0300 Subject: [PATCH 166/652] Fix whitespace --- src/Compression/CompressionFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 6e7444ca508..a703040e98c 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -116,7 +116,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( codec_family_name == "LZSSE4" || codec_family_name == "LZSSE8") { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec {} is experimental and not meant to be used in production." " You can enable it with the 'allow_experimental_codecs' setting.", codec_family_name); From a25edc192ca70743ec535f462251ecbb51b3840f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:31:58 +0300 Subject: [PATCH 167/652] Fix whitespace --- src/Compression/CompressionCodecLZSSE4.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Compression/CompressionCodecLZSSE4.cpp b/src/Compression/CompressionCodecLZSSE4.cpp index 1f5f00cdf91..7189a9cee55 100644 --- a/src/Compression/CompressionCodecLZSSE4.cpp +++ b/src/Compression/CompressionCodecLZSSE4.cpp @@ -18,7 +18,6 @@ namespace ErrorCodes CompressionCodecLZSSE4::CompressionCodecLZSSE4(int level_) : level(level_) { - setCodecDescription("LZSSE4", {std::make_shared(static_cast(level))}); } From 1a091c7a17e5fc165553a39f2d6497232144d9d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:32:22 +0300 Subject: [PATCH 168/652] Fix unbalanced whitespace --- src/Compression/CompressionCodecLZSSE2.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecLZSSE2.h b/src/Compression/CompressionCodecLZSSE2.h index b42ae4e7aca..d644645e3ac 100644 --- a/src/Compression/CompressionCodecLZSSE2.h +++ b/src/Compression/CompressionCodecLZSSE2.h @@ -5,10 +5,10 @@ namespace DB { + class CompressionCodecLZSSE2 : public ICompressionCodec { public: - CompressionCodecLZSSE2(int level_); uint8_t getMethodByte() const override; From 77414b50e6a2e065d6bd7ccd08f67fdf3ad5fcdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 03:55:16 +0300 Subject: [PATCH 169/652] Remove copy-paste --- src/Compression/CompressionCodecDensity.cpp | 1 - src/Compression/CompressionCodecLZSSE.cpp | 129 ++++++++++++++++++ ...nCodecLZSSE8.h => CompressionCodecLZSSE.h} | 14 +- src/Compression/CompressionCodecLZSSE2.cpp | 91 ------------ src/Compression/CompressionCodecLZSSE2.h | 33 ----- src/Compression/CompressionCodecLZSSE4.cpp | 90 ------------ src/Compression/CompressionCodecLZSSE4.h | 33 ----- src/Compression/CompressionCodecLZSSE8.cpp | 90 ------------ src/Compression/CompressionCodecLizard.cpp | 1 - src/Compression/CompressionFactory.cpp | 27 ++-- src/Compression/CompressionInfo.h | 2 +- src/Storages/System/StorageSystemErrors.cpp | 3 +- 12 files changed, 152 insertions(+), 362 deletions(-) create mode 100644 src/Compression/CompressionCodecLZSSE.cpp rename src/Compression/{CompressionCodecLZSSE8.h => CompressionCodecLZSSE.h} (79%) delete mode 100644 src/Compression/CompressionCodecLZSSE2.cpp delete mode 100644 src/Compression/CompressionCodecLZSSE2.h delete mode 100644 src/Compression/CompressionCodecLZSSE4.cpp delete mode 100644 src/Compression/CompressionCodecLZSSE4.h delete mode 100644 src/Compression/CompressionCodecLZSSE8.cpp diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index 4404b890899..bd1fb25e18f 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB diff --git a/src/Compression/CompressionCodecLZSSE.cpp b/src/Compression/CompressionCodecLZSSE.cpp new file mode 100644 index 00000000000..05c5f507e24 --- /dev/null +++ b/src/Compression/CompressionCodecLZSSE.cpp @@ -0,0 +1,129 @@ +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; + extern const int LOGICAL_ERROR; +} + +CompressionCodecLZSSE::CompressionCodecLZSSE(UInt32 type_, UInt32 level_) : type(type_), level(level_) +{ + if (type != 2 && type != 4 && type != 8) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no LZSSE{} codec", type); + + setCodecDescription(fmt::format("LZSSE{}", type), {std::make_shared(static_cast(level))}); +} + +uint8_t CompressionCodecLZSSE::getMethodByte() const +{ + switch (type) + { + case 2: return static_cast(CompressionMethodByte::LZSSE2); + case 4: return static_cast(CompressionMethodByte::LZSSE4); + case 8: return static_cast(CompressionMethodByte::LZSSE8); + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no LZSSE{} codec", type); + } +} + +void CompressionCodecLZSSE::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + +UInt32 CompressionCodecLZSSE::getMaxCompressedDataSize(UInt32 uncompressed_size) const +{ + return uncompressed_size; +} + +UInt32 CompressionCodecLZSSE::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + UInt32 res = 0; + switch (type) + { + case 2: + { + LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); + res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE2_FreeOptimalParseState(state); + break; + } + case 4: + { + LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); + res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE2_FreeOptimalParseState(state); + break; + } + case 8: + { + LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); + res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE2_FreeOptimalParseState(state); + break; + } + default: + break; + } + + if (res == 0) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress block with LZSSE{}", type); + return res; +} + +void CompressionCodecLZSSE::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +{ + UInt32 res = LZSSE2_Decompress(source, source_size, dest, uncompressed_size); + if (res < uncompressed_size) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress block with LZSSE{}", type); +} + +void registerCodecsLZSSE(CompressionCodecFactory & factory) +{ + for (auto [type, method_byte] : std::initializer_list> + { + {2, CompressionMethodByte::LZSSE2}, + {4, CompressionMethodByte::LZSSE4}, + {8, CompressionMethodByte::LZSSE8} + }) + { + factory.registerCompressionCodec( + fmt::format("LZSSE{}", type), + uint8_t(method_byte), + [type = type](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = 1; + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() != 1) + throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, + "LZSSE{} codec must have 1 parameter, {} given", type, arguments->children.size()); + + const auto children = arguments->children; + const auto * level_literal = children[0]->as(); + if (!level_literal) + throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, + "LZSSE{} first codec argument must be integer", type); + + level = level_literal->value.safeGet(); + } + + return std::make_shared(type, level); + }); + } +} + +} diff --git a/src/Compression/CompressionCodecLZSSE8.h b/src/Compression/CompressionCodecLZSSE.h similarity index 79% rename from src/Compression/CompressionCodecLZSSE8.h rename to src/Compression/CompressionCodecLZSSE.h index 3d51a5ead5b..85cc59c7be0 100644 --- a/src/Compression/CompressionCodecLZSSE8.h +++ b/src/Compression/CompressionCodecLZSSE.h @@ -5,29 +5,25 @@ namespace DB { -class CompressionCodecLZSSE8 : public ICompressionCodec + +class CompressionCodecLZSSE : public ICompressionCodec { public: - - CompressionCodecLZSSE8(int level_); + CompressionCodecLZSSE(UInt32 type_, UInt32 level_); uint8_t getMethodByte() const override; - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - void updateHash(SipHash & hash) const override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } private: - const int level; + const UInt32 type; + const UInt32 level; }; } diff --git a/src/Compression/CompressionCodecLZSSE2.cpp b/src/Compression/CompressionCodecLZSSE2.cpp deleted file mode 100644 index bbbe63690f0..00000000000 --- a/src/Compression/CompressionCodecLZSSE2.cpp +++ /dev/null @@ -1,91 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int CANNOT_COMPRESS; - extern const int CANNOT_DECOMPRESS; - extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; - extern const int ILLEGAL_CODEC_PARAMETER; -} - -CompressionCodecLZSSE2::CompressionCodecLZSSE2(int level_) : level(level_) -{ - - setCodecDescription("LZSSE2", {std::make_shared(static_cast(level))}); -} - -uint8_t CompressionCodecLZSSE2::getMethodByte() const -{ - return static_cast(CompressionMethodByte::LZSSE2); -} - -void CompressionCodecLZSSE2::updateHash(SipHash & hash) const -{ - getCodecDesc()->updateTreeHash(hash); -} - -UInt32 CompressionCodecLZSSE2::getMaxCompressedDataSize(UInt32 uncompressed_size) const -{ - return uncompressed_size; -} - -UInt32 CompressionCodecLZSSE2::doCompressData(const char * source, UInt32 source_size, char * dest) const -{ - UInt32 res; - LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); - res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); - LZSSE2_FreeOptimalParseState(state); - - if (res == 0) - throw Exception("Cannot compress block with LZSSE2", ErrorCodes::CANNOT_COMPRESS); - - return res; -} - -void CompressionCodecLZSSE2::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const -{ - UInt32 res; - res = LZSSE2_Decompress(source, source_size, dest, uncompressed_size); - - if (res < uncompressed_size) - throw Exception("Cannot decompress block with LZSSE2", ErrorCodes::CANNOT_DECOMPRESS); -} - -void registerCodecLZSSE2(CompressionCodecFactory & factory) -{ - UInt8 method_code = UInt8(CompressionMethodByte::LZSSE2); - factory.registerCompressionCodec( - "LZSSE2", - method_code, - [&](const ASTPtr & arguments) -> CompressionCodecPtr - { - int level = 1; - if (arguments && !arguments->children.empty()) - { - if (arguments->children.size() != 1) - throw Exception( - "LZSSE2 codec must have 1 parameter, given " + std::to_string(arguments->children.size()), - ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); - - const auto children = arguments->children; - const auto * level_literal = children[0]->as(); - if (!level_literal) - throw Exception("LZSSE2 first codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - - level = level_literal->value.safeGet(); - - } - - return std::make_shared(level); - }); -} - -} diff --git a/src/Compression/CompressionCodecLZSSE2.h b/src/Compression/CompressionCodecLZSSE2.h deleted file mode 100644 index d644645e3ac..00000000000 --- a/src/Compression/CompressionCodecLZSSE2.h +++ /dev/null @@ -1,33 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class CompressionCodecLZSSE2 : public ICompressionCodec -{ -public: - CompressionCodecLZSSE2(int level_); - - uint8_t getMethodByte() const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - - bool isGenericCompression() const override { return true; } - -private: - const int level; -}; - -} diff --git a/src/Compression/CompressionCodecLZSSE4.cpp b/src/Compression/CompressionCodecLZSSE4.cpp deleted file mode 100644 index 7189a9cee55..00000000000 --- a/src/Compression/CompressionCodecLZSSE4.cpp +++ /dev/null @@ -1,90 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int CANNOT_COMPRESS; - extern const int CANNOT_DECOMPRESS; - extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; - extern const int ILLEGAL_CODEC_PARAMETER; -} - -CompressionCodecLZSSE4::CompressionCodecLZSSE4(int level_) : level(level_) -{ - setCodecDescription("LZSSE4", {std::make_shared(static_cast(level))}); -} - -uint8_t CompressionCodecLZSSE4::getMethodByte() const -{ - return static_cast(CompressionMethodByte::LZSSE4); -} - -void CompressionCodecLZSSE4::updateHash(SipHash & hash) const -{ - getCodecDesc()->updateTreeHash(hash); -} - -UInt32 CompressionCodecLZSSE4::getMaxCompressedDataSize(UInt32 uncompressed_size) const -{ - return uncompressed_size; -} - -UInt32 CompressionCodecLZSSE4::doCompressData(const char * source, UInt32 source_size, char * dest) const -{ - UInt32 res; - LZSSE4_OptimalParseState * state = LZSSE4_MakeOptimalParseState(source_size); - res = LZSSE4_CompressOptimalParse(state, source, source_size, dest, source_size, level); - LZSSE4_FreeOptimalParseState(state); - - if (res == 0) - throw Exception("Cannot compress block with LZSSE4", ErrorCodes::CANNOT_COMPRESS); - - return res; -} - -void CompressionCodecLZSSE4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const -{ - UInt32 res; - res = LZSSE4_Decompress(source, source_size, dest, uncompressed_size); - - if (res < uncompressed_size) - throw Exception("Cannot decompress block with LZSSE4", ErrorCodes::CANNOT_DECOMPRESS); -} - -void registerCodecLZSSE4(CompressionCodecFactory & factory) -{ - UInt8 method_code = UInt8(CompressionMethodByte::LZSSE4); - factory.registerCompressionCodec( - "LZSSE4", - method_code, - [&](const ASTPtr & arguments) -> CompressionCodecPtr - { - int level = 1; - if (arguments && !arguments->children.empty()) - { - if (arguments->children.size() != 1) - throw Exception( - "LZSSE4 codec must have 1 parameter, given " + std::to_string(arguments->children.size()), - ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); - - const auto children = arguments->children; - const auto * level_literal = children[0]->as(); - if (!level_literal) - throw Exception("LZSSE4 first codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - - level = level_literal->value.safeGet(); - - } - - return std::make_shared(level); - }); -} - -} diff --git a/src/Compression/CompressionCodecLZSSE4.h b/src/Compression/CompressionCodecLZSSE4.h deleted file mode 100644 index 9c59e56a053..00000000000 --- a/src/Compression/CompressionCodecLZSSE4.h +++ /dev/null @@ -1,33 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ -class CompressionCodecLZSSE4 : public ICompressionCodec -{ -public: - - CompressionCodecLZSSE4(int level_); - - uint8_t getMethodByte() const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - - bool isGenericCompression() const override { return true; } - -private: - const int level; -}; - -} diff --git a/src/Compression/CompressionCodecLZSSE8.cpp b/src/Compression/CompressionCodecLZSSE8.cpp deleted file mode 100644 index e1e86c75f4e..00000000000 --- a/src/Compression/CompressionCodecLZSSE8.cpp +++ /dev/null @@ -1,90 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int CANNOT_COMPRESS; - extern const int CANNOT_DECOMPRESS; - extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; - extern const int ILLEGAL_CODEC_PARAMETER; -} - -CompressionCodecLZSSE8::CompressionCodecLZSSE8(int level_) : level(level_) -{ - setCodecDescription("LZSSE8", {std::make_shared(static_cast(level))}); -} - -uint8_t CompressionCodecLZSSE8::getMethodByte() const -{ - return static_cast(CompressionMethodByte::LZSSE8); -} - -void CompressionCodecLZSSE8::updateHash(SipHash & hash) const -{ - getCodecDesc()->updateTreeHash(hash); -} - -UInt32 CompressionCodecLZSSE8::getMaxCompressedDataSize(UInt32 uncompressed_size) const -{ - return uncompressed_size; -} - -UInt32 CompressionCodecLZSSE8::doCompressData(const char * source, UInt32 source_size, char * dest) const -{ - UInt32 res; - LZSSE8_OptimalParseState * state = LZSSE8_MakeOptimalParseState(source_size); - res = LZSSE8_CompressOptimalParse(state, source, source_size, dest, source_size, level); - LZSSE8_FreeOptimalParseState(state); - - if (res == 0) - throw Exception("Cannot compress block with LZSSE", ErrorCodes::CANNOT_COMPRESS); - - return res; -} - -void CompressionCodecLZSSE8::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const -{ - UInt32 res; - res = LZSSE8_Decompress(source, source_size, dest, uncompressed_size); - - if (res < uncompressed_size) - throw Exception("Cannot decompress block with LZSSE", ErrorCodes::CANNOT_DECOMPRESS); -} - -void registerCodecLZSSE8(CompressionCodecFactory & factory) -{ - UInt8 method_code = UInt8(CompressionMethodByte::LZSSE8); - factory.registerCompressionCodec( - "LZSSE8", - method_code, - [&](const ASTPtr & arguments) -> CompressionCodecPtr - { - int level = 1; - if (arguments && !arguments->children.empty()) - { - if (arguments->children.size() != 1) - throw Exception( - "LZSSE8 codec must have 1 parameter, given " + std::to_string(arguments->children.size()), - ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); - - const auto children = arguments->children; - const auto * level_literal = children[0]->as(); - if (!level_literal) - throw Exception("LZSSE8 first codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - - level = level_literal->value.safeGet(); - - } - - return std::make_shared(level); - }); -} - -} diff --git a/src/Compression/CompressionCodecLizard.cpp b/src/Compression/CompressionCodecLizard.cpp index 2006a708493..17486693a7b 100644 --- a/src/Compression/CompressionCodecLizard.cpp +++ b/src/Compression/CompressionCodecLizard.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index a703040e98c..ebd30fea535 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -61,7 +61,8 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std } } -void CompressionCodecFactory::validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const +void CompressionCodecFactory::validateCodec( + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const { if (family_name.empty()) throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS); @@ -69,12 +70,14 @@ void CompressionCodecFactory::validateCodec(const String & family_name, std::opt if (level) { auto literal = std::make_shared(static_cast(*level)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check, allow_experimental_codecs); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), + {}, sanity_check, allow_experimental_codecs); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), {}, sanity_check, allow_experimental_codecs); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), + {}, sanity_check, allow_experimental_codecs); } } @@ -140,7 +143,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( if (column_type) { CompressionCodecPtr prev_codec; - IDataType::StreamCallbackWithType callback = [&](const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) + IDataType::StreamCallbackWithType callback = [&]( + const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) { if (ISerialization::isSpecialCompressionAllowed(substream_path)) { @@ -229,7 +233,9 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); } -CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const + +CompressionCodecPtr CompressionCodecFactory::get( + const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const { if (current_default == nullptr) current_default = default_codec; @@ -280,6 +286,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IData throw Exception("Unexpected AST structure for compression codec: " + queryToString(ast), ErrorCodes::UNEXPECTED_AST_STRUCTURE); } + CompressionCodecPtr CompressionCodecFactory::get(const uint8_t byte_code) const { const auto family_code_and_creator = family_code_with_codec.find(byte_code); @@ -337,7 +344,7 @@ void CompressionCodecFactory::registerSimpleCompressionCodec( registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast) { if (ast) - throw Exception("Compression codec " + family_name + " cannot have arguments", ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS); + throw Exception(ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS, "Compression codec {} cannot have arguments", family_name); return creator(); }); } @@ -354,9 +361,7 @@ void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); void registerCodecLizard(CompressionCodecFactory & factory); void registerCodecDensity(CompressionCodecFactory & factory); -void registerCodecLZSSE2(CompressionCodecFactory & factory); -void registerCodecLZSSE4(CompressionCodecFactory & factory); -void registerCodecLZSSE8(CompressionCodecFactory & factory); +void registerCodecsLZSSE(CompressionCodecFactory & factory); CompressionCodecFactory::CompressionCodecFactory() { @@ -371,9 +376,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecMultiple(*this); registerCodecLizard(*this); registerCodecDensity(*this); - registerCodecLZSSE2(*this); - registerCodecLZSSE4(*this); - registerCodecLZSSE8(*this); + registerCodecsLZSSE(*this); default_codec = get("LZ4", {}); } diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h index d1d4ad4618f..44bed7d3c1d 100644 --- a/src/Compression/CompressionInfo.h +++ b/src/Compression/CompressionInfo.h @@ -47,7 +47,7 @@ enum class CompressionMethodByte : uint8_t Density = 0x97, LZSSE2 = 0x98, LZSSE4 = 0x99, - LZSSE8 = 0xa0, + LZSSE8 = 0xA0, }; diff --git a/src/Storages/System/StorageSystemErrors.cpp b/src/Storages/System/StorageSystemErrors.cpp index d08ffd730ac..4c8c8e60d69 100644 --- a/src/Storages/System/StorageSystemErrors.cpp +++ b/src/Storages/System/StorageSystemErrors.cpp @@ -3,8 +3,9 @@ #include #include #include -#include #include +#include + namespace DB { From deb68b15daeaa7e4279140cf4fa09bd1cdd154e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 04:03:42 +0300 Subject: [PATCH 170/652] Fix bad dependencies in code --- src/Compression/CompressionCodecDensity.h | 2 +- src/Compression/CompressionCodecLZSSE.h | 1 + src/Compression/CompressionCodecLizard.h | 3 +-- src/Compression/CompressionFactory.cpp | 23 +++++++---------------- src/Compression/ICompressionCodec.h | 4 ++++ 5 files changed, 14 insertions(+), 19 deletions(-) diff --git a/src/Compression/CompressionCodecDensity.h b/src/Compression/CompressionCodecDensity.h index ffdbd8f3619..8b6f2a46109 100644 --- a/src/Compression/CompressionCodecDensity.h +++ b/src/Compression/CompressionCodecDensity.h @@ -25,8 +25,8 @@ protected: void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } private: const DENSITY_ALGORITHM algo; diff --git a/src/Compression/CompressionCodecLZSSE.h b/src/Compression/CompressionCodecLZSSE.h index 85cc59c7be0..8c32987f873 100644 --- a/src/Compression/CompressionCodecLZSSE.h +++ b/src/Compression/CompressionCodecLZSSE.h @@ -20,6 +20,7 @@ protected: void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; bool isCompression() const override { return true; } bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } private: const UInt32 type; diff --git a/src/Compression/CompressionCodecLizard.h b/src/Compression/CompressionCodecLizard.h index 8b39f224dbc..7304297a80c 100644 --- a/src/Compression/CompressionCodecLizard.h +++ b/src/Compression/CompressionCodecLizard.h @@ -20,12 +20,11 @@ public: 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; bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } private: const int level; diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index ebd30fea535..d73c94c8cc3 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -93,7 +93,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( std::optional generic_compression_codec_pos; bool can_substitute_codec_arguments = true; - for (size_t i = 0; i < func->arguments->children.size(); ++i) + for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) { const auto & inner_codec_ast = func->arguments->children[i]; String codec_family_name; @@ -111,21 +111,6 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( else throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (!allow_experimental_codecs) - { - if (codec_family_name == "Lizard" || - codec_family_name == "Density" || - codec_family_name == "LZSSE2" || - codec_family_name == "LZSSE4" || - codec_family_name == "LZSSE8") - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Codec {} is experimental and not meant to be used in production." - " You can enable it with the 'allow_experimental_codecs' setting.", - codec_family_name); - } - } - /// Default codec replaced with current default codec which may depend on different /// settings (and properties of data) in runtime. CompressionCodecPtr result_codec; @@ -169,6 +154,12 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( result_codec = getImpl(codec_family_name, codec_arguments, nullptr); } + if (!allow_experimental_codecs && result_codec->isExperimental()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Codec {} is experimental and not meant to be used in production." + " You can enable it with the 'allow_experimental_codecs' setting.", + codec_family_name); + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 2018218f2e7..47b4d9bfb43 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -73,6 +73,10 @@ public: /// Is it a generic compression algorithm like lz4, zstd. Usually it does not make sense to apply generic compression more than single time. virtual bool isGenericCompression() const = 0; + /// It is a codec available only for evaluation purposes and not meant to be used in production. + /// It will not be allowed to use unless the user will turn off the safety switch. + virtual bool isExperimental() const { return false; } + /// If it does nothing. virtual bool isNone() const { return false; } From 3057bbe8312b4a9c72ec82f9b87287d553066fbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 04:12:30 +0300 Subject: [PATCH 171/652] Remove header files --- src/Compression/CompressionCodecDelta.cpp | 25 +++- src/Compression/CompressionCodecDelta.h | 32 ----- src/Compression/CompressionCodecDensity.cpp | 32 ++++- src/Compression/CompressionCodecDensity.h | 35 ------ .../CompressionCodecDoubleDelta.cpp | 114 ++++++++++++++++- src/Compression/CompressionCodecDoubleDelta.h | 118 ------------------ src/Compression/CompressionCodecGorilla.cpp | 111 +++++++++++++++- src/Compression/CompressionCodecGorilla.h | 115 ----------------- src/Compression/CompressionCodecLZ4.cpp | 55 +++++++- src/Compression/CompressionCodecLZ4.h | 52 -------- src/Compression/CompressionCodecLZSSE.cpp | 25 +++- src/Compression/CompressionCodecLZSSE.h | 30 ----- src/Compression/CompressionCodecLizard.cpp | 28 ++++- src/Compression/CompressionCodecLizard.h | 33 ----- src/Compression/CompressionCodecT64.cpp | 57 ++++++++- src/Compression/CompressionCodecT64.h | 53 -------- src/Compression/CompressionCodecZSTD.cpp | 36 +++++- src/Compression/CompressionCodecZSTD.h | 42 ------- 18 files changed, 465 insertions(+), 528 deletions(-) delete mode 100644 src/Compression/CompressionCodecDelta.h delete mode 100644 src/Compression/CompressionCodecDensity.h delete mode 100644 src/Compression/CompressionCodecDoubleDelta.h delete mode 100644 src/Compression/CompressionCodecGorilla.h delete mode 100644 src/Compression/CompressionCodecLZ4.h delete mode 100644 src/Compression/CompressionCodecLZSSE.h delete mode 100644 src/Compression/CompressionCodecLizard.h delete mode 100644 src/Compression/CompressionCodecT64.h delete mode 100644 src/Compression/CompressionCodecZSTD.h diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index f3d6953bd2a..7d0824d4d71 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -11,6 +11,29 @@ namespace DB { +class CompressionCodecDelta : public ICompressionCodec +{ +public: + CompressionCodecDelta(UInt8 delta_bytes_size_); + + uint8_t getMethodByte() const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; } + + bool isCompression() const override { return false; } + bool isGenericCompression() const override { return false; } + +private: + UInt8 delta_bytes_size; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecDelta.h b/src/Compression/CompressionCodecDelta.h deleted file mode 100644 index e892aa04242..00000000000 --- a/src/Compression/CompressionCodecDelta.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class CompressionCodecDelta : public ICompressionCodec -{ -public: - CompressionCodecDelta(UInt8 delta_bytes_size_); - - uint8_t getMethodByte() const override; - - void updateHash(SipHash & hash) const override; - -protected: - - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; } - - bool isCompression() const override { return false; } - bool isGenericCompression() const override { return false; } - -private: - UInt8 delta_bytes_size; -}; - -} diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index bd1fb25e18f..cdee69005be 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -1,11 +1,41 @@ +#include #include #include -#include #include +#include + namespace DB { + +class CompressionCodecDensity : public ICompressionCodec +{ +public: + static constexpr auto DENSITY_DEFAULT_ALGO = DENSITY_ALGORITHM_CHAMELEON; // by default aim on speed + + CompressionCodecDensity(DENSITY_ALGORITHM algo_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } + +private: + const DENSITY_ALGORITHM algo; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecDensity.h b/src/Compression/CompressionCodecDensity.h deleted file mode 100644 index 8b6f2a46109..00000000000 --- a/src/Compression/CompressionCodecDensity.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ -class CompressionCodecDensity : public ICompressionCodec -{ -public: - static constexpr auto DENSITY_DEFAULT_ALGO = DENSITY_ALGORITHM_CHAMELEON; // by default aim on speed - - CompressionCodecDensity(DENSITY_ALGORITHM algo_); - - uint8_t getMethodByte() const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } - bool isExperimental() const override { return true; } - -private: - const DENSITY_ALGORITHM algo; -}; - -} diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 6895a80264d..d992132c4f6 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -15,9 +15,121 @@ #include #include + namespace DB { +/** DoubleDelta column codec implementation. + * + * Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf, which was extended + * to support 64bit types. The drawback is 1 extra bit for 32-byte wide deltas: 5-bit prefix + * instead of 4-bit prefix. + * + * This codec is best used against monotonic integer sequences with constant (or almost constant) + * stride, like event timestamp for some monitoring application. + * + * Given input sequence a: [a0, a1, ... an]: + * + * First, write number of items (sizeof(int32)*8 bits): n + * Then write first item as is (sizeof(a[0])*8 bits): a[0] + * Second item is written as delta (sizeof(a[0])*8 bits): a[1] - a[0] + * Loop over remaining items and calculate double delta: + * double_delta = a[i] - 2 * a[i - 1] + a[i - 2] + * Write it in compact binary form with `BitWriter` + * if double_delta == 0: + * write 1bit: 0 + * else if -63 < double_delta < 64: + * write 2 bit prefix: 10 + * write sign bit (1 if signed): x + * write 7-1 bits of abs(double_delta - 1): xxxxxx + * else if -255 < double_delta < 256: + * write 3 bit prefix: 110 + * write sign bit (1 if signed): x + * write 9-1 bits of abs(double_delta - 1): xxxxxxxx + * else if -2047 < double_delta < 2048: + * write 4 bit prefix: 1110 + * write sign bit (1 if signed): x + * write 12-1 bits of abs(double_delta - 1): xxxxxxxxxxx + * else if double_delta fits into 32-bit int: + * write 5 bit prefix: 11110 + * write sign bit (1 if signed): x + * write 32-1 bits of abs(double_delta - 1): xxxxxxxxxxx... + * else + * write 5 bit prefix: 11111 + * write sign bit (1 if signed): x + * write 64-1 bits of abs(double_delta - 1): xxxxxxxxxxx... + * + * @example sequence of UInt8 values [1, 2, 3, 4, 5, 6, 7, 8, 9 10] is encoded as (codec header is omitted): + * + * .- 4-byte little-endian sequence length (10 == 0xa) + * | .- 1 byte (sizeof(UInt8) a[0] : 0x01 + * | | .- 1 byte of delta: a[1] - a[0] = 2 - 1 = 1 : 0x01 + * | | | .- 8 zero bits since double delta for remaining 8 elements was 0 : 0x00 + * v_______________v___v___v___ + * \x0a\x00\x00\x00\x01\x01\x00 + * + * @example sequence of Int16 values [-10, 10, -20, 20, -40, 40] is encoded as: + * + * .- 4-byte little endian sequence length = 6 : 0x00000006 + * | .- 2 bytes (sizeof(Int16) a[0] as UInt16 = -10 : 0xfff6 + * | | .- 2 bytes of delta: a[1] - a[0] = 10 - (-10) = 20 : 0x0014 + * | | | .- 4 encoded double deltas (see below) + * v_______________ v______ v______ v______________________ + * \x06\x00\x00\x00\xf6\xff\x14\x00\xb8\xe2\x2e\xb1\xe4\x58 + * + * 4 binary encoded double deltas (\xb8\xe2\x2e\xb1\xe4\x58): + * double_delta (DD) = -20 - 2 * 10 + (-10) = -50 + * .- 2-bit prefix : 0b10 + * | .- sign-bit : 0b1 + * | |.- abs(DD - 1) = 49 : 0b110001 + * | || + * | || DD = 20 - 2 * (-20) + 10 = 70 + * | || .- 3-bit prefix : 0b110 + * | || | .- sign bit : 0b0 + * | || | |.- abs(DD - 1) = 69 : 0b1000101 + * | || | || + * | || | || DD = -40 - 2 * 20 + (-20) = -100 + * | || | || .- 3-bit prefix : 0b110 + * | || | || | .- sign-bit : 0b0 + * | || | || | |.- abs(DD - 1) = 99 : 0b1100011 + * | || | || | || + * | || | || | || DD = 40 - 2 * (-40) + 20 = 140 + * | || | || | || .- 3-bit prefix : 0b110 + * | || | || | || | .- sign bit : 0b0 + * | || | || | || | |.- abs(DD - 1) = 139 : 0b10001011 + * | || | || | || | || + * V_vv______V__vv________V____vv_______V__vv________,- padding bits + * 10111000 11100010 00101110 10110001 11100100 01011000 + * + * Please also see unit tests for: + * * Examples on what output `BitWriter` produces on predefined input. + * * Compatibility tests solidifying encoded binary output on set of predefined sequences. + */ +class CompressionCodecDoubleDelta : public ICompressionCodec +{ +public: + CompressionCodecDoubleDelta(UInt8 data_bytes_size_); + + uint8_t getMethodByte() const override; + + void updateHash(SipHash & hash) const override; + +protected: + + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return false; } + +private: + UInt8 data_bytes_size; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecDoubleDelta.h b/src/Compression/CompressionCodecDoubleDelta.h deleted file mode 100644 index 11140ded61e..00000000000 --- a/src/Compression/CompressionCodecDoubleDelta.h +++ /dev/null @@ -1,118 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** DoubleDelta column codec implementation. - * - * Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf, which was extended - * to support 64bit types. The drawback is 1 extra bit for 32-byte wide deltas: 5-bit prefix - * instead of 4-bit prefix. - * - * This codec is best used against monotonic integer sequences with constant (or almost constant) - * stride, like event timestamp for some monitoring application. - * - * Given input sequence a: [a0, a1, ... an]: - * - * First, write number of items (sizeof(int32)*8 bits): n - * Then write first item as is (sizeof(a[0])*8 bits): a[0] - * Second item is written as delta (sizeof(a[0])*8 bits): a[1] - a[0] - * Loop over remaining items and calculate double delta: - * double_delta = a[i] - 2 * a[i - 1] + a[i - 2] - * Write it in compact binary form with `BitWriter` - * if double_delta == 0: - * write 1bit: 0 - * else if -63 < double_delta < 64: - * write 2 bit prefix: 10 - * write sign bit (1 if signed): x - * write 7-1 bits of abs(double_delta - 1): xxxxxx - * else if -255 < double_delta < 256: - * write 3 bit prefix: 110 - * write sign bit (1 if signed): x - * write 9-1 bits of abs(double_delta - 1): xxxxxxxx - * else if -2047 < double_delta < 2048: - * write 4 bit prefix: 1110 - * write sign bit (1 if signed): x - * write 12-1 bits of abs(double_delta - 1): xxxxxxxxxxx - * else if double_delta fits into 32-bit int: - * write 5 bit prefix: 11110 - * write sign bit (1 if signed): x - * write 32-1 bits of abs(double_delta - 1): xxxxxxxxxxx... - * else - * write 5 bit prefix: 11111 - * write sign bit (1 if signed): x - * write 64-1 bits of abs(double_delta - 1): xxxxxxxxxxx... - * - * @example sequence of UInt8 values [1, 2, 3, 4, 5, 6, 7, 8, 9 10] is encoded as (codec header is omitted): - * - * .- 4-byte little-endian sequence length (10 == 0xa) - * | .- 1 byte (sizeof(UInt8) a[0] : 0x01 - * | | .- 1 byte of delta: a[1] - a[0] = 2 - 1 = 1 : 0x01 - * | | | .- 8 zero bits since double delta for remaining 8 elements was 0 : 0x00 - * v_______________v___v___v___ - * \x0a\x00\x00\x00\x01\x01\x00 - * - * @example sequence of Int16 values [-10, 10, -20, 20, -40, 40] is encoded as: - * - * .- 4-byte little endian sequence length = 6 : 0x00000006 - * | .- 2 bytes (sizeof(Int16) a[0] as UInt16 = -10 : 0xfff6 - * | | .- 2 bytes of delta: a[1] - a[0] = 10 - (-10) = 20 : 0x0014 - * | | | .- 4 encoded double deltas (see below) - * v_______________ v______ v______ v______________________ - * \x06\x00\x00\x00\xf6\xff\x14\x00\xb8\xe2\x2e\xb1\xe4\x58 - * - * 4 binary encoded double deltas (\xb8\xe2\x2e\xb1\xe4\x58): - * double_delta (DD) = -20 - 2 * 10 + (-10) = -50 - * .- 2-bit prefix : 0b10 - * | .- sign-bit : 0b1 - * | |.- abs(DD - 1) = 49 : 0b110001 - * | || - * | || DD = 20 - 2 * (-20) + 10 = 70 - * | || .- 3-bit prefix : 0b110 - * | || | .- sign bit : 0b0 - * | || | |.- abs(DD - 1) = 69 : 0b1000101 - * | || | || - * | || | || DD = -40 - 2 * 20 + (-20) = -100 - * | || | || .- 3-bit prefix : 0b110 - * | || | || | .- sign-bit : 0b0 - * | || | || | |.- abs(DD - 1) = 99 : 0b1100011 - * | || | || | || - * | || | || | || DD = 40 - 2 * (-40) + 20 = 140 - * | || | || | || .- 3-bit prefix : 0b110 - * | || | || | || | .- sign bit : 0b0 - * | || | || | || | |.- abs(DD - 1) = 139 : 0b10001011 - * | || | || | || | || - * V_vv______V__vv________V____vv_______V__vv________,- padding bits - * 10111000 11100010 00101110 10110001 11100100 01011000 - * - * Please also see unit tests for: - * * Examples on what output `BitWriter` produces on predefined input. - * * Compatibility tests solidifying encoded binary output on set of predefined sequences. - */ -class CompressionCodecDoubleDelta : public ICompressionCodec -{ -public: - CompressionCodecDoubleDelta(UInt8 data_bytes_size_); - - uint8_t getMethodByte() const override; - - void updateHash(SipHash & hash) const override; - -protected: - - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return false; } - -private: - UInt8 data_bytes_size; -}; - -} diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 582a9f41874..db76a854a97 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -14,9 +14,118 @@ #include + namespace DB { +/** Gorilla column codec implementation. + * + * Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf + * + * This codec is best used against monotonic floating sequences, like CPU usage percentage + * or any other gauge. + * + * Given input sequence a: [a0, a1, ... an] + * + * First, write number of items (sizeof(int32)*8 bits): n + * Then write first item as is (sizeof(a[0])*8 bits): a[0] + * Loop over remaining items and calculate xor_diff: + * xor_diff = a[i] ^ a[i - 1] (e.g. 00000011'10110100) + * Write it in compact binary form with `BitWriter` + * if xor_diff == 0: + * write 1 bit: 0 + * else: + * calculate leading zero bits (lzb) + * and trailing zero bits (tzb) of xor_diff, + * compare to lzb and tzb of previous xor_diff + * (X = sizeof(a[i]) * 8, e.g. X = 16, lzb = 6, tzb = 2) + * if lzb >= prev_lzb && tzb >= prev_tzb: + * (e.g. prev_lzb=4, prev_tzb=1) + * write 2 bit prefix: 0b10 + * write xor_diff >> prev_tzb (X - prev_lzb - prev_tzb bits):0b00111011010 + * (where X = sizeof(a[i]) * 8, e.g. 16) + * else: + * write 2 bit prefix: 0b11 + * write 5 bits of lzb: 0b00110 + * write 6 bits of (X - lzb - tzb)=(16-6-2)=8: 0b001000 + * write (X - lzb - tzb) non-zero bits of xor_diff: 0b11101101 + * prev_lzb = lzb + * prev_tzb = tzb + * + * @example sequence of Float32 values [0.1, 0.1, 0.11, 0.2, 0.1] is encoded as: + * + * .- 4-byte little endian sequence length: 5 : 0x00000005 + * | .- 4 byte (sizeof(Float32) a[0] as UInt32 : -10 : 0xcdcccc3d + * | | .- 4 encoded xor diffs (see below) + * v_______________ v______________ v__________________________________________________ + * \x05\x00\x00\x00\xcd\xcc\xcc\x3d\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00 + * + * 4 binary encoded xor diffs (\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00): + * + * ........................................... + * a[i-1] = 00111101110011001100110011001101 + * a[i] = 00111101110011001100110011001101 + * xor_diff = 00000000000000000000000000000000 + * .- 1-bit prefix : 0b0 + * | + * | ........................................... + * | a[i-1] = 00111101110011001100110011001101 + * ! a[i] = 00111101111000010100011110101110 + * | xor_diff = 00000000001011011000101101100011 + * | lzb = 10 + * | tzb = 0 + * |.- 2-bit prefix : 0b11 + * || .- lzb (10) : 0b1010 + * || | .- data length (32-10-0): 22 : 0b010110 + * || | | .- data : 0b1011011000101101100011 + * || | | | + * || | | | ........................................... + * || | | | a[i-1] = 00111101111000010100011110101110 + * || | | | a[i] = 00111110010011001100110011001101 + * || | | | xor_diff = 00000011101011011000101101100011 + * || | | | .- 2-bit prefix : 0b11 + * || | | | | .- lzb = 6 : 0b00110 + * || | | | | | .- data length = (32 - 6) = 26 : 0b011010 + * || | | | | | | .- data : 0b11101011011000101101100011 + * || | | | | | | | + * || | | | | | | | ........................................... + * || | | | | | | | a[i-1] = 00111110010011001100110011001101 + * || | | | | | | | a[i] = 00111101110011001100110011001101 + * || | | | | | | | xor_diff = 00000011100000000000000000000000 + * || | | | | | | | .- 2-bit prefix : 0b10 + * || | | | | | | | | .- data : 0b11100000000000000000000000 + * VV_v____ v_____v________________________V_v_____v______v____________________________V_v_____________________________ + * 01101010 01011010 11011000 10110110 00111100 11001101 01110101 10110001 01101100 01110111 00000000 00000000 00000000 + * + * Please also see unit tests for: + * * Examples on what output `BitWriter` produces on predefined input. + * * Compatibility tests solidifying encoded binary output on set of predefined sequences. + */ +class CompressionCodecGorilla : public ICompressionCodec +{ +public: + CompressionCodecGorilla(UInt8 data_bytes_size_); + + uint8_t getMethodByte() const override; + + void updateHash(SipHash & hash) const override; + +protected: + + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return false; } + +private: + UInt8 data_bytes_size; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecGorilla.h b/src/Compression/CompressionCodecGorilla.h deleted file mode 100644 index 3613ab2a96f..00000000000 --- a/src/Compression/CompressionCodecGorilla.h +++ /dev/null @@ -1,115 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** Gorilla column codec implementation. - * - * Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf - * - * This codec is best used against monotonic floating sequences, like CPU usage percentage - * or any other gauge. - * - * Given input sequence a: [a0, a1, ... an] - * - * First, write number of items (sizeof(int32)*8 bits): n - * Then write first item as is (sizeof(a[0])*8 bits): a[0] - * Loop over remaining items and calculate xor_diff: - * xor_diff = a[i] ^ a[i - 1] (e.g. 00000011'10110100) - * Write it in compact binary form with `BitWriter` - * if xor_diff == 0: - * write 1 bit: 0 - * else: - * calculate leading zero bits (lzb) - * and trailing zero bits (tzb) of xor_diff, - * compare to lzb and tzb of previous xor_diff - * (X = sizeof(a[i]) * 8, e.g. X = 16, lzb = 6, tzb = 2) - * if lzb >= prev_lzb && tzb >= prev_tzb: - * (e.g. prev_lzb=4, prev_tzb=1) - * write 2 bit prefix: 0b10 - * write xor_diff >> prev_tzb (X - prev_lzb - prev_tzb bits):0b00111011010 - * (where X = sizeof(a[i]) * 8, e.g. 16) - * else: - * write 2 bit prefix: 0b11 - * write 5 bits of lzb: 0b00110 - * write 6 bits of (X - lzb - tzb)=(16-6-2)=8: 0b001000 - * write (X - lzb - tzb) non-zero bits of xor_diff: 0b11101101 - * prev_lzb = lzb - * prev_tzb = tzb - * - * @example sequence of Float32 values [0.1, 0.1, 0.11, 0.2, 0.1] is encoded as: - * - * .- 4-byte little endian sequence length: 5 : 0x00000005 - * | .- 4 byte (sizeof(Float32) a[0] as UInt32 : -10 : 0xcdcccc3d - * | | .- 4 encoded xor diffs (see below) - * v_______________ v______________ v__________________________________________________ - * \x05\x00\x00\x00\xcd\xcc\xcc\x3d\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00 - * - * 4 binary encoded xor diffs (\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00): - * - * ........................................... - * a[i-1] = 00111101110011001100110011001101 - * a[i] = 00111101110011001100110011001101 - * xor_diff = 00000000000000000000000000000000 - * .- 1-bit prefix : 0b0 - * | - * | ........................................... - * | a[i-1] = 00111101110011001100110011001101 - * ! a[i] = 00111101111000010100011110101110 - * | xor_diff = 00000000001011011000101101100011 - * | lzb = 10 - * | tzb = 0 - * |.- 2-bit prefix : 0b11 - * || .- lzb (10) : 0b1010 - * || | .- data length (32-10-0): 22 : 0b010110 - * || | | .- data : 0b1011011000101101100011 - * || | | | - * || | | | ........................................... - * || | | | a[i-1] = 00111101111000010100011110101110 - * || | | | a[i] = 00111110010011001100110011001101 - * || | | | xor_diff = 00000011101011011000101101100011 - * || | | | .- 2-bit prefix : 0b11 - * || | | | | .- lzb = 6 : 0b00110 - * || | | | | | .- data length = (32 - 6) = 26 : 0b011010 - * || | | | | | | .- data : 0b11101011011000101101100011 - * || | | | | | | | - * || | | | | | | | ........................................... - * || | | | | | | | a[i-1] = 00111110010011001100110011001101 - * || | | | | | | | a[i] = 00111101110011001100110011001101 - * || | | | | | | | xor_diff = 00000011100000000000000000000000 - * || | | | | | | | .- 2-bit prefix : 0b10 - * || | | | | | | | | .- data : 0b11100000000000000000000000 - * VV_v____ v_____v________________________V_v_____v______v____________________________V_v_____________________________ - * 01101010 01011010 11011000 10110110 00111100 11001101 01110101 10110001 01101100 01110111 00000000 00000000 00000000 - * - * Please also see unit tests for: - * * Examples on what output `BitWriter` produces on predefined input. - * * Compatibility tests solidifying encoded binary output on set of predefined sequences. - */ -class CompressionCodecGorilla : public ICompressionCodec -{ -public: - CompressionCodecGorilla(UInt8 data_bytes_size_); - - uint8_t getMethodByte() const override; - - void updateHash(SipHash & hash) const override; - -protected: - - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return false; } - -private: - UInt8 data_bytes_size; -}; - -} diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index 5f43b49706f..597a6c195eb 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -1,7 +1,7 @@ -#include "CompressionCodecLZ4.h" - #include #include + +#include #include #include #include @@ -9,7 +9,12 @@ #include #include #include +#include #include +#include + +#include + #pragma GCC diagnostic ignored "-Wold-style-cast" @@ -17,11 +22,51 @@ namespace DB { +class CompressionCodecLZ4 : public ICompressionCodec +{ +public: + CompressionCodecLZ4(); + + uint8_t getMethodByte() const override; + + UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; } + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + +private: + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + mutable LZ4::PerformanceStatistics lz4_stat; + ASTPtr codec_desc; +}; + + +class CompressionCodecLZ4HC : public CompressionCodecLZ4 +{ +public: + CompressionCodecLZ4HC(int level_); + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + +private: + const int level; +}; + + namespace ErrorCodes { -extern const int CANNOT_COMPRESS; -extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; -extern const int ILLEGAL_CODEC_PARAMETER; + extern const int CANNOT_COMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; } CompressionCodecLZ4::CompressionCodecLZ4() diff --git a/src/Compression/CompressionCodecLZ4.h b/src/Compression/CompressionCodecLZ4.h deleted file mode 100644 index bf8b4e2dd1f..00000000000 --- a/src/Compression/CompressionCodecLZ4.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class CompressionCodecLZ4 : public ICompressionCodec -{ -public: - CompressionCodecLZ4(); - - uint8_t getMethodByte() const override; - - UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; } - - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } - -private: - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - mutable LZ4::PerformanceStatistics lz4_stat; - ASTPtr codec_desc; -}; - - -class CompressionCodecLZ4HC : public CompressionCodecLZ4 -{ -public: - CompressionCodecLZ4HC(int level_); - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - -private: - const int level; -}; - -} diff --git a/src/Compression/CompressionCodecLZSSE.cpp b/src/Compression/CompressionCodecLZSSE.cpp index 05c5f507e24..50585766b34 100644 --- a/src/Compression/CompressionCodecLZSSE.cpp +++ b/src/Compression/CompressionCodecLZSSE.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -10,6 +10,29 @@ namespace DB { + +class CompressionCodecLZSSE : public ICompressionCodec +{ +public: + CompressionCodecLZSSE(UInt32 type_, UInt32 level_); + + uint8_t getMethodByte() const override; + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } + +private: + const UInt32 type; + const UInt32 level; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecLZSSE.h b/src/Compression/CompressionCodecLZSSE.h deleted file mode 100644 index 8c32987f873..00000000000 --- a/src/Compression/CompressionCodecLZSSE.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class CompressionCodecLZSSE : public ICompressionCodec -{ -public: - CompressionCodecLZSSE(UInt32 type_, UInt32 level_); - - uint8_t getMethodByte() const override; - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } - bool isExperimental() const override { return true; } - -private: - const UInt32 type; - const UInt32 level; -}; - -} diff --git a/src/Compression/CompressionCodecLizard.cpp b/src/Compression/CompressionCodecLizard.cpp index 17486693a7b..2059b12bab1 100644 --- a/src/Compression/CompressionCodecLizard.cpp +++ b/src/Compression/CompressionCodecLizard.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,6 +8,32 @@ namespace DB { +class CompressionCodecLizard : public ICompressionCodec +{ +public: + static constexpr auto LIZARD_DEFAULT_LEVEL = 1; + + CompressionCodecLizard(int level_); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + bool isExperimental() const override { return true; } + +private: + const int level; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecLizard.h b/src/Compression/CompressionCodecLizard.h deleted file mode 100644 index 7304297a80c..00000000000 --- a/src/Compression/CompressionCodecLizard.h +++ /dev/null @@ -1,33 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ -class CompressionCodecLizard : public ICompressionCodec -{ -public: - static constexpr auto LIZARD_DEFAULT_LEVEL = 1; - - CompressionCodecLizard(int level_); - - uint8_t getMethodByte() const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } - bool isExperimental() const override { return true; } - -private: - const int level; -}; - -} diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index f0fb8351410..377db915bce 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include @@ -8,18 +8,63 @@ #include #include #include +#include namespace DB { +/// Get 64 integer values, makes 64x64 bit matrix, transpose it and crop unused bits (most significant zeroes). +/// In example, if we have UInt8 with only 0 and 1 inside 64xUInt8 would be compressed into 1xUInt64. +/// It detects unused bits by calculating min and max values of data part, saving them in header in compression phase. +/// There's a special case with signed integers parts with crossing zero data. Here it stores one more bit to detect sign of value. +class CompressionCodecT64 : public ICompressionCodec +{ +public: + static constexpr UInt32 HEADER_SIZE = 1 + 2 * sizeof(UInt64); + static constexpr UInt32 MAX_COMPRESSED_BLOCK_SIZE = sizeof(UInt64) * 64; + + /// There're 2 compression variants: + /// Byte - transpose bit matrix by bytes (only the last not full byte is transposed by bits). It's default. + /// Bits - full bit-transpose of the bit matrix. It uses more resources and leads to better compression with ZSTD (but worse with LZ4). + enum class Variant + { + Byte, + Bit + }; + + CompressionCodecT64(TypeIndex type_idx_, Variant variant_); + + uint8_t getMethodByte() const override; + + void updateHash(SipHash & hash) const override; + +protected: + UInt32 doCompressData(const char * src, UInt32 src_size, char * dst) const override; + void doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override + { + /// uncompressed_size - (uncompressed_size % (sizeof(T) * 64)) + sizeof(UInt64) * sizeof(T) + header_size + return uncompressed_size + MAX_COMPRESSED_BLOCK_SIZE + HEADER_SIZE; + } + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return false; } + +private: + TypeIndex type_idx; + Variant variant; +}; + + namespace ErrorCodes { -extern const int CANNOT_COMPRESS; -extern const int CANNOT_DECOMPRESS; -extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; -extern const int ILLEGAL_CODEC_PARAMETER; -extern const int LOGICAL_ERROR; + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; + extern const int LOGICAL_ERROR; } namespace diff --git a/src/Compression/CompressionCodecT64.h b/src/Compression/CompressionCodecT64.h deleted file mode 100644 index c7664679d5a..00000000000 --- a/src/Compression/CompressionCodecT64.h +++ /dev/null @@ -1,53 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/// Get 64 integer values, makes 64x64 bit matrix, transpose it and crop unused bits (most significant zeroes). -/// In example, if we have UInt8 with only 0 and 1 inside 64xUInt8 would be compressed into 1xUInt64. -/// It detects unused bits by calculating min and max values of data part, saving them in header in compression phase. -/// There's a special case with signed integers parts with crossing zero data. Here it stores one more bit to detect sign of value. -class CompressionCodecT64 : public ICompressionCodec -{ -public: - static constexpr UInt32 HEADER_SIZE = 1 + 2 * sizeof(UInt64); - static constexpr UInt32 MAX_COMPRESSED_BLOCK_SIZE = sizeof(UInt64) * 64; - - /// There're 2 compression variants: - /// Byte - transpose bit matrix by bytes (only the last not full byte is transposed by bits). It's default. - /// Bits - full bit-transpose of the bit matrix. It uses more resources and leads to better compression with ZSTD (but worse with LZ4). - enum class Variant - { - Byte, - Bit - }; - - CompressionCodecT64(TypeIndex type_idx_, Variant variant_); - - uint8_t getMethodByte() const override; - - void updateHash(SipHash & hash) const override; - -protected: - UInt32 doCompressData(const char * src, UInt32 src_size, char * dst) const override; - void doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override - { - /// uncompressed_size - (uncompressed_size % (sizeof(T) * 64)) + sizeof(UInt64) * sizeof(T) + header_size - return uncompressed_size + MAX_COMPRESSED_BLOCK_SIZE + HEADER_SIZE; - } - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return false; } - -private: - TypeIndex type_idx; - Variant variant; -}; - -} diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 378f74ce7cf..4d3b5a181d1 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -7,11 +7,45 @@ #include #include #include +#include +#include namespace DB { +class CompressionCodecZSTD : public ICompressionCodec +{ +public: + static constexpr auto ZSTD_DEFAULT_LEVEL = 1; + static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24; + + CompressionCodecZSTD(int level_); + + CompressionCodecZSTD(int level_, int window_log); + + uint8_t getMethodByte() const override; + + UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + + void updateHash(SipHash & hash) const override; + +protected: + + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + bool isCompression() const override { return true; } + bool isGenericCompression() const override { return true; } + +private: + const int level; + const bool enable_long_range; + const int window_log; +}; + + namespace ErrorCodes { extern const int CANNOT_COMPRESS; diff --git a/src/Compression/CompressionCodecZSTD.h b/src/Compression/CompressionCodecZSTD.h deleted file mode 100644 index 2d67ff9cecb..00000000000 --- a/src/Compression/CompressionCodecZSTD.h +++ /dev/null @@ -1,42 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class CompressionCodecZSTD : public ICompressionCodec -{ -public: - static constexpr auto ZSTD_DEFAULT_LEVEL = 1; - static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24; - - CompressionCodecZSTD(int level_); - - CompressionCodecZSTD(int level_, int window_log); - - uint8_t getMethodByte() const override; - - UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - - void updateHash(SipHash & hash) const override; - -protected: - - UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - - void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - - bool isCompression() const override { return true; } - bool isGenericCompression() const override { return true; } - -private: - const int level; - const bool enable_long_range; - const int window_log; -}; - -} From fdc5bb72f54aa78534047201fc10644d1bb14d81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 04:14:29 +0300 Subject: [PATCH 172/652] Minor modification --- src/Compression/CompressionFactory.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index d73c94c8cc3..8965b499ea1 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -13,6 +13,7 @@ namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -49,14 +50,16 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std if (level && param) { auto level_literal = std::make_shared(static_cast(*level)); - auto param_literal = std::make_shared(static_cast(*param)); + auto param_literal = std::make_shared(*param); return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), level_literal, param_literal)), {}); } else if (param) { - auto param_literal = std::make_shared(static_cast(*param)); + auto param_literal = std::make_shared(*param); return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), param_literal)), {}); - } else { + } + else + { return get(family_name, level); } } From 40d4f0678f0ea24ff97b10bb090b4122aa507198 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 May 2021 04:25:06 +0300 Subject: [PATCH 173/652] Remove overload (harmful) --- programs/compressor/Compressor.cpp | 2 +- src/Client/Connection.cpp | 2 +- src/Compression/CompressionFactory.cpp | 24 +++++++------------ src/Compression/CompressionFactory.h | 2 -- src/DataStreams/TemporaryFileStream.h | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/CompressionCodecSelector.h | 2 +- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 ++-- .../MergeTree/MergeTreeWriteAheadLog.cpp | 7 +++++- 12 files changed, 26 insertions(+), 31 deletions(-) diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 9f99f8eb95e..520fff199f0 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -144,7 +144,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) codec = CompressionCodecFactory::instance().get(ast, nullptr); } else - codec = CompressionCodecFactory::instance().get(method_family, level); + codec = CompressionCodecFactory::instance().get(method_family, level, {}); std::unique_ptr rb; diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 87f768d7e75..4346510269c 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -425,7 +425,7 @@ void Connection::sendQuery( level = settings->network_zstd_compression_level; CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs); - compression_codec = CompressionCodecFactory::instance().get(method, level); + compression_codec = CompressionCodecFactory::instance().get(method, level, {}); } else compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 8965b499ea1..1ce8c82fdd0 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -31,20 +31,6 @@ CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const } -CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level) const -{ - if (level) - { - auto literal = std::make_shared(static_cast(*level)); - return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}); - } - else - { - auto identifier = std::make_shared(Poco::toUpper(family_name)); - return get(makeASTFunction("CODEC", identifier), {}); - } -} - CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level, std::optional param) const { if (level && param) @@ -58,9 +44,15 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std auto param_literal = std::make_shared(*param); return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), param_literal)), {}); } + else if (level) + { + auto level_literal = std::make_shared(static_cast(*level)); + return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), level_literal)), {}); + } else { - return get(family_name, level); + auto identifier = std::make_shared(Poco::toUpper(family_name)); + return get(makeASTFunction("CODEC", identifier), {}); } } @@ -372,7 +364,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecDensity(*this); registerCodecsLZSSE(*this); - default_codec = get("LZ4", {}); + default_codec = get("LZ4", {}, {}); } CompressionCodecFactory & CompressionCodecFactory::instance() diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 34dd8633611..a616b06c285 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -70,8 +70,6 @@ public: CompressionCodecPtr get(const uint8_t byte_code) const; /// For backward compatibility with config settings - CompressionCodecPtr get(const String & family_name, std::optional level) const; - CompressionCodecPtr get(const String & family_name, std::optional level, std::optional param) const; /// Register codec with parameters and column type diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index ce9071801d0..ac366106897 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -36,7 +36,7 @@ struct TemporaryFileStream std::atomic * is_cancelled, const std::string & codec) { WriteBufferFromFile file_buf(path); - CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {})); + CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {}, {})); NativeBlockOutputStream output(compressed_buf, 0, header); copyData(input, output, is_cancelled); compressed_buf.finalize(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e147083d3b7..4baa3c455d2 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1401,7 +1401,7 @@ void TCPHandler::initBlockOutput(const Block & block) CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs); state.maybe_compressed_out = std::make_shared( - *out, CompressionCodecFactory::instance().get(method, level)); + *out, CompressionCodecFactory::instance().get(method, level, {})); } else state.maybe_compressed_out = out; diff --git a/src/Storages/CompressionCodecSelector.h b/src/Storages/CompressionCodecSelector.h index 746b3ce37ee..76c88546547 100644 --- a/src/Storages/CompressionCodecSelector.h +++ b/src/Storages/CompressionCodecSelector.h @@ -92,7 +92,7 @@ public: for (const auto & element : elements) if (element.check(part_size, part_size_ratio)) - res = factory.get(element.family_name, element.level); + res = factory.get(element.family_name, element.level, {}); return res; } diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index b91f5b10cf2..ed40290c0d3 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -633,7 +633,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: compression_level = settings.network_zstd_compression_level; CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs); - CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); + CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level, {}); /// tmp directory is used to ensure atomicity of transactions /// and keep monitor thread out from reading incomplete data diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index d5f4d91850c..f290d6780d5 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -600,7 +600,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( metadata_snapshot->projections.get(projection_name).metadata, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {})); + CompressionCodecFactory::instance().get("NONE", {}, {})); part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_projection_part); @@ -622,7 +622,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->partition.create(metadata_snapshot, block, 0); MergedBlockOutputStream part_out( - new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); + new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}, {})); part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index dc68a1dcb41..9fac317bfbd 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -691,7 +691,7 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() /// In memory parts doesn't have any compression if (!isStoredOnDisk()) { - default_codec = CompressionCodecFactory::instance().get("NONE", {}); + default_codec = CompressionCodecFactory::instance().get("NONE", {}, {}); return; } @@ -733,7 +733,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { /// In memory parts doesn't have any compression if (!isStoredOnDisk()) - return CompressionCodecFactory::instance().get("NONE", {}); + return CompressionCodecFactory::instance().get("NONE", {}, {}); auto metadata_snapshot = storage.getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 473f2e598a9..555b514bd38 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -26,7 +26,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( const IMergeTreeDataPart * parent_part_) : IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::IN_MEMORY, parent_part_) { - default_codec = CompressionCodecFactory::instance().get("NONE", {}); + default_codec = CompressionCodecFactory::instance().get("NONE", {}, {}); } MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( @@ -38,7 +38,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( const IMergeTreeDataPart * parent_part_) : IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::IN_MEMORY, parent_part_) { - default_codec = CompressionCodecFactory::instance().get("NONE", {}); + default_codec = CompressionCodecFactory::instance().get("NONE", {}, {}); } IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 4c92d4f6136..bd5412b0bdf 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -189,7 +189,12 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor if (action_type == ActionType::ADD_PART) { - MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); + MergedBlockOutputStream part_out( + part, + metadata_snapshot, + block.getNamesAndTypesList(), + {}, + CompressionCodecFactory::instance().get("NONE", {}, {})); part->minmax_idx.update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0); From 408b828777f4bb6f037e7a12338b4da89ef12689 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 00:48:11 +0300 Subject: [PATCH 174/652] Fix typo --- programs/compressor/Compressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 520fff199f0..99ae4b3039f 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -81,7 +81,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") - ("param", po::value(), "extra params for compresion algorithm") + ("param", po::value(), "extra params for compression algorithm") ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") ("level", po::value(), "compression level for codecs specified via flags") ("none", "use no compression instead of LZ4") From 57a3f132d7ad2a764c804b81233454eed57fce9f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 00:48:54 +0300 Subject: [PATCH 175/652] Fix style --- src/Compression/CompressionCodecLZ4.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index 597a6c195eb..5bf6b48258c 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -13,9 +13,6 @@ #include #include -#include - - #pragma GCC diagnostic ignored "-Wold-style-cast" From 8d2fa6cf50cb9dd78e34b248da2bc65c311efca5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 01:18:12 +0300 Subject: [PATCH 176/652] Allow to build without experimental compression libraries and remove from fasttest --- CMakeLists.txt | 4 +++- cmake/find/density.cmake | 9 +++++++++ cmake/find/lizard.cmake | 9 +++++++++ cmake/find/lzsse.cmake | 9 +++++++++ cmake/find/yaml-cpp.cmake | 2 +- contrib/CMakeLists.txt | 15 +++++++++++---- docker/test/fasttest/run.sh | 3 --- src/Common/config.h.in | 5 ++++- src/Compression/CompressionCodecDensity.cpp | 8 ++++++++ src/Compression/CompressionCodecLZSSE.cpp | 8 ++++++++ src/Compression/CompressionCodecLizard.cpp | 8 ++++++++ src/Compression/CompressionFactory.cpp | 19 +++++++++++++++++++ src/Core/config_core.h.in | 3 +++ 13 files changed, 92 insertions(+), 10 deletions(-) create mode 100644 cmake/find/density.cmake create mode 100644 cmake/find/lizard.cmake create mode 100644 cmake/find/lzsse.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 866d9f542e1..cc016d4f2cd 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -528,7 +528,9 @@ include (cmake/find/rocksdb.cmake) include (cmake/find/libpqxx.cmake) include (cmake/find/nuraft.cmake) include (cmake/find/yaml-cpp.cmake) - +include (cmake/find/lizard.cmake) +include (cmake/find/lzsse.cmake) +include (cmake/find/density.cmake) if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") diff --git a/cmake/find/density.cmake b/cmake/find/density.cmake new file mode 100644 index 00000000000..f4963d67709 --- /dev/null +++ b/cmake/find/density.cmake @@ -0,0 +1,9 @@ +option(USE_DENSITY "Enable DENSITY experimental compression library" ${ENABLE_LIBRARIES}) + +if (NOT USE_DENSITY) + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/density/README.md") + message (ERROR "submodule contrib/density is missing. to fix try run: \n git submodule update --init --recursive") +endif() diff --git a/cmake/find/lizard.cmake b/cmake/find/lizard.cmake new file mode 100644 index 00000000000..27c8945c45a --- /dev/null +++ b/cmake/find/lizard.cmake @@ -0,0 +1,9 @@ +option(USE_LIZARD "Enable Lizard experimental compression library" ${ENABLE_LIBRARIES}) + +if (NOT USE_LIZARD) + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/lizard/README.md") + message (ERROR "submodule contrib/lizard is missing. to fix try run: \n git submodule update --init --recursive") +endif() diff --git a/cmake/find/lzsse.cmake b/cmake/find/lzsse.cmake new file mode 100644 index 00000000000..a5f58a255f8 --- /dev/null +++ b/cmake/find/lzsse.cmake @@ -0,0 +1,9 @@ +option(USE_LZSSE "Enable LZSSE experimental compression library" ${ENABLE_LIBRARIES}) + +if (NOT USE_LZSSE) + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/lzsse/README.md") + message (ERROR "submodule contrib/lzsse is missing. to fix try run: \n git submodule update --init --recursive") +endif() diff --git a/cmake/find/yaml-cpp.cmake b/cmake/find/yaml-cpp.cmake index 9b9d9bd39d6..4633d559980 100644 --- a/cmake/find/yaml-cpp.cmake +++ b/cmake/find/yaml-cpp.cmake @@ -4,6 +4,6 @@ if (NOT USE_YAML_CPP) return() endif() -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp") +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp/README.md") message (ERROR "submodule contrib/yaml-cpp is missing. to fix try run: \n git submodule update --init --recursive") endif() diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 67e4a5f1e24..94c8199db0d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -58,14 +58,21 @@ if (USE_INTERNAL_XZ_LIBRARY) add_subdirectory (xz) endif() -add_subdirectory (lizard-cmake) -add_subdirectory (density-cmake) -add_subdirectory (lzsse-cmake) +if (USE_LIZARD) + add_subdirectory (lizard-cmake) +endif () + +if (USE_DENSITY) + add_subdirectory (density-cmake) +endif () + +if (USE_LZSSE) + add_subdirectory (lzsse-cmake) +endif () add_subdirectory (poco-cmake) add_subdirectory (croaring-cmake) - # TODO: refactor the contrib libraries below this comment. if (USE_INTERNAL_ZSTD_LIBRARY) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index d94ed5c876a..3a19a249f8e 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -185,9 +185,6 @@ function clone_submodules contrib/dragonbox contrib/fast_float contrib/NuRaft - contrib/lizard - contrib/density - contrib/lzsse ) git submodule sync diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 6844f0fa9e3..7396e6f6222 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -14,6 +14,9 @@ #cmakedefine01 USE_SENTRY #cmakedefine01 USE_GRPC #cmakedefine01 USE_STATS -#cmakedefine01 CLICKHOUSE_SPLIT_BINARY #cmakedefine01 USE_DATASKETCHES #cmakedefine01 USE_YAML_CPP +#cmakedefine01 USE_LIZARD +#cmakedefine01 USE_DENSITY +#cmakedefine01 USE_LZSSE +#cmakedefine01 CLICKHOUSE_SPLIT_BINARY diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index cdee69005be..033f2627364 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_DENSITY + #include #include #include @@ -118,3 +124,5 @@ void registerCodecDensity(CompressionCodecFactory & factory) } } + +#endif diff --git a/src/Compression/CompressionCodecLZSSE.cpp b/src/Compression/CompressionCodecLZSSE.cpp index 50585766b34..48465c52125 100644 --- a/src/Compression/CompressionCodecLZSSE.cpp +++ b/src/Compression/CompressionCodecLZSSE.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_LZSSE + #include #include #include @@ -150,3 +156,5 @@ void registerCodecsLZSSE(CompressionCodecFactory & factory) } } + +#endif diff --git a/src/Compression/CompressionCodecLizard.cpp b/src/Compression/CompressionCodecLizard.cpp index 2059b12bab1..bc6607c0da9 100644 --- a/src/Compression/CompressionCodecLizard.cpp +++ b/src/Compression/CompressionCodecLizard.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_LIZARD + #include #include #include @@ -111,3 +117,5 @@ void registerCodecLizard(CompressionCodecFactory & factory) } } + +#endif diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 1ce8c82fdd0..db73e6df4df 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -1,3 +1,7 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #include #include #include @@ -345,9 +349,18 @@ void registerCodecT64(CompressionCodecFactory & factory); void registerCodecDoubleDelta(CompressionCodecFactory & factory); void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); + +#if USE_LIZARD void registerCodecLizard(CompressionCodecFactory & factory); +#endif + +#if USE_DENSITY void registerCodecDensity(CompressionCodecFactory & factory); +#endif + +#if USE_LZSSE void registerCodecsLZSSE(CompressionCodecFactory & factory); +#endif CompressionCodecFactory::CompressionCodecFactory() { @@ -360,9 +373,15 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecDoubleDelta(*this); registerCodecGorilla(*this); registerCodecMultiple(*this); +#if USE_LIZARD registerCodecLizard(*this); +#endif +#if USE_DENSITY registerCodecDensity(*this); +#endif +#if USE_LZSSE registerCodecsLZSSE(*this); +#endif default_codec = get("LZ4", {}, {}); } diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index e250e013913..da923ca23a1 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -15,3 +15,6 @@ #cmakedefine01 USE_LIBPQXX #cmakedefine01 USE_NURAFT #cmakedefine01 USE_KRB5 +#cmakedefine01 USE_LIZARD +#cmakedefine01 USE_DENSITY +#cmakedefine01 USE_LZSSE From 99b85bd032e6a64674b88a4b15768d23b82cc8d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 01:34:59 +0300 Subject: [PATCH 177/652] Fix CMake --- src/CMakeLists.txt | 24 ++++++------------------ 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 25775d1f0c1..06aaa781292 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -379,28 +379,16 @@ if (XZ_LIBRARY) target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${XZ_INCLUDE_DIR}) endif() -set (LIZARD_LIBRARY lizard) -set (LIZARD_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/lizard/lib/) -if (LIZARD_LIBRARY) - dbms_target_link_libraries(PRIVATE ${LIZARD_LIBRARY}) - #target_link_libraries (clickhouse_compression PUBLIC ${LZMA_LIBRARY}) - #target_include_directories (clickhouse_compression SYSTEM BEFORE PUBLIC ${}) +if (USE_LIZARD) + dbms_target_link_libraries(PRIVATE lizard) endif() -set (DENSITY_LIBRARY density) -set (DENSITY_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/density/src) -if (DENSITY_LIBRARY) - dbms_target_link_libraries(PRIVATE ${DENSITY_LIBRARY}) - #target_link_libraries (clickhouse_compression PUBLIC ${LZMA_LIBRARY}) - #target_include_directories (clickhouse_compression SYSTEM BEFORE PUBLIC ${}) +if (USE_DENSITY) + dbms_target_link_libraries(PRIVATE density) endif() -set (LZSSE_LIBRARY lzsse) -set (LZSSE_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/lzsse/) -if (LZSSE_LIBRARY) - dbms_target_link_libraries(PRIVATE ${LZSSE_LIBRARY}) - #target_link_libraries (clickhouse_compression PUBLIC ${LZMA_LIBRARY}) - #target_include_directories (clickhouse_compression SYSTEM BEFORE PUBLIC ${}) +if (USE_LZSSE) + dbms_target_link_libraries(PRIVATE lzsse) endif() if (USE_ICU) From 70d9f5c1545041a118a9379fbdcdf7e18d8667ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 02:36:50 +0300 Subject: [PATCH 178/652] Fix error --- src/Compression/CompressionCodecLZSSE.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Compression/CompressionCodecLZSSE.cpp b/src/Compression/CompressionCodecLZSSE.cpp index 48465c52125..436f2dea1f2 100644 --- a/src/Compression/CompressionCodecLZSSE.cpp +++ b/src/Compression/CompressionCodecLZSSE.cpp @@ -92,16 +92,16 @@ UInt32 CompressionCodecLZSSE::doCompressData(const char * source, UInt32 source_ } case 4: { - LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); - res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); - LZSSE2_FreeOptimalParseState(state); + LZSSE4_OptimalParseState * state = LZSSE4_MakeOptimalParseState(source_size); + res = LZSSE4_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE4_FreeOptimalParseState(state); break; } case 8: { - LZSSE2_OptimalParseState * state = LZSSE2_MakeOptimalParseState(source_size); - res = LZSSE2_CompressOptimalParse(state, source, source_size, dest, source_size, level); - LZSSE2_FreeOptimalParseState(state); + LZSSE8_OptimalParseState * state = LZSSE8_MakeOptimalParseState(source_size); + res = LZSSE8_CompressOptimalParse(state, source, source_size, dest, source_size, level); + LZSSE8_FreeOptimalParseState(state); break; } default: From 541224290836c2dc040f633222597f96bd8d40c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 02:38:46 +0300 Subject: [PATCH 179/652] Fix error --- src/Compression/CompressionCodecLZSSE.cpp | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecLZSSE.cpp b/src/Compression/CompressionCodecLZSSE.cpp index 436f2dea1f2..1585f375eef 100644 --- a/src/Compression/CompressionCodecLZSSE.cpp +++ b/src/Compression/CompressionCodecLZSSE.cpp @@ -115,7 +115,27 @@ UInt32 CompressionCodecLZSSE::doCompressData(const char * source, UInt32 source_ void CompressionCodecLZSSE::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { - UInt32 res = LZSSE2_Decompress(source, source_size, dest, uncompressed_size); + UInt32 res = 0; + switch (type) + { + case 2: + { + res = LZSSE2_Decompress(source, source_size, dest, uncompressed_size); + break; + } + case 4: + { + res = LZSSE4_Decompress(source, source_size, dest, uncompressed_size); + break; + } + case 8: + { + res = LZSSE8_Decompress(source, source_size, dest, uncompressed_size); + break; + } + default: + break; + } if (res < uncompressed_size) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress block with LZSSE{}", type); } From ff7b5d34bba959d0dfc3063345949f91917dd8dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 02:56:03 +0300 Subject: [PATCH 180/652] Mark false positives for PVS-Studio --- src/Common/tests/gtest_thread_pool_global_full.cpp | 4 ++-- src/Interpreters/ActionsDAG.cpp | 5 ++--- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Parsers/New/ParserErrorListener.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +++++--- src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 2 +- src/Storages/SelectQueryInfo.h | 2 +- src/Storages/StorageLog.cpp | 2 +- 9 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/Common/tests/gtest_thread_pool_global_full.cpp b/src/Common/tests/gtest_thread_pool_global_full.cpp index 583d43be1bb..c6133f7ca11 100644 --- a/src/Common/tests/gtest_thread_pool_global_full.cpp +++ b/src/Common/tests/gtest_thread_pool_global_full.cpp @@ -25,7 +25,7 @@ TEST(ThreadPool, GlobalFull1) std::atomic counter = 0; static constexpr size_t num_jobs = capacity + 1; - auto func = [&] { ++counter; while (counter != num_jobs) {} }; + auto func = [&] { ++counter; while (counter != num_jobs) {} }; //-V776 ThreadPool pool(num_jobs); @@ -63,7 +63,7 @@ TEST(ThreadPool, GlobalFull2) global_pool.wait(); std::atomic counter = 0; - auto func = [&] { ++counter; while (counter != capacity + 1) {} }; + auto func = [&] { ++counter; while (counter != capacity + 1) {} }; //-V776 ThreadPool pool(capacity, 0, capacity); for (size_t i = 0; i < capacity; ++i) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 21c24956453..d1c10e1c037 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -540,7 +540,7 @@ Block ActionsDAG::updateHeader(Block header) const struct Frame { - const Node * node; + const Node * node = nullptr; size_t next_child = 0; }; @@ -587,8 +587,7 @@ Block ActionsDAG::updateHeader(Block header) const } } - auto & column = node_to_column[output]; - if (column.column) + if (node_to_column[output].column) result_columns.push_back(node_to_column[output]); } } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cca29dd364d..9c463d89a59 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -263,7 +263,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) } /// We use global context here, because storages lifetime is bigger than query context lifetime - database->loadStoredObjects(getContext()->getGlobalContext(), has_force_restore_data_flag, create.attach && force_attach); + database->loadStoredObjects(getContext()->getGlobalContext(), has_force_restore_data_flag, create.attach && force_attach); //-V560 } catch (...) { diff --git a/src/Parsers/New/ParserErrorListener.cpp b/src/Parsers/New/ParserErrorListener.cpp index bc5ee84cba9..f6ac0f0c451 100644 --- a/src/Parsers/New/ParserErrorListener.cpp +++ b/src/Parsers/New/ParserErrorListener.cpp @@ -26,7 +26,7 @@ void ParserErrorListener::syntaxError( auto * parser = dynamic_cast(recognizer); assert(parser); - LOG_ERROR(&Poco::Logger::get("ClickHouseParser"), + LOG_ERROR(&Poco::Logger::get("ClickHouseParser"), //-V522 "Last element parsed so far:\n" "{}\n" "Parser error: (pos {}) {}", parser->getRuleContext()->toStringTree(parser, true), token->getStartIndex(), message); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0f5e69448d2..c31585d1373 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -577,6 +577,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeDataSelectSamplingData sampling = use_cache ? std::move(cache->sampling) : MergeTreeDataSelectSamplingData{}; if (!use_cache) { + assert(key_condition.has_value()); + RelativeSize relative_sample_size = 0; RelativeSize relative_sample_offset = 0; @@ -605,7 +607,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( /// read) into the relative `SAMPLE 0.1` (how much data to read). size_t approx_total_rows = 0; if (relative_sample_size > 1 || relative_sample_offset > 1) - approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, *key_condition, settings); + approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, *key_condition, settings); //-V1007 if (relative_sample_size > 1) { @@ -764,7 +766,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (has_lower_limit) { - if (!key_condition->addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) + if (!key_condition->addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) //-V1007 throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); ASTPtr args = std::make_shared(); @@ -781,7 +783,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (has_upper_limit) { - if (!key_condition->addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) + if (!key_condition->addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) //-V1007 throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); ASTPtr args = std::make_shared(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8335e54213f..70025b49658 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -15,7 +15,7 @@ class KeyCondition; struct MergeTreeDataSelectSamplingData { - bool use_sampling; + bool use_sampling = false; std::shared_ptr filter_function; ActionsDAGPtr filter_expression; }; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 0da2f643eb0..f2357a87535 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -173,7 +173,7 @@ void MergeTreeReaderWide::readData( { auto get_stream_getter = [&](bool stream_for_prefix) -> ISerialization::InputStreamGetter { - return [&, stream_for_prefix](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer * + return [&, stream_for_prefix](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer * //-V1047 { /// If substream have already been read. if (cache.count(ISerialization::getSubcolumnNameForStream(substream_path))) diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index e971e126972..afed41189c2 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -122,7 +122,7 @@ struct MergeTreeDataSelectCache; // The projection selected to execute current query struct ProjectionCandidate { - const ProjectionDescription * desc; + const ProjectionDescription * desc{}; PrewhereInfoPtr prewhere_info; ActionsDAGPtr before_where; String where_column_name; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 663deddfd1f..61fbbbc3086 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -172,7 +172,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu auto create_stream_getter = [&](bool stream_for_prefix) { - return [&, stream_for_prefix] (const ISerialization::SubstreamPath & path) -> ReadBuffer * + return [&, stream_for_prefix] (const ISerialization::SubstreamPath & path) -> ReadBuffer * //-V1047 { if (cache.count(ISerialization::getSubcolumnNameForStream(path))) return nullptr; From 4290b48a6169784a64f80cc63f6c437f547fcabb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:00:57 +0300 Subject: [PATCH 181/652] Fix clang-tidy --- src/Compression/CompressionCodecDelta.cpp | 2 +- src/Compression/CompressionCodecDensity.cpp | 2 +- src/Compression/CompressionCodecDoubleDelta.cpp | 7 ++++++- src/Compression/CompressionCodecGorilla.cpp | 2 +- src/Compression/CompressionCodecLZ4.cpp | 4 ++-- src/Compression/CompressionCodecLZSSE.cpp | 2 +- src/Compression/CompressionCodecLizard.cpp | 2 +- src/Compression/CompressionCodecZSTD.cpp | 3 +-- 8 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 7d0824d4d71..447abe9e840 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -14,7 +14,7 @@ namespace DB class CompressionCodecDelta : public ICompressionCodec { public: - CompressionCodecDelta(UInt8 delta_bytes_size_); + explicit CompressionCodecDelta(UInt8 delta_bytes_size_); uint8_t getMethodByte() const override; diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index 033f2627364..88d5a294c81 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -20,7 +20,7 @@ class CompressionCodecDensity : public ICompressionCodec public: static constexpr auto DENSITY_DEFAULT_ALGO = DENSITY_ALGORITHM_CHAMELEON; // by default aim on speed - CompressionCodecDensity(DENSITY_ALGORITHM algo_); + explicit CompressionCodecDensity(DENSITY_ALGORITHM algo_); uint8_t getMethodByte() const override; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index d992132c4f6..79ced55594a 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -19,6 +19,11 @@ namespace DB { +/** NOTE DoubleDelta is surprisingly bad name. The only excuse is that it comes from an academic paper. + * Most people will think that "double delta" is just applying delta transform twice. + * But in fact it is something more than applying delta transform twice. + */ + /** DoubleDelta column codec implementation. * * Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf, which was extended @@ -108,7 +113,7 @@ namespace DB class CompressionCodecDoubleDelta : public ICompressionCodec { public: - CompressionCodecDoubleDelta(UInt8 data_bytes_size_); + explicit CompressionCodecDoubleDelta(UInt8 data_bytes_size_); uint8_t getMethodByte() const override; diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index db76a854a97..7fcb2183503 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -104,7 +104,7 @@ namespace DB class CompressionCodecGorilla : public ICompressionCodec { public: - CompressionCodecGorilla(UInt8 data_bytes_size_); + explicit CompressionCodecGorilla(UInt8 data_bytes_size_); uint8_t getMethodByte() const override; diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index 5bf6b48258c..8cb81e460b1 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -22,7 +22,7 @@ namespace DB class CompressionCodecLZ4 : public ICompressionCodec { public: - CompressionCodecLZ4(); + explicit CompressionCodecLZ4(); uint8_t getMethodByte() const override; @@ -49,7 +49,7 @@ private: class CompressionCodecLZ4HC : public CompressionCodecLZ4 { public: - CompressionCodecLZ4HC(int level_); + explicit CompressionCodecLZ4HC(int level_); protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecLZSSE.cpp b/src/Compression/CompressionCodecLZSSE.cpp index 1585f375eef..62fb641bc12 100644 --- a/src/Compression/CompressionCodecLZSSE.cpp +++ b/src/Compression/CompressionCodecLZSSE.cpp @@ -20,7 +20,7 @@ namespace DB class CompressionCodecLZSSE : public ICompressionCodec { public: - CompressionCodecLZSSE(UInt32 type_, UInt32 level_); + explicit CompressionCodecLZSSE(UInt32 type_, UInt32 level_); uint8_t getMethodByte() const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionCodecLizard.cpp b/src/Compression/CompressionCodecLizard.cpp index bc6607c0da9..cd2d49a0538 100644 --- a/src/Compression/CompressionCodecLizard.cpp +++ b/src/Compression/CompressionCodecLizard.cpp @@ -19,7 +19,7 @@ class CompressionCodecLizard : public ICompressionCodec public: static constexpr auto LIZARD_DEFAULT_LEVEL = 1; - CompressionCodecLizard(int level_); + explicit CompressionCodecLizard(int level_); uint8_t getMethodByte() const override; diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 4d3b5a181d1..a950f1fee74 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -20,8 +20,7 @@ public: static constexpr auto ZSTD_DEFAULT_LEVEL = 1; static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24; - CompressionCodecZSTD(int level_); - + explicit CompressionCodecZSTD(int level_); CompressionCodecZSTD(int level_, int window_log); uint8_t getMethodByte() const override; From ad9481b1bbb94ee41c84532ae1ec1d1250fc3ea9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:04:13 +0300 Subject: [PATCH 182/652] Disable LZSSE on AArch64 --- cmake/find/lzsse.cmake | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/cmake/find/lzsse.cmake b/cmake/find/lzsse.cmake index a5f58a255f8..763483d8ea9 100644 --- a/cmake/find/lzsse.cmake +++ b/cmake/find/lzsse.cmake @@ -1,4 +1,9 @@ -option(USE_LZSSE "Enable LZSSE experimental compression library" ${ENABLE_LIBRARIES}) +set (DEFAULT_USE_LZSSE 0) +if (ENABLE_LIBRARIES AND ARCH_AMD64) + set (DEFAULT_USE_LZSSE 1) +endif() + +option(USE_LZSSE "Enable LZSSE experimental compression library" ${DEFAULT_USE_LZSSE}) if (NOT USE_LZSSE) return() From e206959390fbd6080bf1a06e8dbb76c30d05fa88 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:18:57 +0300 Subject: [PATCH 183/652] Use entropy coders from zstd --- contrib/lizard-cmake/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/lizard-cmake/CMakeLists.txt b/contrib/lizard-cmake/CMakeLists.txt index ad5f5bf132d..7fe5274ecb6 100644 --- a/contrib/lizard-cmake/CMakeLists.txt +++ b/contrib/lizard-cmake/CMakeLists.txt @@ -14,3 +14,6 @@ SET (Headers ADD_LIBRARY(lizard ${Sources} ${Headers}) target_include_directories (lizard PUBLIC ${LIBRARY_DIR}) + +# It is using some symbols (HUF_* and FSE_*) from zstd. +target_link_libraries (lizard PRIVATE zstd) From 273226de32b6880d403012935e0e8a16d7ba542e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:43:25 +0300 Subject: [PATCH 184/652] Remove string parameter for Density --- programs/compressor/Compressor.cpp | 2 +- src/Client/Connection.cpp | 2 +- src/Compression/CompressionCodecDensity.cpp | 24 +++++++------------ src/Compression/CompressionFactory.cpp | 17 +++---------- src/Compression/CompressionFactory.h | 2 +- src/DataStreams/TemporaryFileStream.h | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/CompressionCodecSelector.h | 2 +- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 ++-- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../test_non_default_compression/test.py | 2 +- 14 files changed, 27 insertions(+), 44 deletions(-) diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 99ae4b3039f..35534eaee9e 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -144,7 +144,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) codec = CompressionCodecFactory::instance().get(ast, nullptr); } else - codec = CompressionCodecFactory::instance().get(method_family, level, {}); + codec = CompressionCodecFactory::instance().get(method_family, level); std::unique_ptr rb; diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 4346510269c..87f768d7e75 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -425,7 +425,7 @@ void Connection::sendQuery( level = settings->network_zstd_compression_level; CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs); - compression_codec = CompressionCodecFactory::instance().get(method, level, {}); + compression_codec = CompressionCodecFactory::instance().get(method, level); } else compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); diff --git a/src/Compression/CompressionCodecDensity.cpp b/src/Compression/CompressionCodecDensity.cpp index 88d5a294c81..e462ff2765b 100644 --- a/src/Compression/CompressionCodecDensity.cpp +++ b/src/Compression/CompressionCodecDensity.cpp @@ -18,8 +18,6 @@ namespace DB class CompressionCodecDensity : public ICompressionCodec { public: - static constexpr auto DENSITY_DEFAULT_ALGO = DENSITY_ALGORITHM_CHAMELEON; // by default aim on speed - explicit CompressionCodecDensity(DENSITY_ALGORITHM algo_); uint8_t getMethodByte() const override; @@ -93,7 +91,7 @@ void registerCodecDensity(CompressionCodecFactory & factory) method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr { - DENSITY_ALGORITHM algo = CompressionCodecDensity::DENSITY_DEFAULT_ALGO; + DENSITY_ALGORITHM algorithm = DENSITY_ALGORITHM_CHAMELEON; if (arguments && !arguments->children.empty()) { @@ -104,22 +102,18 @@ void registerCodecDensity(CompressionCodecFactory & factory) const auto children = arguments->children; const auto * algo_literal = children[0]->as(); - if (!algo_literal || algo_literal->value.getType() != Field::Types::String) - throw Exception("Density codec argument must be string (algorithm), one of: 'lion', 'chameleon', 'cheetah'", + if (!algo_literal || algo_literal->value.getType() != Field::Types::UInt64) + throw Exception("Density codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); - const auto algorithm = algo_literal->value.safeGet(); - if (algorithm == "lion") - algo = DENSITY_ALGORITHM_LION; - else if (algorithm == "chameleon") - algo = DENSITY_ALGORITHM_CHAMELEON; - else if (algorithm == "cheetah") - algo = DENSITY_ALGORITHM_CHEETAH; - else - throw Exception("Density codec argument may be one of: 'lion', 'chameleon', 'cheetah'", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + const UInt64 algorithm_num = algo_literal->value.safeGet(); + if (algorithm_num < 1 || algorithm_num > 3) + throw Exception("Density codec level can be 1, 2 or 3.", ErrorCodes::ILLEGAL_CODEC_PARAMETER); + + algorithm = static_cast(algorithm_num); } - return std::make_shared(algo); + return std::make_shared(algorithm); }); } diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index db73e6df4df..fbd369fdfe7 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -35,20 +35,9 @@ CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const } -CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level, std::optional param) const +CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level) const { - if (level && param) - { - auto level_literal = std::make_shared(static_cast(*level)); - auto param_literal = std::make_shared(*param); - return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), level_literal, param_literal)), {}); - } - else if (param) - { - auto param_literal = std::make_shared(*param); - return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), param_literal)), {}); - } - else if (level) + if (level) { auto level_literal = std::make_shared(static_cast(*level)); return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), level_literal)), {}); @@ -383,7 +372,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecsLZSSE(*this); #endif - default_codec = get("LZ4", {}, {}); + default_codec = get("LZ4", {}); } CompressionCodecFactory & CompressionCodecFactory::instance() diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index a616b06c285..cdbb663935a 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -70,7 +70,7 @@ public: CompressionCodecPtr get(const uint8_t byte_code) const; /// For backward compatibility with config settings - CompressionCodecPtr get(const String & family_name, std::optional level, std::optional param) const; + CompressionCodecPtr get(const String & family_name, std::optional level) const; /// Register codec with parameters and column type void registerCompressionCodecWithType(const String & family_name, std::optional byte_code, CreatorWithType creator); diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index ac366106897..ce9071801d0 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -36,7 +36,7 @@ struct TemporaryFileStream std::atomic * is_cancelled, const std::string & codec) { WriteBufferFromFile file_buf(path); - CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {}, {})); + CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {})); NativeBlockOutputStream output(compressed_buf, 0, header); copyData(input, output, is_cancelled); compressed_buf.finalize(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4baa3c455d2..e147083d3b7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1401,7 +1401,7 @@ void TCPHandler::initBlockOutput(const Block & block) CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs); state.maybe_compressed_out = std::make_shared( - *out, CompressionCodecFactory::instance().get(method, level, {})); + *out, CompressionCodecFactory::instance().get(method, level)); } else state.maybe_compressed_out = out; diff --git a/src/Storages/CompressionCodecSelector.h b/src/Storages/CompressionCodecSelector.h index 76c88546547..746b3ce37ee 100644 --- a/src/Storages/CompressionCodecSelector.h +++ b/src/Storages/CompressionCodecSelector.h @@ -92,7 +92,7 @@ public: for (const auto & element : elements) if (element.check(part_size, part_size_ratio)) - res = factory.get(element.family_name, element.level, {}); + res = factory.get(element.family_name, element.level); return res; } diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index ed40290c0d3..b91f5b10cf2 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -633,7 +633,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: compression_level = settings.network_zstd_compression_level; CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs); - CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level, {}); + CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions /// and keep monitor thread out from reading incomplete data diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index f290d6780d5..d5f4d91850c 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -600,7 +600,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( metadata_snapshot->projections.get(projection_name).metadata, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {}, {})); + CompressionCodecFactory::instance().get("NONE", {})); part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_projection_part); @@ -622,7 +622,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->partition.create(metadata_snapshot, block, 0); MergedBlockOutputStream part_out( - new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}, {})); + new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 9fac317bfbd..dc68a1dcb41 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -691,7 +691,7 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() /// In memory parts doesn't have any compression if (!isStoredOnDisk()) { - default_codec = CompressionCodecFactory::instance().get("NONE", {}, {}); + default_codec = CompressionCodecFactory::instance().get("NONE", {}); return; } @@ -733,7 +733,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { /// In memory parts doesn't have any compression if (!isStoredOnDisk()) - return CompressionCodecFactory::instance().get("NONE", {}, {}); + return CompressionCodecFactory::instance().get("NONE", {}); auto metadata_snapshot = storage.getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 555b514bd38..473f2e598a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -26,7 +26,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( const IMergeTreeDataPart * parent_part_) : IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::IN_MEMORY, parent_part_) { - default_codec = CompressionCodecFactory::instance().get("NONE", {}, {}); + default_codec = CompressionCodecFactory::instance().get("NONE", {}); } MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( @@ -38,7 +38,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( const IMergeTreeDataPart * parent_part_) : IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::IN_MEMORY, parent_part_) { - default_codec = CompressionCodecFactory::instance().get("NONE", {}, {}); + default_codec = CompressionCodecFactory::instance().get("NONE", {}); } IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index bd5412b0bdf..c8ae051b041 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -194,7 +194,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor metadata_snapshot, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {}, {})); + CompressionCodecFactory::instance().get("NONE", {})); part->minmax_idx.update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0); diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index 9689b4110a2..54e14132d3b 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -143,7 +143,7 @@ def test_experimental_codecs(start_cluster): node6.query(""" CREATE TABLE compression_experimental_codecs ( somedate Date CODEC(Lizard(12)), - id UInt64 CODEC(Density('lion')), + id UInt64 CODEC(Density(3)), data String CODEC(LZSSE4(3)) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; """) From 54f0591aa5b400073b10325f774d97373c7c7909 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:47:00 +0300 Subject: [PATCH 185/652] Allow to ATTACH a table with experimental codecs --- src/Interpreters/InterpreterCreateQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9c463d89a59..8bc74e5b82d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -447,7 +447,8 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context_); bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs; - bool allow_experimental_codecs = context_->getSettingsRef().allow_experimental_codecs; + bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs; + ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); for (auto ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it) From 890b6e5e47fb5dbb12b163c58a37d292da1899cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:54:42 +0300 Subject: [PATCH 186/652] Add a test --- .../00096_experimental_codecs.reference | 8 +++ .../1_stateful/00096_experimental_codecs.sql | 61 +++++++++++++++++++ 2 files changed, 69 insertions(+) create mode 100644 tests/queries/1_stateful/00096_experimental_codecs.reference create mode 100644 tests/queries/1_stateful/00096_experimental_codecs.sql diff --git a/tests/queries/1_stateful/00096_experimental_codecs.reference b/tests/queries/1_stateful/00096_experimental_codecs.reference new file mode 100644 index 00000000000..a34182c23e5 --- /dev/null +++ b/tests/queries/1_stateful/00096_experimental_codecs.reference @@ -0,0 +1,8 @@ +17415138241754778329 +17415138241754778329 +17415138241754778329 +17415138241754778329 +17415138241754778329 +3798944011853532000 +3798944011853532000 +3798944011853532000 diff --git a/tests/queries/1_stateful/00096_experimental_codecs.sql b/tests/queries/1_stateful/00096_experimental_codecs.sql new file mode 100644 index 00000000000..e085e55f886 --- /dev/null +++ b/tests/queries/1_stateful/00096_experimental_codecs.sql @@ -0,0 +1,61 @@ +DROP TABLE IF EXISTS hits_experimental; + +-- It's not allowed to create a table with experimental codecs unless the user turns off the safety switch. +CREATE TABLE hits_experimental (Title String CODEC(Lizard(10))) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 } + +SET allow_experimental_codecs = 1; + +-- Lizard + +CREATE TABLE hits_experimental (Title String CODEC(Lizard(10))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits; +SELECT sum(cityHash64(*)) FROM hits_experimental; + +-- It's always allowed to attach a table with experimental codecs. +DETACH TABLE hits_experimental; +SET allow_experimental_codecs = 0; +ATTACH TABLE hits_experimental; +SELECT sum(cityHash64(*)) FROM hits_experimental; +SET allow_experimental_codecs = 1; + +DROP TABLE hits_experimental; + +-- Density + +-- Check out of range levels of Density. +CREATE TABLE hits_experimental (Title String CODEC(Density(-1))) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 433 } +CREATE TABLE hits_experimental (Title String CODEC(Density(0))) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 433 } +CREATE TABLE hits_experimental (Title String CODEC(Density(4))) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 433 } +CREATE TABLE hits_experimental (Title String CODEC(Density('hello'))) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 433 } + +CREATE TABLE hits_experimental (Title String CODEC(Density(1))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits; +SELECT sum(cityHash64(*)) FROM hits_experimental; +DROP TABLE hits_experimental; + +CREATE TABLE hits_experimental (Title String CODEC(Density(2))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits; +SELECT sum(cityHash64(*)) FROM hits_experimental; +DROP TABLE hits_experimental; + +CREATE TABLE hits_experimental (Title String CODEC(Density(3))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits; +SELECT sum(cityHash64(*)) FROM hits_experimental; +DROP TABLE hits_experimental; + +-- LZSSE + +CREATE TABLE hits_experimental (Title String CODEC(LZSSE2)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits ORDER BY Title LIMIT 100000; +SELECT sum(cityHash64(*)) FROM hits_experimental; +DROP TABLE hits_experimental; + +CREATE TABLE hits_experimental (Title String CODEC(LZSSE4)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits ORDER BY Title LIMIT 100000; +SELECT sum(cityHash64(*)) FROM hits_experimental; +DROP TABLE hits_experimental; + +CREATE TABLE hits_experimental (Title String CODEC(LZSSE8)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO hits_experimental SELECT Title FROM test.hits ORDER BY Title LIMIT 100000; +SELECT sum(cityHash64(*)) FROM hits_experimental; +DROP TABLE hits_experimental; From 0ab67f74915421b5e72c8ed1c8614dcf9ebe741f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 06:57:02 +0300 Subject: [PATCH 187/652] Fix YAML lint --- programs/server/config-example.yaml | 101 ++++++++++++++-------------- 1 file changed, 51 insertions(+), 50 deletions(-) diff --git a/programs/server/config-example.yaml b/programs/server/config-example.yaml index af62455ccef..3e275adf2fa 100644 --- a/programs/server/config-example.yaml +++ b/programs/server/config-example.yaml @@ -391,56 +391,57 @@ user_directories: local_directory: # Path to folder where users created by SQL commands are stored. path: /var/lib/clickhouse/access/ - # To add an LDAP server as a remote user directory of users that are not defined locally, define a single 'ldap' section - # with the following parameters: - # server - one of LDAP server names defined in 'ldap_servers' config section above. - # This parameter is mandatory and cannot be empty. - # roles - section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. - # If no roles are specified here or assigned during role mapping (below), user will not be able to perform any - # actions after authentication. - # role_mapping - section with LDAP search parameters and mapping rules. - # When a user authenticates, while still bound to LDAP, an LDAP search is performed using search_filter and the - # name of the logged in user. For each entry found during that search, the value of the specified attribute is - # extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the - # value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by - # CREATE ROLE command. - # There can be multiple 'role_mapping' sections defined inside the same 'ldap' section. All of them will be - # applied. - # base_dn - template used to construct the base DN for the LDAP search. - # The resulting DN will be constructed by replacing all '{user_name}', '{bind_dn}', and '{user_dn}' - # substrings of the template with the actual user name, bind DN, and user DN during each LDAP search. - # scope - scope of the LDAP search. - # Accepted values are: 'base', 'one_level', 'children', 'subtree' (the default). - # search_filter - template used to construct the search filter for the LDAP search. - # The resulting filter will be constructed by replacing all '{user_name}', '{bind_dn}', '{user_dn}', and - # '{base_dn}' substrings of the template with the actual user name, bind DN, user DN, and base DN during - # each LDAP search. - # Note, that the special characters must be escaped properly in XML. - # attribute - attribute name whose values will be returned by the LDAP search. 'cn', by default. - # prefix - prefix, that will be expected to be in front of each string in the original list of strings returned by - # the LDAP search. Prefix will be removed from the original strings and resulting strings will be treated - # as local role names. Empty, by default. - # Example: - # ldap: - # server: my_ldap_server - # roles: - # my_local_role1: '' - # my_local_role2: '' - # role_mapping: - # base_dn: 'ou=groups,dc=example,dc=com' - # scope: subtree - # search_filter: '(&(objectClass=groupOfNames)(member={bind_dn}))' - # attribute: cn - # prefix: clickhouse_ - # Example (typical Active Directory with role mapping that relies on the detected user DN): - # ldap: - # server: my_ad_server - # role_mapping: - # base_dn: 'CN=Users,DC=example,DC=com' - # attribute: CN - # scope: subtree - # search_filter: '(&(objectClass=group)(member={user_dn}))' - # prefix: clickhouse_ + +# # To add an LDAP server as a remote user directory of users that are not defined locally, define a single 'ldap' section +# # with the following parameters: +# # server - one of LDAP server names defined in 'ldap_servers' config section above. +# # This parameter is mandatory and cannot be empty. +# # roles - section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. +# # If no roles are specified here or assigned during role mapping (below), user will not be able to perform any +# # actions after authentication. +# # role_mapping - section with LDAP search parameters and mapping rules. +# # When a user authenticates, while still bound to LDAP, an LDAP search is performed using search_filter and the +# # name of the logged in user. For each entry found during that search, the value of the specified attribute is +# # extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the +# # value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by +# # CREATE ROLE command. +# # There can be multiple 'role_mapping' sections defined inside the same 'ldap' section. All of them will be +# # applied. +# # base_dn - template used to construct the base DN for the LDAP search. +# # The resulting DN will be constructed by replacing all '{user_name}', '{bind_dn}', and '{user_dn}' +# # substrings of the template with the actual user name, bind DN, and user DN during each LDAP search. +# # scope - scope of the LDAP search. +# # Accepted values are: 'base', 'one_level', 'children', 'subtree' (the default). +# # search_filter - template used to construct the search filter for the LDAP search. +# # The resulting filter will be constructed by replacing all '{user_name}', '{bind_dn}', '{user_dn}', and +# # '{base_dn}' substrings of the template with the actual user name, bind DN, user DN, and base DN during +# # each LDAP search. +# # Note, that the special characters must be escaped properly in XML. +# # attribute - attribute name whose values will be returned by the LDAP search. 'cn', by default. +# # prefix - prefix, that will be expected to be in front of each string in the original list of strings returned by +# # the LDAP search. Prefix will be removed from the original strings and resulting strings will be treated +# # as local role names. Empty, by default. +# # Example: +# # ldap: +# # server: my_ldap_server +# # roles: +# # my_local_role1: '' +# # my_local_role2: '' +# # role_mapping: +# # base_dn: 'ou=groups,dc=example,dc=com' +# # scope: subtree +# # search_filter: '(&(objectClass=groupOfNames)(member={bind_dn}))' +# # attribute: cn +# # prefix: clickhouse_ +# # Example (typical Active Directory with role mapping that relies on the detected user DN): +# # ldap: +# # server: my_ad_server +# # role_mapping: +# # base_dn: 'CN=Users,DC=example,DC=com' +# # attribute: CN +# # scope: subtree +# # search_filter: '(&(objectClass=group)(member={user_dn}))' +# # prefix: clickhouse_ # Default profile of settings. default_profile: default From 8939daf7e7d538dce0fcce11d5ca4c8565b450da Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 24 May 2021 11:23:04 +0300 Subject: [PATCH 188/652] more --- tests/integration/helpers/cluster.py | 25 +++++++++++++------- tests/integration/runner | 9 ++++++- tests/integration/test_grpc_protocol/test.py | 2 +- 3 files changed, 26 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6e188d298e3..ab8c33e4b53 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1042,7 +1042,7 @@ class ClickHouseCluster: while time.time() - start < timeout: try: for instance in ['zoo1', 'zoo2', 'zoo3']: - conn = self.get_kazoo_client(instance, self.zookeeper_secure_port, use_ssl=True) + conn = self.get_kazoo_client(instance) conn.get_children('/') logging.debug("All instances of ZooKeeper Secure started") return @@ -1058,7 +1058,7 @@ class ClickHouseCluster: while time.time() - start < timeout: try: for instance in ['zoo1', 'zoo2', 'zoo3']: - conn = self.get_kazoo_client(instance, self.zookeeper_port) + conn = self.get_kazoo_client(instance) conn.get_children('/') logging.debug("All instances of ZooKeeper started") return @@ -1248,7 +1248,7 @@ class ClickHouseCluster: self.wait_zookeeper_secure_to_start() for command in self.pre_zookeeper_commands: - self.run_kazoo_commands_with_retries(command, repeats=5, secure=True) + self.run_kazoo_commands_with_retries(command, repeats=5) if self.with_zookeeper and self.base_zookeeper_cmd: logging.debug('Setup ZooKeeper') @@ -1490,7 +1490,16 @@ class ClickHouseCluster: def open_bash_shell(self, instance_name): os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash'])) - def get_kazoo_client(self, zoo_instance_name, port, use_ssl=False): + def get_kazoo_client(self, zoo_instance_name): + use_ssl = False + if self.with_zookeeper_secure: + port = self.zookeeper_secure_port + use_ssl = True + elif self.with_zookeeper: + port = self.zookeeper_port + else: + raise Exception("Cluster has no ZooKeeper") + ip = self.get_instance_ip(zoo_instance_name) logging.debug(f"get_kazoo_client: {zoo_instance_name}, ip:{ip}, port:{port}, use_ssl:{use_ssl}") zk = KazooClient(hosts=f"{ip}:{port}", use_ssl=use_ssl, verify_certs=False, certfile=self.zookeeper_certfile, @@ -1498,16 +1507,16 @@ class ClickHouseCluster: zk.start() return zk - def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1, secure=False): - logging.debug(f"run_kazoo_commands_with_retries: {zoo_instance_name}, {secure}, {kazoo_callback}") + def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1): + logging.debug(f"run_kazoo_commands_with_retries: {zoo_instance_name}, {kazoo_callback}") for i in range(repeats - 1): try: - kazoo_callback(self.get_kazoo_client(zoo_instance_name, self.zookeeper_secure_port if secure else self.zookeeper_port)) + kazoo_callback(self.get_kazoo_client(zoo_instance_name)) return except KazooException as e: logging.debug(repr(e)) time.sleep(sleep_for) - kazoo_callback(self.get_kazoo_client(zoo_instance_name, self.zookeeper_secure_port if secure else self.zookeeper_port)) + kazoo_callback(self.get_kazoo_client(zoo_instance_name)) def add_zookeeper_startup_command(self, command): self.pre_zookeeper_commands.append(command) diff --git a/tests/integration/runner b/tests/integration/runner index 9947fa33f11..711d0f4873e 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -147,6 +147,11 @@ if __name__ == "__main__": default=False, help="Don't use net host in parent docker container") + parser.add_argument( + "--network", + default='host', + help="Set network driver for runnner container") + parser.add_argument( "--docker-image-version", default="latest", @@ -196,7 +201,9 @@ if __name__ == "__main__": parallel_args += " -n {}".format(args.parallel) net = "" - if not args.disable_net_host: + if args.network: + net = "--net={}".format(args.network) + elif not args.disable_net_host: net = "--net=host" env_tags = "" diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index d8604276281..ee7e94bad1d 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -133,7 +133,7 @@ def start_cluster(): @pytest.fixture(autouse=True) def reset_after_test(): yield - query("DROP TABLE IF EXISTS t") + node.query_with_retry("DROP TABLE IF EXISTS t") # Actual tests From 7bc13f7ae8cb06dbfbc83d129047cbfff249fe42 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 20 May 2021 21:21:42 +0800 Subject: [PATCH 189/652] Add initial_query_start_time to query log --- src/Core/Defines.h | 8 +++--- src/Interpreters/ClientInfo.cpp | 12 +++++++++ src/Interpreters/ClientInfo.h | 2 ++ src/Interpreters/QueryLog.cpp | 4 +++ src/Interpreters/QueryThreadLog.cpp | 2 ++ src/Interpreters/executeQuery.cpp | 11 +++++++- src/Server/TCPHandler.cpp | 5 ++-- .../01872_initial_query_start_time.reference | 1 + .../01872_initial_query_start_time.sh | 27 +++++++++++++++++++ 9 files changed, 66 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01872_initial_query_start_time.reference create mode 100755 tests/queries/0_stateless/01872_initial_query_start_time.sh diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 94df16758bf..994478cd8a1 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -83,10 +83,12 @@ #define DBMS_MIN_REVISION_WITH_X_FORWARDED_FOR_IN_CLIENT_INFO 54443 #define DBMS_MIN_REVISION_WITH_REFERER_IN_CLIENT_INFO 54447 -/// Version of ClickHouse TCP protocol. Increment it manually when you change the protocol. -#define DBMS_TCP_PROTOCOL_VERSION 54448 - #define DBMS_MIN_PROTOCOL_VERSION_WITH_DISTRIBUTED_DEPTH 54448 + +/// Version of ClickHouse TCP protocol. Increment it manually when you change the protocol. +#define DBMS_TCP_PROTOCOL_VERSION 54449 + +#define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 /// The boundary on which the blocks for asynchronous file operations should be aligned. #define DEFAULT_AIO_FILE_BLOCK_SIZE 4096 diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 248c8cc6b80..ec694952b07 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -34,6 +34,12 @@ void ClientInfo::write(WriteBuffer & out, const UInt64 server_protocol_revision) writeBinary(initial_query_id, out); writeBinary(initial_address.toString(), out); + if (server_protocol_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME) + { + writeBinary(initial_query_start_time, out); + writeBinary(initial_query_start_time_microseconds, out); + } + writeBinary(UInt8(interface), out); if (interface == Interface::TCP) @@ -109,6 +115,12 @@ void ClientInfo::read(ReadBuffer & in, const UInt64 client_protocol_revision) readBinary(initial_address_string, in); initial_address = Poco::Net::SocketAddress(initial_address_string); + if (client_protocol_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME) + { + readBinary(initial_query_start_time, in); + readBinary(initial_query_start_time_microseconds, in); + } + UInt8 read_interface = 0; readBinary(read_interface, in); interface = Interface(read_interface); diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index b4579877ef6..d6158a2d7d5 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -61,6 +61,8 @@ public: String initial_user; String initial_query_id; Poco::Net::SocketAddress initial_address; + time_t initial_query_start_time{}; + Decimal64 initial_query_start_time_microseconds{}; // OpenTelemetry trace context we received from client, or which we are going // to send to server. diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 1242af48676..f78acdd3737 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -78,6 +78,8 @@ Block QueryLogElement::createBlock() {std::make_shared(), "initial_query_id"}, {DataTypeFactory::instance().get("IPv6"), "initial_address"}, {std::make_shared(), "initial_port"}, + {std::make_shared(), "initial_query_start_time"}, + {std::make_shared(6), "initial_query_start_time_microseconds"}, {std::make_shared(), "interface"}, {std::make_shared(), "os_user"}, {std::make_shared(), "client_hostname"}, @@ -256,6 +258,8 @@ void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableCo columns[i++]->insert(client_info.initial_query_id); columns[i++]->insertData(IPv6ToBinary(client_info.initial_address.host()).data(), 16); columns[i++]->insert(client_info.initial_address.port()); + columns[i++]->insert(client_info.initial_query_start_time); + columns[i++]->insert(client_info.initial_query_start_time_microseconds); columns[i++]->insert(UInt64(client_info.interface)); diff --git a/src/Interpreters/QueryThreadLog.cpp b/src/Interpreters/QueryThreadLog.cpp index 31f1fddc87f..df591b42de2 100644 --- a/src/Interpreters/QueryThreadLog.cpp +++ b/src/Interpreters/QueryThreadLog.cpp @@ -51,6 +51,8 @@ Block QueryThreadLogElement::createBlock() {std::make_shared(), "initial_query_id"}, {DataTypeFactory::instance().get("IPv6"), "initial_address"}, {std::make_shared(), "initial_port"}, + {std::make_shared(), "initial_query_start_time"}, + {std::make_shared(6), "initial_query_start_time_microseconds"}, {std::make_shared(), "interface"}, {std::make_shared(), "os_user"}, {std::make_shared(), "client_hostname"}, diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 542ed7ca0f9..843f16f9d35 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -355,6 +355,15 @@ static std::tuple executeQueryImpl( { const auto current_time = std::chrono::system_clock::now(); + auto & client_info = context->getClientInfo(); + + // If it's an initial query, set to current_time + if (client_info.initial_query_start_time == 0) + { + client_info.initial_query_start_time = time_in_seconds(current_time); + client_info.initial_query_start_time_microseconds = time_in_microseconds(current_time); + } + #if !defined(ARCADIA_BUILD) assert(internal || CurrentThread::get().getQueryContext()); assert(internal || CurrentThread::get().getQueryContext()->getCurrentQueryId() == CurrentThread::getQueryId()); @@ -643,7 +652,7 @@ static std::tuple executeQueryImpl( elem.query = query_for_logging; elem.normalized_query_hash = normalizedQueryHash(query_for_logging); - elem.client_info = context->getClientInfo(); + elem.client_info = client_info; bool log_queries = settings.log_queries && !internal; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9ef0292ee06..201ee4e2664 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1135,8 +1135,9 @@ void TCPHandler::receiveQuery() /// Per query settings are also passed via TCP. /// We need to check them before applying due to they can violate the settings constraints. - auto settings_format = (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS - : SettingsWriteFormat::BINARY; + auto settings_format = (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) + ? SettingsWriteFormat::STRINGS_WITH_FLAGS + : SettingsWriteFormat::BINARY; Settings passed_settings; passed_settings.read(*in, settings_format); diff --git a/tests/queries/0_stateless/01872_initial_query_start_time.reference b/tests/queries/0_stateless/01872_initial_query_start_time.reference new file mode 100644 index 00000000000..9972842f982 --- /dev/null +++ b/tests/queries/0_stateless/01872_initial_query_start_time.reference @@ -0,0 +1 @@ +1 1 diff --git a/tests/queries/0_stateless/01872_initial_query_start_time.sh b/tests/queries/0_stateless/01872_initial_query_start_time.sh new file mode 100755 index 00000000000..dbfb89a05a1 --- /dev/null +++ b/tests/queries/0_stateless/01872_initial_query_start_time.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +set -ue + +# this test doesn't need 'current_database = currentDatabase()', + +unset CLICKHOUSE_LOG_COMMENT + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "drop table if exists m" +${CLICKHOUSE_CLIENT} -q "create table m (dummy UInt8) ENGINE = Distributed('test_cluster_two_shards', 'system', 'one')" + +query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") +${CLICKHOUSE_CLIENT} -q "select * from m format Null" "--query_id=$query_id" + +${CLICKHOUSE_CLIENT} -n -q " +system flush logs; +select + anyIf(initial_query_start_time, is_initial_query) = anyIf(initial_query_start_time, not is_initial_query), + anyIf(initial_query_start_time_microseconds, is_initial_query) = anyIf(initial_query_start_time_microseconds, not is_initial_query) +from system.query_log +where initial_query_id = '$query_id' and type = 'QueryFinish'; +" + +${CLICKHOUSE_CLIENT} -q "drop table m" From d1b9dcdcec489ed4e4973ffbf883976258d549b7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 May 2021 15:57:03 +0300 Subject: [PATCH 190/652] fix build --- src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp | 2 +- src/Interpreters/RewriteFunctionToSubcolumnVisitor.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp index 2086d2dc229..b0647baaa7d 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp @@ -52,7 +52,7 @@ const std::unordered_mapchildren; if (arguments.size() != 1) diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h index cfd87d8449e..e7b8385f710 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h @@ -13,7 +13,7 @@ class RewriteFunctionToSubcolumnData { public: using TypeToVisit = ASTFunction; - void visit(ASTFunction & function, ASTPtr & ast); + void visit(ASTFunction & function, ASTPtr & ast) const; StorageMetadataPtr metadata_snapshot; }; From a4d1b9b07d7fb6c8b1340e5ef63b0531d963dc5a Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Mon, 24 May 2021 16:55:05 +0300 Subject: [PATCH 191/652] Fix --- src/Storages/StorageBuffer.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 6688112c9f1..e5af154d7bd 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -841,7 +841,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc size_t block_rows = block_to_write.rows(); size_t block_bytes = block_to_write.bytes(); - size_t block_allocated_bytes = block_to_write.allocatedBytes(); + size_t block_allocated_bytes_delta = block_to_write.allocatedBytes() - buffer.data.allocatedBytes(); CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_rows); CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_bytes); @@ -851,7 +851,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc if (!destination_id) { total_writes.rows -= block_rows; - total_writes.bytes -= block_allocated_bytes; + total_writes.bytes -= block_allocated_bytes_delta; LOG_DEBUG(log, "Flushing buffer with {} rows (discarded), {} bytes, age {} seconds {}.", rows, bytes, time_passed, (check_thresholds ? "(bg)" : "(direct)")); return; @@ -890,7 +890,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc } total_writes.rows -= block_rows; - total_writes.bytes -= block_allocated_bytes; + total_writes.bytes -= block_allocated_bytes_delta; UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Flushing buffer with {} rows, {} bytes, age {} seconds, took {} ms {}.", rows, bytes, time_passed, milliseconds, (check_thresholds ? "(bg)" : "(direct)")); From c191a631e86e29b4e274229c8d977b1cea225281 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 24 May 2021 18:30:51 +0300 Subject: [PATCH 192/652] touch --- tests/integration/test_attach_without_fetching/test.py | 2 +- .../configs/dictionaries/ssd_complex_key_cache_string.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 1a7a2a1074b..c48c73f4d70 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -56,7 +56,7 @@ def check_data(nodes, detached_parts): for other in nodes: if other != node: - print("> Checking data consistency,", other.name, "vs", node.name) + logging.debug(f"> Checking data consistency, {other.name} vs {node.name}") assert_eq_with_retry(other, "SELECT * FROM test ORDER BY n", res) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml index 729dc11d0a1..7492a4331cb 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml @@ -41,7 +41,7 @@ 16777216 131072 1048576 - /etc/clickhouse-server/dictionaries/radars + /etc/clickhouse-server/radars 1 From b37f9c20e2feb82cef4754b81210f68cebef05ea Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 24 May 2021 20:07:43 +0300 Subject: [PATCH 193/652] Update dictionary.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил пример. --- docs/en/sql-reference/table-functions/dictionary.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 608dba01bea..3e3d2a10984 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -36,8 +36,7 @@ Create a dictionary: ``` sql CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) -LAYOUT(DIRECT()); +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) LAYOUT(DIRECT()); ``` Query: From 8c57f0d50db1a980ad2f4cb937a1c09f3cfd25de Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 24 May 2021 20:24:40 +0300 Subject: [PATCH 194/652] Edit and translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../external-authenticators/ldap.md | 1 - .../external-authenticators/ldap.md | 46 ++++++++++++++++--- docs/ru/sql-reference/statements/grant.md | 3 +- 3 files changed, 41 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/external-authenticators/ldap.md b/docs/en/operations/external-authenticators/ldap.md index 805d45e1b38..83bc575802f 100644 --- a/docs/en/operations/external-authenticators/ldap.md +++ b/docs/en/operations/external-authenticators/ldap.md @@ -108,7 +108,6 @@ Note, that user `my_user` refers to `my_ldap_server`. This LDAP server must be c When SQL-driven [Access Control and Account Management](../access-rights.md#access-control) is enabled, users that are authenticated by LDAP servers can also be created using the [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement) statement. - Query: ```sql diff --git a/docs/ru/operations/external-authenticators/ldap.md b/docs/ru/operations/external-authenticators/ldap.md index 312020000ea..dc8fe103595 100644 --- a/docs/ru/operations/external-authenticators/ldap.md +++ b/docs/ru/operations/external-authenticators/ldap.md @@ -1,4 +1,4 @@ -# LDAP {#external-authenticators-ldap} +# LDAP {#external-authenticators-ldap} Для аутентификации пользователей ClickHouse можно использовать сервер LDAP. Существуют два подхода: @@ -17,6 +17,7 @@ + localhost 636 @@ -31,6 +32,18 @@ /path/to/tls_ca_cert_dir ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384 + + + + localhost + 389 + EXAMPLE\{user_name} + + CN=Users,DC=example,DC=com + (&(objectClass=user)(sAMAccountName={user_name})) + + no + ``` @@ -43,6 +56,15 @@ - `port` — порт сервера LDAP. Если настройка `enable_tls` равна `true`, то по умолчанию используется порт `636`, иначе — порт `389`. - `bind_dn` — шаблон для создания DN для привязки. - При формировании DN все подстроки `{user_name}` в шаблоне будут заменяться на фактическое имя пользователя при каждой попытке аутентификации. +- `user_dn_detection` — секция с параметрами LDAP поиска для определения фактического значения DN привязанного пользователя. + - Это в основном используется в фильтрах поиска для дальнейшего сопоставления ролей, когда сервер является Active Directory. Полученный DN пользователя будет использоваться при замене подстрок `{user_dn}` везде, где они разрешены. По умолчанию DN пользователя устанавливается равным DN привязки, но после выполнения поиска он будет обновлен до фактического обнаруженного значения DN пользователя. + - `base_dn` — шаблон, который используется для создания базового DN для LDAP поиска. + - При формировании DN все подстроки `{user_name}` и `{bind_dn}` в шаблоне будут заменяться на фактическое имя пользователя и DN привязки соответственно при каждом LDAP поиске. + - `scope` — область LDAP поиска. + - Возможные значения: `base`, `one_level`, `children`, `subtree` (по умолчанию). + - `search_filter` — шаблон, который используется для создания фильтра для каждого LDAP поиска. + - При формировании фильтра все подстроки `{user_name}`, `{bind_dn}`, `{user_dn}` и `{base_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN привязки, DN пользователя и базовый DN соответственно при каждом LDAP поиске. + - Обратите внимание, что специальные символы должны быть правильно экранированы в XML. - `verification_cooldown` — промежуток времени (в секундах) после успешной попытки привязки, в течение которого пользователь будет считаться аутентифицированным и сможет выполнять запросы без повторного обращения к серверам LDAP. - Чтобы отключить кеширование и заставить обращаться к серверу LDAP для каждого запроса аутентификации, укажите `0` (значение по умолчанию). - `enable_tls` — флаг, включающий использование защищенного соединения с сервером LDAP. @@ -106,7 +128,7 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; - + my_ldap_server @@ -121,6 +143,18 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; clickhouse_ + + + + my_ad_server + + CN=Users,DC=example,DC=com + CN + subtree + (&(objectClass=group)(member={user_dn})) + clickhouse_ + + ``` @@ -136,13 +170,13 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; - При аутентификации пользователя, пока еще связанного с LDAP, производится LDAP поиск с помощью `search_filter` и имени этого пользователя. Для каждой записи, найденной в ходе поиска, выделяется значение указанного атрибута. У каждого атрибута, имеющего указанный префикс, этот префикс удаляется, а остальная часть значения становится именем локальной роли, определенной в ClickHouse, причем предполагается, что эта роль была ранее создана запросом [CREATE ROLE](../../sql-reference/statements/create/role.md#create-role-statement) до этого. - Внутри одной секции `ldap` может быть несколько секций `role_mapping`. Все они будут применены. - `base_dn` — шаблон, который используется для создания базового DN для LDAP поиска. - - При формировании DN все подстроки `{user_name}` и `{bind_dn}` в шаблоне будут заменяться на фактическое имя пользователя и DN привязки соответственно при каждом LDAP поиске. - - `scope` — Область LDAP поиска. + - При формировании DN все подстроки `{user_name}`, `{bind_dn}` и `{user_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN привязки и DN пользователя соответственно при каждом LDAP поиске. + - `scope` — область LDAP поиска. - Возможные значения: `base`, `one_level`, `children`, `subtree` (по умолчанию). - `search_filter` — шаблон, который используется для создания фильтра для каждого LDAP поиска. - - при формировании фильтра все подстроки `{user_name}`, `{bind_dn}` и `{base_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN привязки и базовый DN соответственно при каждом LDAP поиске. + - При формировании фильтра все подстроки `{user_name}`, `{bind_dn}`, `{user_dn}` и `{base_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN привязки, DN пользователя и базовый DN соответственно при каждом LDAP поиске. - Обратите внимание, что специальные символы должны быть правильно экранированы в XML. - - `attribute` — имя атрибута, значение которого будет возвращаться LDAP поиском. + - `attribute` — имя атрибута, значение которого будет возвращаться LDAP поиском. По умолчанию: `cn`. - `prefix` — префикс, который, как предполагается, будет находиться перед началом каждой строки в исходном списке строк, возвращаемых LDAP поиском. Префикс будет удален из исходных строк, а сами они будут рассматриваться как имена локальных ролей. По умолчанию: пустая строка. [Оригинальная статья](https://clickhouse.tech/docs/en/operations/external-authenticators/ldap) diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index 093e6eb3b93..05ffaa22bbd 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -319,13 +319,12 @@ GRANT INSERT(x,y) ON db.table TO john Разрешает выполнять запросы [DROP](misc.md#drop) и [DETACH](misc.md#detach-statement) в соответствии со следующей иерархией привилегий: -- `DROP`. Уровень: +- `DROP`. Уровень: `GROUP` - `DROP DATABASE`. Уровень: `DATABASE` - `DROP TABLE`. Уровень: `TABLE` - `DROP VIEW`. Уровень: `VIEW` - `DROP DICTIONARY`. Уровень: `DICTIONARY` - ### TRUNCATE {#grant-truncate} Разрешает выполнять запросы [TRUNCATE](../../sql-reference/statements/truncate.md). From 6a434c0ce89438c1f7009c17940d742fc35647d6 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Tue, 25 May 2021 12:19:03 +0300 Subject: [PATCH 195/652] Started to implement fuzzing for YAMLParser --- src/Parsers/examples/YAML_fuzzer.cpp | 26 ++++ tests/fuzz/YAML_fuzzer.dict | 198 +++++++++++++++++++++++++++ tests/fuzz/YAML_fuzzer.options | 2 + 3 files changed, 226 insertions(+) create mode 100644 src/Parsers/examples/YAML_fuzzer.cpp create mode 100644 tests/fuzz/YAML_fuzzer.dict create mode 100644 tests/fuzz/YAML_fuzzer.options diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Parsers/examples/YAML_fuzzer.cpp new file mode 100644 index 00000000000..1abd9adfcb7 --- /dev/null +++ b/src/Parsers/examples/YAML_fuzzer.cpp @@ -0,0 +1,26 @@ +#include +#include +#include + +#include +#include +#include +#include + + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +{ + std::string input = std::string(reinterpret_cast(data), size); + DB::YAMLParser parser; + + std::ofstream temp_file("YAML_fuzzer_data.yaml"); + temp_file << input; + temp_file.close(); + + YAMLParser::parse("YAML_fuzzer_data.yaml"); + + remove("YAML_fuzzer_data.yaml"); + + return 0; +} + diff --git a/tests/fuzz/YAML_fuzzer.dict b/tests/fuzz/YAML_fuzzer.dict new file mode 100644 index 00000000000..1e34e62b620 --- /dev/null +++ b/tests/fuzz/YAML_fuzzer.dict @@ -0,0 +1,198 @@ +"key1" +"key2" +"key3" +"key4" +"key5" +"key6" +"key7" +"key8" +"key9" +"key10" +"key11" +"key12" +"key13" +"key14" +"key15" +"key16" +"key17" +"key18" +"key19" +"key20" +"key21" +"key22" +"key23" +"key24" +"key25" +"key26" +"key27" +"key28" +"key29" +"key30" +"key31" +"key32" +"key33" +"key34" +"key35" +"key36" +"key37" +"key38" +"key39" +"key40" +"key41" +"key42" +"key43" +"key44" +"key45" +"key46" +"key47" +"key48" +"key49" +"key50" +"key51" +"key52" +"key53" +"key54" +"key55" +"key56" +"key57" +"key58" +"key59" +"key60" +"key61" +"key62" +"key63" +"key64" +"key65" +"key66" +"key67" +"key68" +"key69" +"key70" +"key71" +"key72" +"key73" +"key74" +"key75" +"key76" +"key77" +"key78" +"key79" +"key80" +"key81" +"key82" +"key83" +"key84" +"key85" +"key86" +"key87" +"key88" +"key89" +"key90" +"key91" +"key92" +"key93" +"key94" +"key95" +"key96" +"key97" +"key98" +"key99" +"val1" +"val2" +"val3" +"val4" +"val5" +"val6" +"val7" +"val8" +"val9" +"val10" +"val11" +"val12" +"val13" +"val14" +"val15" +"val16" +"val17" +"val18" +"val19" +"val20" +"val21" +"val22" +"val23" +"val24" +"val25" +"val26" +"val27" +"val28" +"val29" +"val30" +"val31" +"val32" +"val33" +"val34" +"val35" +"val36" +"val37" +"val38" +"val39" +"val40" +"val41" +"val42" +"val43" +"val44" +"val45" +"val46" +"val47" +"val48" +"val49" +"val50" +"val51" +"val52" +"val53" +"val54" +"val55" +"val56" +"val57" +"val58" +"val59" +"val60" +"val61" +"val62" +"val63" +"val64" +"val65" +"val66" +"val67" +"val68" +"val69" +"val70" +"val71" +"val72" +"val73" +"val74" +"val75" +"val76" +"val77" +"val78" +"val79" +"val80" +"val81" +"val82" +"val83" +"val84" +"val85" +"val86" +"val87" +"val88" +"val89" +"val90" +"val91" +"val92" +"val93" +"val94" +"val95" +"val96" +"val97" +"val98" +"val99" diff --git a/tests/fuzz/YAML_fuzzer.options b/tests/fuzz/YAML_fuzzer.options new file mode 100644 index 00000000000..1a0d84a96c4 --- /dev/null +++ b/tests/fuzz/YAML_fuzzer.options @@ -0,0 +1,2 @@ +[libfuzzer] +dict = YAML_fuzzer.dict From 10372b12b799fa142c903d89906a98bf92fecfab Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 15:29:22 +0300 Subject: [PATCH 196/652] rm containers in entrypooint --- docker/test/integration/runner/dockerd-entrypoint.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 2321c938990..6ef077585ba 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -27,6 +27,10 @@ while true; do done set -e +# cleanup for retry run if volume is not recreated +docker kill `docker ps -aq` || true +docker rm `docker ps -aq` || true + echo "Start tests" export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse From 893cd47bd27dec0d8db7caf80f74b8db92a4e5b1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 15:40:59 +0300 Subject: [PATCH 197/652] better --- tests/integration/test_attach_without_fetching/test.py | 7 ++++--- tests/integration/test_storage_kafka/test.py | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index c48c73f4d70..954f19ea39e 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -92,11 +92,12 @@ def test_attach_without_fetching(start_cluster): # 3. Break the part data on the second node to corrupt the checksums. # Replica 3 should download the data from replica 1 as there is no local data. # Replica 2 should also download the data from 1 as the checksums won't match. - print("Checking attach with corrupted part data with files missing") + logging.debug("Checking attach with corrupted part data with files missing") - print("Before deleting:", node_2.exec_in_container(['bash', '-c', + to_delete = node_2.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin'.format( - p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True)) + p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True) + logging.debug("Before deleting:", to_delete) node_2.exec_in_container(['bash', '-c', 'cd {p} && rm -fr *.bin'.format( diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 75445e40017..a29a4dcb2f9 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2334,7 +2334,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): assert TSV(result) == TSV('{0}\t{0}\t{0}'.format(i[0] - 1)), 'Missing data!' -@pytest.mark.timeout(120) +@pytest.mark.timeout(300) def test_bad_reschedule(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) From e9ccf906c4e8f94ea3bd0e8245386d2ca626ae16 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 16:40:22 +0300 Subject: [PATCH 198/652] improvements --- tests/integration/test_alter_codec/test.py | 3 --- tests/integration/test_attach_without_fetching/test.py | 1 + .../integration/test_storage_kerberized_hdfs/secrets/krb.conf | 2 +- tests/integration/test_storage_kerberized_hdfs/test.py | 4 ++-- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py index f51dc9a54ff..2117893af5b 100644 --- a/tests/integration/test_alter_codec/test.py +++ b/tests/integration/test_alter_codec/test.py @@ -7,9 +7,6 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/logs_config.xml']) -node2 = cluster.add_instance('node2', - main_configs=['configs/logs_config.xml']) - @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 954f19ea39e..c714d09cf58 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -1,5 +1,6 @@ import time import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry diff --git a/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf b/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf index 2c1e6f15f77..90ad6b61c93 100644 --- a/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf +++ b/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf @@ -7,7 +7,7 @@ default_realm = TEST.CLICKHOUSE.TECH dns_lookup_realm = false dns_lookup_kdc = false - ticket_lifetime = 15s + ticket_lifetime = 5s forwardable = true default_tgs_enctypes = des3-hmac-sha1 default_tkt_enctypes = des3-hmac-sha1 diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index fa520752d1c..115c7005a84 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -53,7 +53,7 @@ def test_write_storage_not_expired(started_cluster): node1.query("create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')") - time.sleep(45) # wait for ticket expiration + time.sleep(15) # wait for ticket expiration node1.query("insert into SimpleHDFSStorageNotExpired values (1, 'Mark', 72.53)") api_read = hdfs_api.read_data("/simple_storage_not_expired") @@ -83,7 +83,7 @@ def test_read_table_expired(started_cluster): hdfs_api.write_data("/simple_table_function_relogin", data) started_cluster.pause_container('hdfskerberos') - time.sleep(45) + time.sleep(15) try: select_read = node1.query("select * from hdfs('hdfs://reloginuser&kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") From a667a1c032c54394985dff14dfd802353e75ecca Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 18:32:24 +0300 Subject: [PATCH 199/652] better --- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/pytest.ini | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ab8c33e4b53..3f0a995c173 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -851,7 +851,7 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - node.wait_for_start(start_timeout=20.0, connection_timeout=600.0) # seconds + node.wait_for_start(start_timeout=180.0, connection_timeout=600.0) # seconds return node def restart_service(self, service_name): @@ -1399,7 +1399,7 @@ class ClickHouseCluster: run_and_check(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_timeout = 60.0 # seconds + start_timeout = 180.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index 47086d45b3d..a714188d08a 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -6,8 +6,8 @@ junit_duration_report = call junit_suite_name = integration log_cli = 1 log_cli_level = CRITICAL -log_cli_format = %%(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) +log_cli_format = %%(asctime)s [%(levelname)8s] %(funcName)s %(message)s (%(filename)s:%(lineno)s) log_file = pytest.log log_file_level = DEBUG -log_file_format = %(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) +log_file_format = %(asctime)s [%(levelname)8s] %(funcName)s %(message)s (%(filename)s:%(lineno)s) log_file_date_format=%Y-%m-%d %H:%M:%S From 34eaa48294ee9e92e405b3136235b7444eef74ca Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 May 2021 19:34:43 +0300 Subject: [PATCH 200/652] Part 1. --- .../QueryPlan/ReadFromMergeTree.cpp | 192 +++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 25 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 280 +++++++++++++++++- .../MergeTree/MergeTreeDataSelectExecutor.h | 27 +- 4 files changed, 498 insertions(+), 26 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d7a78e6cc6b..771f95cc8fd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -2,38 +2,54 @@ #include #include #include +#include #include #include #include +#include +#include #include #include namespace DB { +namespace ErrorCodes +{ + extern const int INDEX_NOT_USED; +} + ReadFromMergeTree::ReadFromMergeTree( + SelectQueryInfo query_info_, + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + ContextPtr context_, + const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, String query_id_, - Names required_columns_, - RangesInDataParts parts_, - IndexStatPtr index_stats_, + Names real_column_names_, + MergeTreeData::DataPartsVector parts_, + //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, size_t num_streams_, ReadType read_type_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( - metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), + metadata_snapshot_->getSampleBlockForColumns(real_column_names_, storage_.getVirtuals(), storage_.getStorageID()), prewhere_info_, storage_.getPartitionValueType(), virt_column_names_)}) + , query_info(std::move(query_info_)) + , max_block_numbers_to_read(max_block_numbers_to_read_) + , context(std::move(context_)) + , data(data_) , storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) , query_id(std::move(query_id_)) - , required_columns(std::move(required_columns_)) + , real_column_names(std::move(real_column_names_)) , parts(std::move(parts_)) - , index_stats(std::move(index_stats_)) + //, index_stats(std::move(index_stats_)) , prewhere_info(std::move(prewhere_info_)) , virt_column_names(std::move(virt_column_names_)) , settings(std::move(settings_)) @@ -140,8 +156,172 @@ Pipe ReadFromMergeTree::read() return pipe; } +static std::optional> filterPartsByVirtualColumns( + const MergeTreeData & data, + MergeTreeData::DataPartsVector & parts, + ASTPtr & query, + ContextPtr context) +{ + std::unordered_set part_values; + ASTPtr expression_ast; + auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); + + // Generate valid expressions for filtering + VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); + + // If there is still something left, fill the virtual block and do the filtering. + if (expression_ast) + { + virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); + VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); + return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + } + + return {}; +} + +static void filterPartsByPartition( + StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & data, + SelectQueryInfo & query_info, + ContextPtr & context, + ContextPtr & query_context, + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read, + Poco::Logger * log, + ReadFromMergeTree::IndexStats & index_stats) +{ + const Settings & settings = context->getSettingsRef(); + std::optional partition_pruner; + std::optional minmax_idx_condition; + DataTypes minmax_columns_types; + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + minmax_idx_condition.emplace( + query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); + partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); + + if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) + { + String msg = "Neither MinMax index by columns ("; + bool first = true; + for (const String & col : minmax_columns_names) + { + if (first) + first = false; + else + msg += ", "; + msg += col; + } + msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; + + throw Exception(msg, ErrorCodes::INDEX_NOT_USED); + } + } + + MergeTreeDataSelectExecutor::PartFilterCounters part_filter_counters; + if (query_context->getSettingsRef().allow_experimental_query_deduplication) + MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( + parts, + part_values, + data.getPinnedPartUUIDs(), + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + query_context, + part_filter_counters, + log); + else + MergeTreeDataSelectExecutor::selectPartsToRead( + parts, + part_values, + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + part_filter_counters); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::None, + .num_parts_after = part_filter_counters.num_initial_selected_parts, + .num_granules_after = part_filter_counters.num_initial_selected_granules}); + + if (minmax_idx_condition) + { + auto description = minmax_idx_condition->getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::MinMax, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_minmax, + .num_granules_after = part_filter_counters.num_granules_after_minmax}); + LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); + } + + if (partition_pruner) + { + auto description = partition_pruner->getKeyCondition().getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Partition, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, + .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); + } +} + void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { + auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); + if (part_values && part_values->empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + /// If there are only virtual columns in the query, you must request at least one non-virtual one. + if (real_column_names.empty()) + { + NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); + real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + } + + metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); + + // Build and check if primary key is used when necessary + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + Names primary_key_columns = primary_key.column_names; + KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); + + if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) + { + throw Exception( + ErrorCodes::INDEX_NOT_USED, + "Primary key ({}) is not used and setting 'force_primary_key' is set.", + fmt::join(primary_key_columns, ", ")); + } + LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); + + const auto & select = query_info.query->as(); + auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; + + filterPartsByPartition( + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); + + bool sample_factor_column_queried = false; + for (const auto & col : virt_column_names) + if (col == "_sample_factor") + sample_factor_column_queried = true; + + auto sampling = MergeTreeDataSelectExecutor::getSampling( + select, parts, metadata_snapshot, key_condition, settings, data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context) + Pipe pipe = read(); for (const auto & processor : pipe.getProcessors()) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 479610b3edc..762d7fea0e7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -36,7 +37,6 @@ public: }; using IndexStats = std::vector; - using IndexStatPtr = std::unique_ptr; /// Part of settings which are needed for reading. struct Settings @@ -46,6 +46,7 @@ public: size_t preferred_max_column_in_block_size_bytes; size_t min_marks_for_concurrent_read; bool use_uncompressed_cache; + bool force_primary_key; MergeTreeReaderSettings reader_settings; MergeTreeReadPool::BackoffSettings backoff_settings; @@ -67,12 +68,16 @@ public: }; ReadFromMergeTree( + SelectQueryInfo query_info_, + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + ContextPtr context_, + const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, String query_id_, - Names required_columns_, - RangesInDataParts parts_, - IndexStatPtr index_stats_, + Names real_column_names_, + MergeTreeData::DataPartsVector parts_, + //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, @@ -91,13 +96,17 @@ public: void describeIndexes(JSONBuilder::JSONMap & map) const override; private: + SelectQueryInfo query_info; + const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read; + ContextPtr context; + const MergeTreeData & data; const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; String query_id; - Names required_columns; - RangesInDataParts parts; - IndexStatPtr index_stats; + Names real_column_names; + MergeTreeData::DataPartsVector parts; + IndexStat index_stats; PrewhereInfoPtr prewhere_info; Names virt_column_names; Settings settings; @@ -105,6 +114,8 @@ private: size_t num_streams; ReadType read_type; + Poco::Logger * log; + Pipe read(); Pipe readFromPool(); Pipe readInOrder(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 821423ca5a5..444d074a4db 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -86,7 +86,8 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, - const Settings & settings) const + const Settings & settings, + Poco::Logger * log) { size_t rows_count = 0; @@ -369,6 +370,269 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( return plan; } +MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( + const ASTSelectQuery & select, + MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata_snapshot, + KeyCondition & key_condition, + const MergeTreeData & data, + Poco::Logger * log, + bool sample_factor_column_queried, + NamesAndTypesList available_real_columns, + ContextPtr context) +{ + const Settings & settings = context->getSettingsRef(); + Float64 used_sample_factor = 1; + /// Sampling. + MergeTreeDataSelectSamplingData sampling; + + RelativeSize relative_sample_size = 0; + RelativeSize relative_sample_offset = 0; + + auto select_sample_size = select.sampleSize(); + auto select_sample_offset = select.sampleOffset(); + + if (select_sample_size) + { + relative_sample_size.assign( + select_sample_size->as().ratio.numerator, + select_sample_size->as().ratio.denominator); + + if (relative_sample_size < 0) + throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + relative_sample_offset = 0; + if (select_sample_offset) + relative_sample_offset.assign( + select_sample_offset->as().ratio.numerator, + select_sample_offset->as().ratio.denominator); + + if (relative_sample_offset < 0) + throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + /// Convert absolute value of the sampling (in form `SAMPLE 1000000` - how many rows to + /// read) into the relative `SAMPLE 0.1` (how much data to read). + size_t approx_total_rows = 0; + if (relative_sample_size > 1 || relative_sample_offset > 1) + approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, key_condition, settings, log); + + if (relative_sample_size > 1) + { + relative_sample_size = convertAbsoluteSampleSizeToRelative(select_sample_size, approx_total_rows); + LOG_DEBUG(log, "Selected relative sample size: {}", toString(relative_sample_size)); + } + + /// SAMPLE 1 is the same as the absence of SAMPLE. + if (relative_sample_size == RelativeSize(1)) + relative_sample_size = 0; + + if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size) + throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (relative_sample_offset > 1) + { + relative_sample_offset = convertAbsoluteSampleSizeToRelative(select_sample_offset, approx_total_rows); + LOG_DEBUG(log, "Selected relative sample offset: {}", toString(relative_sample_offset)); + } + } + + /** Which range of sampling key values do I need to read? + * First, in the whole range ("universe") we select the interval + * of relative `relative_sample_size` size, offset from the beginning by `relative_sample_offset`. + * + * Example: SAMPLE 0.4 OFFSET 0.3 + * + * [------********------] + * ^ - offset + * <------> - size + * + * If the interval passes through the end of the universe, then cut its right side. + * + * Example: SAMPLE 0.4 OFFSET 0.8 + * + * [----------------****] + * ^ - offset + * <------> - size + * + * Next, if the `parallel_replicas_count`, `parallel_replica_offset` settings are set, + * then it is necessary to break the received interval into pieces of the number `parallel_replicas_count`, + * and select a piece with the number `parallel_replica_offset` (from zero). + * + * Example: SAMPLE 0.4 OFFSET 0.3, parallel_replicas_count = 2, parallel_replica_offset = 1 + * + * [----------****------] + * ^ - offset + * <------> - size + * <--><--> - pieces for different `parallel_replica_offset`, select the second one. + * + * It is very important that the intervals for different `parallel_replica_offset` cover the entire range without gaps and overlaps. + * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. + */ + + /// Parallel replicas has been requested but there is no way to sample data. + /// Select all data from first replica and no data from other replicas. + if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) + { + LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" + " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); + sampling.read_nothing = true; + return sampling; + } + + sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); + bool no_data = false; /// There is nothing left after sampling. + + if (sampling.use_sampling) + { + if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) + used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); + + RelativeSize size_of_universum = 0; + const auto & sampling_key = metadata_snapshot->getSamplingKey(); + DataTypePtr sampling_column_type = sampling_key.data_types[0]; + + if (sampling_key.data_types.size() == 1) + { + if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(sampling_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + } + + if (size_of_universum == RelativeSize(0)) + throw Exception( + "Invalid sampling column type in storage parameters: " + sampling_column_type->getName() + + ". Must be one unsigned integer type", + ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); + + if (settings.parallel_replicas_count > 1) + { + if (relative_sample_size == RelativeSize(0)) + relative_sample_size = 1; + + relative_sample_size /= settings.parallel_replicas_count.value; + relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.value); + } + + if (relative_sample_offset >= RelativeSize(1)) + no_data = true; + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_sample_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_sample_offset + relative_sample_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + /*std::cerr << std::fixed << std::setprecision(100) + << "relative_sample_size: " << relative_sample_size << "\n" + << "relative_sample_offset: " << relative_sample_offset << "\n" + << "lower_limit_float: " << lower_limit_rational << "\n" + << "upper_limit_float: " << upper_limit_rational << "\n" + << "lower: " << lower << "\n" + << "upper: " << upper << "\n";*/ + + if ((has_upper_limit && upper == 0) + || (has_lower_limit && has_upper_limit && lower == upper)) + no_data = true; + + if (no_data || (!has_lower_limit && !has_upper_limit)) + { + sampling.use_sampling = false; + } + else + { + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + /// If sample and final are used together no need to calculate sampling expression twice. + /// The first time it was calculated for final, because sample key is a part of the PK. + /// So, assume that we already have calculated column. + ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); + + if (select.final()) + { + sampling_key_ast = std::make_shared(sampling_key.column_names[0]); + /// We do spoil available_real_columns here, but it is not used later. + available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type)); + } + + if (has_lower_limit) + { + if (!key_condition.addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) + throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); + + ASTPtr args = std::make_shared(); + args->children.push_back(sampling_key_ast); + args->children.push_back(std::make_shared(lower)); + + lower_function = std::make_shared(); + lower_function->name = "greaterOrEquals"; + lower_function->arguments = args; + lower_function->children.push_back(lower_function->arguments); + + sampling.filter_function = lower_function; + } + + if (has_upper_limit) + { + if (!key_condition.addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) + throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); + + ASTPtr args = std::make_shared(); + args->children.push_back(sampling_key_ast); + args->children.push_back(std::make_shared(upper)); + + upper_function = std::make_shared(); + upper_function->name = "less"; + upper_function->arguments = args; + upper_function->children.push_back(upper_function->arguments); + + sampling.filter_function = upper_function; + } + + if (has_lower_limit && has_upper_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(lower_function); + args->children.push_back(upper_function); + + sampling.filter_function = std::make_shared(); + sampling.filter_function->name = "and"; + sampling.filter_function->arguments = args; + sampling.filter_function->children.push_back(sampling.filter_function->arguments); + } + + ASTPtr query = sampling.filter_function; + auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); + sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); + } + } + + if (no_data) + { + LOG_DEBUG(log, "Sampling yields no data."); + sampling.read_nothing = true; + } + + return sampling; +} + QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, @@ -534,7 +798,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( partition_pruner, max_block_numbers_to_read, query_context, - part_filter_counters); + part_filter_counters, + log); else selectPartsToRead( parts, @@ -2192,7 +2457,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, @@ -2204,7 +2469,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( for (const auto & part_or_projection : prev_parts) { const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); - if (!part_values.empty() && part_values.find(part->name) == part_values.end()) + if (part_values && part_values->find(part->name) == part_values->end()) continue; if (part->isEmpty()) @@ -2246,14 +2511,15 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context, - PartFilterCounters & counters) const + PartFilterCounters & counters, + Poco::Logger * log) { const Settings & settings = query_context->getSettings(); @@ -2269,7 +2535,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( for (const auto & part_or_projection : prev_parts) { const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); - if (!part_values.empty() && part_values.find(part->name) == part_values.end()) + if (part_values && part_values->find(part->name) == part_values->end()) continue; if (part->isEmpty()) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index c058c7e936b..231908e2ee7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -16,6 +16,7 @@ class KeyCondition; struct MergeTreeDataSelectSamplingData { bool use_sampling = false; + bool read_nothing = false; std::shared_ptr filter_function; ActionsDAGPtr filter_expression; }; @@ -117,11 +118,12 @@ private: const String & query_id) const; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. - size_t getApproximateTotalRowsToRead( + static size_t getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, - const Settings & settings) const; + const Settings & settings, + Poco::Logger * log); static MarkRanges markRangesFromPKRange( const MergeTreeData::DataPartPtr & part, @@ -141,6 +143,7 @@ private: size_t & granules_dropped, Poco::Logger * log); +public: struct PartFilterCounters { size_t num_initial_selected_parts = 0; @@ -155,7 +158,7 @@ private: /// as well as `max_block_number_to_read`. static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, @@ -163,16 +166,28 @@ private: PartFilterCounters & counters); /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. - void selectPartsToReadWithUUIDFilter( + static void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - const std::unordered_set & part_values, + const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context, - PartFilterCounters & counters) const; + PartFilterCounters & counters, + Poco::Logger * log); + + static MergeTreeDataSelectSamplingData getSampling( + const ASTSelectQuery & select, + MergeTreeData::DataPartsVector & parts, + const StorageMetadataPtr & metadata_snapshot, + KeyCondition & key_condition, + const MergeTreeData & data, + Poco::Logger * log, + bool sample_factor_column_queried, + NamesAndTypesList available_real_columns, + ContextPtr context); }; } From 53295f9b514a64042aefd798e97d2549ab8fa129 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 19:53:42 +0300 Subject: [PATCH 201/652] blacklist --- tests/integration/ci-runner.py | 38 ++-- tests/integration/parallel.readme | 6 +- tests/integration/parallel_skip.json | 260 +++++++++++++++++++++++++++ 3 files changed, 286 insertions(+), 18 deletions(-) create mode 100644 tests/integration/parallel_skip.json diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 8ad804fbe00..74ae392d952 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -14,6 +14,7 @@ import csv MAX_RETRY = 2 NUM_WORKERS = 5 SLEEP_BETWEEN_RETRIES = 5 +PARALLEL_GROUP_SIZE = 100 CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge" DOCKERD_LOGS_PATH = "/ClickHouse/tests/integration/dockerd.log" @@ -52,6 +53,11 @@ def filter_existing_tests(tests_to_run, repo_path): def _get_deselect_option(tests): return ' '.join(['--deselect {}'.format(t) for t in tests]) +# https://stackoverflow.com/questions/312443/how-do-you-split-a-list-into-evenly-sized-chunks +def chunks(lst, n): + """Yield successive n-sized chunks from lst.""" + for i in range(0, len(lst), n): + yield lst[i:i + n] def parse_test_results_output(fname): read = False @@ -269,15 +275,15 @@ class ClickhouseIntegrationTestsRunner: all_tests.append(line.strip()) return list(sorted(all_tests)) - def _get_parallel_tests(self, repo_path): - parallel_tests_file_path = "{}/tests/integration/parallel.json".format(repo_path) - if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: - raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) + def _get_parallel_tests_skip_list(self, repo_path): + skip_list_file_path = "{}/tests/integration/parallel.json".format(repo_path) + if not os.path.isfile(skip_list_file_path) or os.path.getsize(skip_list_file_path) == 0: + raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(skip_list_file_path)) - parallel_tests = [] - with open(parallel_tests_file_path, "r") as parallel_tests_file: - parallel_tests = json.load(parallel_tests_file) - return list(sorted(parallel_tests)) + skip_list_tests = [] + with open(skip_list_file_path, "r") as skip_list_file: + skip_list_tests = json.load(skip_list_file) + return list(sorted(skip_list_tests)) def group_test_by_file(self, tests): result = {} @@ -298,7 +304,6 @@ class ClickhouseIntegrationTestsRunner: if test in main_counters["ERROR"]: main_counters["ERROR"].remove(test) is_flaky = True - if is_flaky: main_counters["FLAKY"].append(test) else: @@ -476,16 +481,19 @@ class ClickhouseIntegrationTestsRunner: self._install_clickhouse(build_path) logging.info("Dump iptables before run %s", subprocess.check_output("iptables -L", shell=True)) all_tests = self._get_all_tests(repo_path) - parallel_tests = self._get_parallel_tests(repo_path) + parallel_skip_tests = self._get_parallel_tests_skip_list(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) - filtered_parallel_tests = list(filter(lambda test: test in all_tests, parallel_tests)) - filtered_unparallel_tests = list(filter(lambda test: test not in parallel_tests, all_tests)) - not_found_tests = list(filter(lambda test: test not in all_tests, parallel_tests)) + filtered_sequential_tests = list(filter(lambda test: test in all_tests, parallel_skip_tests)) + filtered_parallel_tests = list(filter(lambda test: test not in parallel_skip_tests, all_tests)) + not_found_tests = list(filter(lambda test: test not in all_tests, parallel_skip_tests)) logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_unparallel_tests)) logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) - grouped_tests = self.group_test_by_file(filtered_unparallel_tests) - grouped_tests["parallel"] = filtered_parallel_tests + grouped_tests = self.group_test_by_file(filtered_sequential_tests) + i = 0 + for par_group in chunks(filtered_parallel_tests, PARALLEL_GROUP_SIZE): + grouped_tests["parallel{}".format(i)] = par_group + i+=1 logging.info("Found %s tests groups", len(grouped_tests)) counters = { diff --git a/tests/integration/parallel.readme b/tests/integration/parallel.readme index 3b987c689f4..ac4f897482c 100644 --- a/tests/integration/parallel.readme +++ b/tests/integration/parallel.readme @@ -1,6 +1,6 @@ -# Parallel tests are currently generated manually. All tests except those in parallel.txt will be run sequentially. +# Parallel tests skip list is currently generated manually. All tests except those in parallel_skip.txt will run in parallel. # Current list is generated with following commands # 1. Generate all tests list as in CI run ./runner ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt -# 2. Filter known tests that can be run in parallel -cat all_tests.txt | grep -v 'grpc\|SKIP\|test_backup_with_other_granularity\|test_ttl_replicated\|test_adaptive_granularity\|kafka' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel.json +# 2. Filter known tests that are currently not run in parallel +cat all_tests.txt | grep '^test_replicated_database\|^test_disabled_mysql_server\|^test_distributed_ddl\|^test_distributed_ddl\|^test_quorum_inserts_parallel\|^test_ddl_worker_non_leader\|^test_consistent_parts_after_clone_replica\|^test_materialize_mysql_database\|^test_atomic_drop_table\|^test_distributed_respect_user_timeouts\|^test_storage_kafka\|^test_replace_partition\|^test_replicated_fetches_timeouts\|^test_system_clusters_actual_information\|^test_delayed_replica_failover\|^test_limited_replicated_fetches\|^test_hedged_requests\|^test_insert_into_distributed\|^test_insert_into_distributed_through_materialized_view\|^test_drop_replica\|^test_attach_without_fetching\|^test_system_replicated_fetches\|^test_cross_replication\|^test_dictionary_allow_read_expired_keys\|^test_dictionary_allow_read_expired_keys\|^test_dictionary_allow_read_expired_keys\|^test_insert_into_distributed_sync_async\|^test_hedged_requests_parallel\|^test_dictionaries_update_field\|^test_broken_part_during_merge\|^test_random_inserts\|^test_reload_clusters_config\|^test_parts_delete_zookeeper\|^test_polymorphic_parts\|^test_keeper_multinode_simple\|^test_https_replication\|^test_storage_kerberized_kafka\|^test_cleanup_dir_after_bad_zk_conn\|^test_system_metrics\|^test_keeper_multinode_blocade_leader' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel_skip.json diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json new file mode 100644 index 00000000000..4be0c9a77ae --- /dev/null +++ b/tests/integration/parallel_skip.json @@ -0,0 +1,260 @@ +[ + "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", + "test_attach_without_fetching/test.py::test_attach_without_fetching", + "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path", + "test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active", + "test_cross_replication/test.py::test", + "test_ddl_worker_non_leader/test.py::test_non_leader_replica", + "test_delayed_replica_failover/test.py::test", + "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", + "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", + "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", + "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", + "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", + "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", + "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", + "test_disabled_mysql_server/test.py::test_disabled_mysql_server", + "test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl", + "test_distributed_ddl_on_cross_replication/test.py::test_atomic_database", + "test_distributed_ddl_parallel/test.py::test_all_in_parallel", + "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_distributed_ddl_parallel/test.py::test_smoke", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", + "test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued", + "test_distributed_ddl_password/test.py::test_alter", + "test_distributed_ddl_password/test.py::test_truncate", + "test_distributed_ddl/test.py::test_allowed_databases[configs]", + "test_distributed_ddl/test.py::test_allowed_databases[configs_secure]", + "test_distributed_ddl/test.py::test_create_as_select[configs]", + "test_distributed_ddl/test.py::test_create_as_select[configs_secure]", + "test_distributed_ddl/test.py::test_create_reserved[configs]", + "test_distributed_ddl/test.py::test_create_reserved[configs_secure]", + "test_distributed_ddl/test.py::test_create_view[configs]", + "test_distributed_ddl/test.py::test_create_view[configs_secure]", + "test_distributed_ddl/test.py::test_default_database[configs]", + "test_distributed_ddl/test.py::test_default_database[configs_secure]", + "test_distributed_ddl/test.py::test_detach_query[configs]", + "test_distributed_ddl/test.py::test_detach_query[configs_secure]", + "test_distributed_ddl/test.py::test_implicit_macros[configs]", + "test_distributed_ddl/test.py::test_implicit_macros[configs_secure]", + "test_distributed_ddl/test.py::test_kill_query[configs]", + "test_distributed_ddl/test.py::test_kill_query[configs_secure]", + "test_distributed_ddl/test.py::test_macro[configs]", + "test_distributed_ddl/test.py::test_macro[configs_secure]", + "test_distributed_ddl/test.py::test_on_connection_loss[configs]", + "test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]", + "test_distributed_ddl/test.py::test_on_server_fail[configs]", + "test_distributed_ddl/test.py::test_on_server_fail[configs_secure]", + "test_distributed_ddl/test.py::test_on_session_expired[configs]", + "test_distributed_ddl/test.py::test_on_session_expired[configs_secure]", + "test_distributed_ddl/test.py::test_optimize_query[configs]", + "test_distributed_ddl/test.py::test_optimize_query[configs_secure]", + "test_distributed_ddl/test.py::test_rename[configs]", + "test_distributed_ddl/test.py::test_rename[configs_secure]", + "test_distributed_ddl/test.py::test_replicated_without_arguments[configs]", + "test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]", + "test_distributed_ddl/test.py::test_simple_alters[configs]", + "test_distributed_ddl/test.py::test_simple_alters[configs_secure]", + "test_distributed_ddl/test.py::test_socket_timeout[configs]", + "test_distributed_ddl/test.py::test_socket_timeout[configs_secure]", + "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]", + "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]", + "test_drop_replica/test.py::test_drop_replica", + "test_hedged_requests_parallel/test.py::test_combination1", + "test_hedged_requests_parallel/test.py::test_combination2", + "test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample", + "test_hedged_requests_parallel/test.py::test_send_data", + "test_hedged_requests_parallel/test.py::test_send_table_status_sleep", + "test_hedged_requests/test.py::test_combination1", + "test_hedged_requests/test.py::test_combination2", + "test_hedged_requests/test.py::test_combination3", + "test_hedged_requests/test.py::test_combination4", + "test_hedged_requests/test.py::test_long_query", + "test_hedged_requests/test.py::test_receive_timeout1", + "test_hedged_requests/test.py::test_receive_timeout2", + "test_hedged_requests/test.py::test_send_data", + "test_hedged_requests/test.py::test_send_data2", + "test_hedged_requests/test.py::test_send_table_status_sleep", + "test_hedged_requests/test.py::test_send_table_status_sleep2", + "test_hedged_requests/test.py::test_stuck_replica", + "test_https_replication/test.py::test_both_http", + "test_https_replication/test.py::test_both_https", + "test_https_replication/test.py::test_mixed_protocol", + "test_https_replication/test.py::test_replication_after_partition", + "test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout", + "test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout", + "test_insert_into_distributed/test.py::test_inserts_batching", + "test_insert_into_distributed/test.py::test_inserts_local", + "test_insert_into_distributed/test.py::test_inserts_low_cardinality", + "test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication", + "test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication", + "test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication", + "test_insert_into_distributed/test.py::test_prefer_localhost_replica", + "test_insert_into_distributed/test.py::test_reconnect", + "test_insert_into_distributed/test.py::test_table_function", + "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED", + "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", + "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", + "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", + "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice", + "test_keeper_multinode_simple/test.py::test_follower_restart", + "test_keeper_multinode_simple/test.py::test_read_write_multinode", + "test_keeper_multinode_simple/test.py::test_session_expiration", + "test_keeper_multinode_simple/test.py::test_simple_replicated_table", + "test_keeper_multinode_simple/test.py::test_watch_on_follower", + "test_limited_replicated_fetches/test.py::test_limited_fetches", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", + "test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper", + "test_polymorphic_parts/test.py::test_compact_parts_only", + "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]", + "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]", + "test_polymorphic_parts/test.py::test_in_memory", + "test_polymorphic_parts/test.py::test_in_memory_alters", + "test_polymorphic_parts/test.py::test_in_memory_deduplication", + "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", + "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", + "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", + "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED", + "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED", + "test_polymorphic_parts/test.py::test_polymorphic_parts_index", + "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", + "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", + "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum", + "test_random_inserts/test.py::test_insert_multithreaded", + "test_random_inserts/test.py::test_random_inserts", + "test_reload_clusters_config/test.py::test_add_cluster", + "test_reload_clusters_config/test.py::test_delete_cluster", + "test_reload_clusters_config/test.py::test_simple_reload", + "test_reload_clusters_config/test.py::test_update_one_cluster", + "test_replace_partition/test.py::test_drop_failover", + "test_replace_partition/test.py::test_normal_work", + "test_replace_partition/test.py::test_replace_after_replace_failover", + "test_replicated_database/test.py::test_alters_from_different_replicas", + "test_replicated_database/test.py::test_create_replicated_table", + "test_replicated_database/test.py::test_recover_staled_replica", + "test_replicated_database/test.py::test_simple_alter_table[MergeTree]", + "test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]", + "test_replicated_database/test.py::test_startup_without_zk", + "test_replicated_fetches_timeouts/test.py::test_no_stall", + "test_storage_kafka/test.py::test_bad_reschedule", + "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", + "test_storage_kafka/test.py::test_exception_from_destructor", + "test_storage_kafka/test.py::test_kafka_commit_on_block_write", + "test_storage_kafka/test.py::test_kafka_consumer_hang", + "test_storage_kafka/test.py::test_kafka_consumer_hang2", + "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", + "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", + "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", + "test_storage_kafka/test.py::test_kafka_flush_by_block_size", + "test_storage_kafka/test.py::test_kafka_flush_by_time", + "test_storage_kafka/test.py::test_kafka_flush_on_big_message", + "test_storage_kafka/test.py::test_kafka_formats", + "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", + "test_storage_kafka/test.py::test_kafka_insert", + "test_storage_kafka/test.py::test_kafka_issue11308", + "test_storage_kafka/test.py::test_kafka_issue14202", + "test_storage_kafka/test.py::test_kafka_issue4116", + "test_storage_kafka/test.py::test_kafka_json_as_string", + "test_storage_kafka/test.py::test_kafka_json_without_delimiter", + "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", + "test_storage_kafka/test.py::test_kafka_many_materialized_views", + "test_storage_kafka/test.py::test_kafka_materialized_view", + "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", + "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", + "test_storage_kafka/test.py::test_kafka_produce_consume", + "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", + "test_storage_kafka/test.py::test_kafka_protobuf", + "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", + "test_storage_kafka/test.py::test_kafka_rebalance", + "test_storage_kafka/test.py::test_kafka_select_empty", + "test_storage_kafka/test.py::test_kafka_settings_new_syntax", + "test_storage_kafka/test.py::test_kafka_settings_old_syntax", + "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", + "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", + "test_storage_kafka/test.py::test_kafka_unavailable", + "test_storage_kafka/test.py::test_kafka_virtual_columns", + "test_storage_kafka/test.py::test_kafka_virtual_columns2", + "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", + "test_storage_kafka/test.py::test_librdkafka_compression", + "test_storage_kafka/test.py::test_premature_flush_on_eof", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", + "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", + "test_system_clusters_actual_information/test.py::test", + "test_system_metrics/test.py::test_readonly_metrics", + "test_system_replicated_fetches/test.py::test_system_replicated_fetches" +] From 9282c7470c330514fd35c38fe0961ff38c345516 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 19:53:55 +0300 Subject: [PATCH 202/652] better --- tests/integration/test_attach_without_fetching/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index c714d09cf58..605ca6a4f51 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -98,7 +98,7 @@ def test_attach_without_fetching(start_cluster): to_delete = node_2.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin'.format( p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True) - logging.debug("Before deleting:", to_delete) + logging.debug(f"Before deleting: {to_delete}") node_2.exec_in_container(['bash', '-c', 'cd {p} && rm -fr *.bin'.format( From b44c056fa4f86a096f91d622d4b62836786e4cad Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 19:55:02 +0300 Subject: [PATCH 203/652] lost file --- tests/integration/parallel.json | 459 ++++++++++++++------------------ 1 file changed, 204 insertions(+), 255 deletions(-) diff --git a/tests/integration/parallel.json b/tests/integration/parallel.json index 58ca2209131..4be0c9a77ae 100644 --- a/tests/integration/parallel.json +++ b/tests/integration/parallel.json @@ -1,125 +1,15 @@ [ - "test_dictionaries_access/test.py::test_create", - "test_dictionaries_access/test.py::test_dictget", - "test_dictionaries_access/test.py::test_drop", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_cassandra.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_cache.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_file.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_http.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_https.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mongo.py::test_simple[hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]", - "test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]", - "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[memory]", - "test_dictionaries_complex_key_cache_string/test.py::test_memory_consumption[ssd]", - "test_dictionaries_ddl/test.py::test_clickhouse_remote", - "test_dictionaries_ddl/test.py::test_conflicting_name", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]", - "test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]", - "test_dictionaries_ddl/test.py::test_dictionary_with_where", - "test_dictionaries_ddl/test.py::test_file_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_http_dictionary_restrictions", - "test_dictionaries_ddl/test.py::test_restricted_database", - "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_dictionary_database[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_explicit_table[node1]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node0]", - "test_dictionaries_dependency/test.py::test_dependency_via_implicit_table[node1]", - "test_dictionaries_dependency_xml/test.py::test_get_data", - "test_dictionaries_mysql/test.py::test_load_mysql_dictionaries", - "test_dictionaries_null_value/test.py::test_null_value", - "test_dictionaries_postgresql/test.py::test_dictionary_with_replicas", - "test_dictionaries_postgresql/test.py::test_invalidate_query", - "test_dictionaries_postgresql/test.py::test_load_dictionaries", - "test_dictionaries_postgresql/test.py::test_postgres_scema", - "test_dictionaries_redis/test.py::test_redis_dictionaries[0]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[1]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[10]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[11]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[12]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[13]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[2]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[3]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[4]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[5]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[6]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[7]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[8]", - "test_dictionaries_redis/test.py::test_redis_dictionaries[9]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_integers_key_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_complex_mixed_key_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_flat]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_hashed]", - "test_dictionaries_select_all/test.py::test_select_all[clickhouse_range_hashed]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_integers_key_cache]", - "test_dictionaries_select_all/test.py::test_select_all_from_cached[clickhouse_complex_mixed_key_cache]", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_system_reload", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_by_timer", - "test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary", - "test_dictionaries_update_and_reload/test.py::test_reload_after_loading", - "test_dictionaries_update_and_reload/test.py::test_reload_while_loading", + "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", + "test_attach_without_fetching/test.py::test_attach_without_fetching", + "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name", + "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path", + "test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active", + "test_cross_replication/test.py::test", + "test_ddl_worker_non_leader/test.py::test_non_leader_replica", + "test_delayed_replica_failover/test.py::test", "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", @@ -127,50 +17,198 @@ "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", - "test_dictionary_custom_settings/test.py::test_work", - "test_dictionary_ddl_on_cluster/test.py::test_dictionary_ddl_on_cluster", - "test_disk_access_storage/test.py::test_alter", - "test_disk_access_storage/test.py::test_create", - "test_disk_access_storage/test.py::test_drop", + "test_disabled_mysql_server/test.py::test_disabled_mysql_server", + "test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl", + "test_distributed_ddl_on_cross_replication/test.py::test_atomic_database", + "test_distributed_ddl_parallel/test.py::test_all_in_parallel", "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", + "test_distributed_ddl_parallel/test.py::test_smoke", + "test_distributed_ddl_parallel/test.py::test_smoke_parallel", "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", - "test_distributed_storage_configuration/test.py::test_insert", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow", - "test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema", - "test_odbc_interaction/test.py::test_sqlite_odbc_cached_dictionary", - "test_odbc_interaction/test.py::test_sqlite_odbc_hashed_dictionary", - "test_odbc_interaction/test.py::test_sqlite_simple_select_storage_works", - "test_range_hashed_dictionary_types/test.py::test_range_hashed_dict", - "test_redirect_url_storage/test.py::test_url_with_globs", - "test_redirect_url_storage/test.py::test_url_with_globs_and_failover", - "test_redirect_url_storage/test.py::test_url_without_redirect", - "test_redirect_url_storage/test.py::test_url_with_redirect_allowed", - "test_redirect_url_storage/test.py::test_url_with_redirect_not_allowed", - "test_reloading_storage_configuration/test.py::test_add_disk", - "test_reloading_storage_configuration/test.py::test_add_disk_to_policy", - "test_reloading_storage_configuration/test.py::test_add_disk_to_separate_config", - "test_reloading_storage_configuration/test.py::test_add_policy", - "test_reloading_storage_configuration/test.py::test_add_volume_to_policy", - "test_reloading_storage_configuration/test.py::test_new_policy_works", - "test_reloading_storage_configuration/test.py::test_remove_disk", - "test_reloading_storage_configuration/test.py::test_remove_disk_from_policy", - "test_reloading_storage_configuration/test.py::test_remove_policy", - "test_reloading_storage_configuration/test.py::test_remove_volume_from_policy", - "test_s3_zero_copy_replication/test.py::test_s3_zero_copy_on_hybrid_storage", - "test_storage_hdfs/test.py::test_bad_hdfs_uri", - "test_storage_hdfs/test.py::test_globs_in_read_table", - "test_storage_hdfs/test.py::test_read_files_with_spaces", - "test_storage_hdfs/test.py::test_read_write_gzip_table", - "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_auto_gz", - "test_storage_hdfs/test.py::test_read_write_gzip_table_with_parameter_gzip", - "test_storage_hdfs/test.py::test_read_write_storage", - "test_storage_hdfs/test.py::test_read_write_storage_with_globs", - "test_storage_hdfs/test.py::test_read_write_table", - "test_storage_hdfs/test.py::test_read_write_table_with_parameter_none", - "test_storage_hdfs/test.py::test_virtual_columns", - "test_storage_hdfs/test.py::test_write_gzip_storage", - "test_storage_hdfs/test.py::test_write_gz_storage", - "test_storage_hdfs/test.py::test_write_table", + "test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued", + "test_distributed_ddl_password/test.py::test_alter", + "test_distributed_ddl_password/test.py::test_truncate", + "test_distributed_ddl/test.py::test_allowed_databases[configs]", + "test_distributed_ddl/test.py::test_allowed_databases[configs_secure]", + "test_distributed_ddl/test.py::test_create_as_select[configs]", + "test_distributed_ddl/test.py::test_create_as_select[configs_secure]", + "test_distributed_ddl/test.py::test_create_reserved[configs]", + "test_distributed_ddl/test.py::test_create_reserved[configs_secure]", + "test_distributed_ddl/test.py::test_create_view[configs]", + "test_distributed_ddl/test.py::test_create_view[configs_secure]", + "test_distributed_ddl/test.py::test_default_database[configs]", + "test_distributed_ddl/test.py::test_default_database[configs_secure]", + "test_distributed_ddl/test.py::test_detach_query[configs]", + "test_distributed_ddl/test.py::test_detach_query[configs_secure]", + "test_distributed_ddl/test.py::test_implicit_macros[configs]", + "test_distributed_ddl/test.py::test_implicit_macros[configs_secure]", + "test_distributed_ddl/test.py::test_kill_query[configs]", + "test_distributed_ddl/test.py::test_kill_query[configs_secure]", + "test_distributed_ddl/test.py::test_macro[configs]", + "test_distributed_ddl/test.py::test_macro[configs_secure]", + "test_distributed_ddl/test.py::test_on_connection_loss[configs]", + "test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]", + "test_distributed_ddl/test.py::test_on_server_fail[configs]", + "test_distributed_ddl/test.py::test_on_server_fail[configs_secure]", + "test_distributed_ddl/test.py::test_on_session_expired[configs]", + "test_distributed_ddl/test.py::test_on_session_expired[configs_secure]", + "test_distributed_ddl/test.py::test_optimize_query[configs]", + "test_distributed_ddl/test.py::test_optimize_query[configs_secure]", + "test_distributed_ddl/test.py::test_rename[configs]", + "test_distributed_ddl/test.py::test_rename[configs_secure]", + "test_distributed_ddl/test.py::test_replicated_without_arguments[configs]", + "test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]", + "test_distributed_ddl/test.py::test_simple_alters[configs]", + "test_distributed_ddl/test.py::test_simple_alters[configs_secure]", + "test_distributed_ddl/test.py::test_socket_timeout[configs]", + "test_distributed_ddl/test.py::test_socket_timeout[configs_secure]", + "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]", + "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]", + "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]", + "test_drop_replica/test.py::test_drop_replica", + "test_hedged_requests_parallel/test.py::test_combination1", + "test_hedged_requests_parallel/test.py::test_combination2", + "test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample", + "test_hedged_requests_parallel/test.py::test_send_data", + "test_hedged_requests_parallel/test.py::test_send_table_status_sleep", + "test_hedged_requests/test.py::test_combination1", + "test_hedged_requests/test.py::test_combination2", + "test_hedged_requests/test.py::test_combination3", + "test_hedged_requests/test.py::test_combination4", + "test_hedged_requests/test.py::test_long_query", + "test_hedged_requests/test.py::test_receive_timeout1", + "test_hedged_requests/test.py::test_receive_timeout2", + "test_hedged_requests/test.py::test_send_data", + "test_hedged_requests/test.py::test_send_data2", + "test_hedged_requests/test.py::test_send_table_status_sleep", + "test_hedged_requests/test.py::test_send_table_status_sleep2", + "test_hedged_requests/test.py::test_stuck_replica", + "test_https_replication/test.py::test_both_http", + "test_https_replication/test.py::test_both_https", + "test_https_replication/test.py::test_mixed_protocol", + "test_https_replication/test.py::test_replication_after_partition", + "test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout", + "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout", + "test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout", + "test_insert_into_distributed/test.py::test_inserts_batching", + "test_insert_into_distributed/test.py::test_inserts_local", + "test_insert_into_distributed/test.py::test_inserts_low_cardinality", + "test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication", + "test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication", + "test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication", + "test_insert_into_distributed/test.py::test_prefer_localhost_replica", + "test_insert_into_distributed/test.py::test_reconnect", + "test_insert_into_distributed/test.py::test_table_function", + "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED", + "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", + "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", + "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", + "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice", + "test_keeper_multinode_simple/test.py::test_follower_restart", + "test_keeper_multinode_simple/test.py::test_read_write_multinode", + "test_keeper_multinode_simple/test.py::test_session_expiration", + "test_keeper_multinode_simple/test.py::test_simple_replicated_table", + "test_keeper_multinode_simple/test.py::test_watch_on_follower", + "test_limited_replicated_fetches/test.py::test_limited_fetches", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]", + "test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", + "test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", + "test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper", + "test_polymorphic_parts/test.py::test_compact_parts_only", + "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]", + "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]", + "test_polymorphic_parts/test.py::test_in_memory", + "test_polymorphic_parts/test.py::test_in_memory_alters", + "test_polymorphic_parts/test.py::test_in_memory_deduplication", + "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", + "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", + "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", + "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED", + "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED", + "test_polymorphic_parts/test.py::test_polymorphic_parts_index", + "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", + "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", + "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum", + "test_random_inserts/test.py::test_insert_multithreaded", + "test_random_inserts/test.py::test_random_inserts", + "test_reload_clusters_config/test.py::test_add_cluster", + "test_reload_clusters_config/test.py::test_delete_cluster", + "test_reload_clusters_config/test.py::test_simple_reload", + "test_reload_clusters_config/test.py::test_update_one_cluster", + "test_replace_partition/test.py::test_drop_failover", + "test_replace_partition/test.py::test_normal_work", + "test_replace_partition/test.py::test_replace_after_replace_failover", + "test_replicated_database/test.py::test_alters_from_different_replicas", + "test_replicated_database/test.py::test_create_replicated_table", + "test_replicated_database/test.py::test_recover_staled_replica", + "test_replicated_database/test.py::test_simple_alter_table[MergeTree]", + "test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]", + "test_replicated_database/test.py::test_startup_without_zk", + "test_replicated_fetches_timeouts/test.py::test_no_stall", "test_storage_kafka/test.py::test_bad_reschedule", "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", "test_storage_kafka/test.py::test_exception_from_destructor", @@ -214,98 +252,9 @@ "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", "test_storage_kafka/test.py::test_librdkafka_compression", "test_storage_kafka/test.py::test_premature_flush_on_eof", - "test_storage_kerberized_hdfs/test.py::test_cache_path", - "test_storage_kerberized_hdfs/test.py::test_prohibited", - "test_storage_kerberized_hdfs/test.py::test_read_table", - "test_storage_kerberized_hdfs/test.py::test_read_table_expired", - "test_storage_kerberized_hdfs/test.py::test_read_write_storage", - "test_storage_kerberized_hdfs/test.py::test_two_users", - "test_storage_kerberized_hdfs/test.py::test_write_storage_not_expired", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", - "test_storage_mongodb/test.py::test_complex_data_type", - "test_storage_mongodb/test.py::test_incorrect_data_type", - "test_storage_mongodb/test.py::test_simple_select", - "test_storage_mysql/test.py::test_binary_type", - "test_storage_mysql/test.py::test_enum_type", - "test_storage_mysql/test.py::test_external_settings", - "test_storage_mysql/test.py::test_insert_on_duplicate_select", - "test_storage_mysql/test.py::test_insert_select", - "test_storage_mysql/test.py::test_many_connections", - "test_storage_mysql/test.py::test_mysql_distributed", - "test_storage_mysql/test.py::test_replace_select", - "test_storage_mysql/test.py::test_table_function", - "test_storage_mysql/test.py::test_where", - "test_storage_postgresql/test.py::test_concurrent_queries", - "test_storage_postgresql/test.py::test_non_default_scema", - "test_storage_postgresql/test.py::test_postgres_conversions", - "test_storage_postgresql/test.py::test_postgres_distributed", - "test_storage_postgresql/test.py::test_postgres_select_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_big_message", - "test_storage_rabbitmq/test.py::test_rabbitmq_commit_on_block_write", - "test_storage_rabbitmq/test.py::test_rabbitmq_csv_with_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_direct_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_fanout_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_format_factory_settings", - "test_storage_rabbitmq/test.py::test_rabbitmq_hash_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_headers_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_insert_headers_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_json_without_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_consumers_to_each_queue", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_inserts", - "test_storage_rabbitmq/test.py::test_rabbitmq_many_materialized_views", - "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view", - "test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view_with_subquery", - "test_storage_rabbitmq/test.py::test_rabbitmq_multiple_bindings", - "test_storage_rabbitmq/test.py::test_rabbitmq_mv_combo", - "test_storage_rabbitmq/test.py::test_rabbitmq_no_connection_at_startup", - "test_storage_rabbitmq/test.py::test_rabbitmq_overloaded_insert", - "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_1", - "test_storage_rabbitmq/test.py::test_rabbitmq_restore_failed_connection_without_losses_2", - "test_storage_rabbitmq/test.py::test_rabbitmq_select", - "test_storage_rabbitmq/test.py::test_rabbitmq_select_empty", - "test_storage_rabbitmq/test.py::test_rabbitmq_sharding_between_queues_publish", - "test_storage_rabbitmq/test.py::test_rabbitmq_topic_exchange", - "test_storage_rabbitmq/test.py::test_rabbitmq_tsv_with_delimiter", - "test_storage_rabbitmq/test.py::test_rabbitmq_vhost", - "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns", - "test_storage_rabbitmq/test.py::test_rabbitmq_virtual_columns_with_materialized_view", - "test_storage_s3/test.py::test_custom_auth_headers", - "test_storage_s3/test.py::test_custom_auth_headers_exclusion", - "test_storage_s3/test.py::test_empty_put[minio]", - "test_storage_s3/test.py::test_get_file_with_special[plus]", - "test_storage_s3/test.py::test_get_file_with_special[space]", - "test_storage_s3/test.py::test_get_path_with_special[plus]", - "test_storage_s3/test.py::test_get_path_with_special[plus2]", - "test_storage_s3/test.py::test_get_path_with_special[space]", - "test_storage_s3/test.py::test_infinite_redirect", - "test_storage_s3/test.py::test_multipart_put[negative]", - "test_storage_s3/test.py::test_multipart_put[positive]", - "test_storage_s3/test.py::test_put[auth_positive]", - "test_storage_s3/test.py::test_put[auto]", - "test_storage_s3/test.py::test_put[brotli]", - "test_storage_s3/test.py::test_put_csv[auth_positive]", - "test_storage_s3/test.py::test_put_csv[negative]", - "test_storage_s3/test.py::test_put_csv[positive]", - "test_storage_s3/test.py::test_put[deflate]", - "test_storage_s3/test.py::test_put_get_with_globs", - "test_storage_s3/test.py::test_put_get_with_redirect", - "test_storage_s3/test.py::test_put[gzip]", - "test_storage_s3/test.py::test_put[positive]", - "test_storage_s3/test.py::test_put_with_zero_redirect", - "test_storage_s3/test.py::test_put[xz]", - "test_storage_s3/test.py::test_put[zstd]", - "test_storage_s3/test.py::test_remote_host_filter", - "test_storage_s3/test.py::test_s3_glob_scheherazade", - "test_storage_s3/test.py::test_storage_s3_get_gzip[bin]", - "test_storage_s3/test.py::test_storage_s3_get_gzip[gz]", - "test_storage_s3/test.py::test_storage_s3_get_unstable", - "test_storage_s3/test.py::test_storage_s3_put_gzip[bin]", - "test_storage_s3/test.py::test_storage_s3_put_gzip[gz]", - "test_storage_s3/test.py::test_storage_s3_put_uncompressed", - "test_storage_s3/test.py::test_wrong_s3_syntax[1_argument]", - "test_storage_s3/test.py::test_wrong_s3_syntax[6_arguments]", - "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[mt_test_moves_work_after_storage_policy_change]", - "test_ttl_move/test.py::test_moves_work_after_storage_policy_change[replicated_mt_test_moves_work_after_storage_policy_change]" + "test_system_clusters_actual_information/test.py::test", + "test_system_metrics/test.py::test_readonly_metrics", + "test_system_replicated_fetches/test.py::test_system_replicated_fetches" ] From 0ca1ce8ab3713d4ec6af82b5611bd29a8e049d17 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 19:55:53 +0300 Subject: [PATCH 204/652] shellcheck fix --- docker/test/integration/runner/dockerd-entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 6ef077585ba..626b6883b91 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -28,8 +28,8 @@ done set -e # cleanup for retry run if volume is not recreated -docker kill `docker ps -aq` || true -docker rm `docker ps -aq` || true +docker kill "$(docker ps -aq)" || true +docker rm "$(docker ps -aq)" || true echo "Start tests" export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse From f5e584b6af53b6a6e1b24a825bd72c1a720ba96e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 25 May 2021 19:56:03 +0300 Subject: [PATCH 205/652] fix --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 74ae392d952..75460f6baa8 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -276,7 +276,7 @@ class ClickhouseIntegrationTestsRunner: return list(sorted(all_tests)) def _get_parallel_tests_skip_list(self, repo_path): - skip_list_file_path = "{}/tests/integration/parallel.json".format(repo_path) + skip_list_file_path = "{}/tests/integration/parallel_skip.json".format(repo_path) if not os.path.isfile(skip_list_file_path) or os.path.getsize(skip_list_file_path) == 0: raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(skip_list_file_path)) From 1fbcfd9c6a443c3d97b6deaff1be295865b0934b Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Wed, 26 May 2021 00:58:49 +0300 Subject: [PATCH 206/652] Added check for temp file existance --- src/Parsers/examples/YAML_fuzzer.cpp | 28 ++-- tests/fuzz/YAML_fuzzer.dict | 198 --------------------------- 2 files changed, 18 insertions(+), 208 deletions(-) delete mode 100644 tests/fuzz/YAML_fuzzer.dict diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Parsers/examples/YAML_fuzzer.cpp index 1abd9adfcb7..f1486c8563b 100644 --- a/src/Parsers/examples/YAML_fuzzer.cpp +++ b/src/Parsers/examples/YAML_fuzzer.cpp @@ -1,26 +1,34 @@ #include #include #include - -#include -#include -#include -#include - +#include +#include extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { + /// How to test: + /// build ClickHouse with YAML_fuzzer.cpp + /// ./YAML_fuzzer YAML_CORPUS + /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer + + srand(time(NULL)); + std::string cur_file = std::to_string(rand()); + + while (std::filesystem::exists(cur_file)) + { + std::string cur_file = std::to_string(rand()); + } + std::string input = std::string(reinterpret_cast(data), size); DB::YAMLParser parser; - std::ofstream temp_file("YAML_fuzzer_data.yaml"); + std::ofstream temp_file(cur_file); temp_file << input; temp_file.close(); - YAMLParser::parse("YAML_fuzzer_data.yaml"); - - remove("YAML_fuzzer_data.yaml"); + DB::YAMLParser::parse(cur_file); + remove(cur_file.c_str()); return 0; } diff --git a/tests/fuzz/YAML_fuzzer.dict b/tests/fuzz/YAML_fuzzer.dict deleted file mode 100644 index 1e34e62b620..00000000000 --- a/tests/fuzz/YAML_fuzzer.dict +++ /dev/null @@ -1,198 +0,0 @@ -"key1" -"key2" -"key3" -"key4" -"key5" -"key6" -"key7" -"key8" -"key9" -"key10" -"key11" -"key12" -"key13" -"key14" -"key15" -"key16" -"key17" -"key18" -"key19" -"key20" -"key21" -"key22" -"key23" -"key24" -"key25" -"key26" -"key27" -"key28" -"key29" -"key30" -"key31" -"key32" -"key33" -"key34" -"key35" -"key36" -"key37" -"key38" -"key39" -"key40" -"key41" -"key42" -"key43" -"key44" -"key45" -"key46" -"key47" -"key48" -"key49" -"key50" -"key51" -"key52" -"key53" -"key54" -"key55" -"key56" -"key57" -"key58" -"key59" -"key60" -"key61" -"key62" -"key63" -"key64" -"key65" -"key66" -"key67" -"key68" -"key69" -"key70" -"key71" -"key72" -"key73" -"key74" -"key75" -"key76" -"key77" -"key78" -"key79" -"key80" -"key81" -"key82" -"key83" -"key84" -"key85" -"key86" -"key87" -"key88" -"key89" -"key90" -"key91" -"key92" -"key93" -"key94" -"key95" -"key96" -"key97" -"key98" -"key99" -"val1" -"val2" -"val3" -"val4" -"val5" -"val6" -"val7" -"val8" -"val9" -"val10" -"val11" -"val12" -"val13" -"val14" -"val15" -"val16" -"val17" -"val18" -"val19" -"val20" -"val21" -"val22" -"val23" -"val24" -"val25" -"val26" -"val27" -"val28" -"val29" -"val30" -"val31" -"val32" -"val33" -"val34" -"val35" -"val36" -"val37" -"val38" -"val39" -"val40" -"val41" -"val42" -"val43" -"val44" -"val45" -"val46" -"val47" -"val48" -"val49" -"val50" -"val51" -"val52" -"val53" -"val54" -"val55" -"val56" -"val57" -"val58" -"val59" -"val60" -"val61" -"val62" -"val63" -"val64" -"val65" -"val66" -"val67" -"val68" -"val69" -"val70" -"val71" -"val72" -"val73" -"val74" -"val75" -"val76" -"val77" -"val78" -"val79" -"val80" -"val81" -"val82" -"val83" -"val84" -"val85" -"val86" -"val87" -"val88" -"val89" -"val90" -"val91" -"val92" -"val93" -"val94" -"val95" -"val96" -"val97" -"val98" -"val99" From 374923ab90bdba0d107427fb1e38f96f94d22a62 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 May 2021 03:48:11 +0300 Subject: [PATCH 207/652] Update integration test --- .../configs/allow_experimental_codecs.xml | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml diff --git a/tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml b/tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml new file mode 100644 index 00000000000..2c7de77cd37 --- /dev/null +++ b/tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml @@ -0,0 +1,7 @@ + + + + 1 + + + From b26b97ec6c830d5f5b60c9e80a51db41728dc108 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 26 May 2021 07:40:53 +0300 Subject: [PATCH 208/652] fix --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 75460f6baa8..1efc5c3e78a 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -486,7 +486,7 @@ class ClickhouseIntegrationTestsRunner: filtered_sequential_tests = list(filter(lambda test: test in all_tests, parallel_skip_tests)) filtered_parallel_tests = list(filter(lambda test: test not in parallel_skip_tests, all_tests)) not_found_tests = list(filter(lambda test: test not in all_tests, parallel_skip_tests)) - logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_unparallel_tests)) + logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_sequential_tests)) logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) grouped_tests = self.group_test_by_file(filtered_sequential_tests) From 84914ca12fdce7f87c7aab98087526de64ce1bc5 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Wed, 26 May 2021 10:56:51 +0300 Subject: [PATCH 209/652] Added YAML_fuzzer to CMakeLists + fixed .as errors after testing --- src/Common/Config/YAMLParser.cpp | 30 +++++++++++++++++++++++++--- src/Parsers/examples/CMakeLists.txt | 3 +++ src/Parsers/examples/YAML_fuzzer.cpp | 2 +- 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index 9eaf1cdc1ad..704a2003e92 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -57,7 +57,15 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen { case YAML::NodeType::Scalar: { - auto value = node.as(); + std::string value; + try + { + value = node.as(); + } + catch (const YAML::TypedBadConversion&) + { + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with value which cannot be represented as string and cannot continue parsing of the file"); + } Poco::AutoPtr xml_value = xml_document->createTextNode(value); parent_xml_element.appendChild(xml_value); break; @@ -110,13 +118,29 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen { const auto & key_node = key_value_pair.first; const auto & value_node = key_value_pair.second; - auto key = key_node.as(); + std::string key; + try + { + key = key_node.as(); + } + catch (const YAML::TypedBadConversion&) + { + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with key which cannot be represented as string and cannot continue parsing of the file"); + } bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar()); if (is_attribute) { /// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key auto attribute_name = key.substr(1); - auto value = value_node.as(); + std::string value; + try + { + value = value_node.as(); + } + catch (const YAML::TypedBadConversion&) + { + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with value which cannot be represented as string and cannot continue parsing of the file"); + } parent_xml_element.setAttribute(attribute_name, value); } else diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 3ff40b3cc3b..578e87761e1 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -18,4 +18,7 @@ if (ENABLE_FUZZING) add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) + + add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) + target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) endif () diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Parsers/examples/YAML_fuzzer.cpp index f1486c8563b..cf47633cb09 100644 --- a/src/Parsers/examples/YAML_fuzzer.cpp +++ b/src/Parsers/examples/YAML_fuzzer.cpp @@ -25,7 +25,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::ofstream temp_file(cur_file); temp_file << input; temp_file.close(); - + DB::YAMLParser::parse(cur_file); remove(cur_file.c_str()); From 46787bf51fd6635a40558585f57586e4a35fa4cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 26 May 2021 12:01:56 +0200 Subject: [PATCH 210/652] Disallow building a uniqXXXXState on top of another aggregation state Fixes https://github.com/ClickHouse/ClickHouse/issues/24461 --- src/AggregateFunctions/AggregateFunctionUniq.cpp | 15 ++++++++++++--- .../AggregateFunctionUniqCombined.cpp | 5 +++++ .../AggregateFunctionUniqUpTo.cpp | 5 +++++ 3 files changed, 22 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index bc44eb8eece..2ee1d42aad8 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -5,11 +5,8 @@ #include #include -#include -#include #include #include -#include "registerAggregateFunctions.h" namespace DB @@ -39,6 +36,12 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); + const WhichDataType t(argument_types[0]); + if (t.isAggregateFunction()) + throw Exception( + "Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (argument_types.size() == 1) { const IDataType & argument_type = *argument_types[0]; @@ -81,6 +84,12 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + const WhichDataType t(argument_types[0]); + if (t.isAggregateFunction()) + throw Exception( + "Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + /// We use exact hash function if the user wants it; /// or if the arguments are not contiguous in memory, because only exact hash function have support for this case. bool use_exact_hash_function = is_exact || !isAllArgumentsContiguousInMemory(argument_types); diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 84a9267ffe0..d30f6f6d4df 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -102,6 +102,11 @@ namespace if (argument_types.empty()) throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + const WhichDataType t(argument_types[0]); + if (t.isAggregateFunction()) + throw Exception( + "Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); switch (precision) { diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp b/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp index 8195bd8964c..d0ad55e5017 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp @@ -45,6 +45,11 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c if (argument_types.empty()) throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + const WhichDataType t(argument_types[0]); + if (t.isAggregateFunction()) + throw Exception( + "Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); From df47bdecbc4232390a2e313f38a8494a5bfa50b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 26 May 2021 13:07:29 +0200 Subject: [PATCH 211/652] Add tests for chained unique*State --- ...882_uniqueState_over_uniqueState.reference | 0 .../01882_uniqueState_over_uniqueState.sh | 20 +++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/01882_uniqueState_over_uniqueState.reference create mode 100755 tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh diff --git a/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.reference b/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh b/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh new file mode 100755 index 00000000000..6be05a128d7 --- /dev/null +++ b/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +function check() +{ + $CLICKHOUSE_CLIENT -q "SELECT id, $1(s) FROM ( SELECT number % 10 as id, uniqState(number) as s FROM ( SELECT number FROM system.numbers LIMIT 100 ) GROUP BY number ) GROUP BY id" 2>&1 | grep -v -P '^(Received exception from server|Code: 43)' ||: +} + +stateFunctions=("uniqState" "uniqExactState" "uniqHLL12State" "uniqCombinedState" "uniqUpToState") # "uniqThetaState" not tested because its availability depends on compilation options + +for i1 in "${stateFunctions[@]}" +do + check "$i1" +done From d9bc0266f7afe3e6a7c31f89d2af7de6fcd08581 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 26 May 2021 13:48:33 +0200 Subject: [PATCH 212/652] Add missing error codes to fix style check --- src/AggregateFunctions/AggregateFunctionUniq.cpp | 1 + src/AggregateFunctions/AggregateFunctionUniqCombined.cpp | 1 + src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp | 1 + 3 files changed, 3 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index 2ee1d42aad8..ba4e041a466 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index d30f6f6d4df..e6843a32642 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp b/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp index d0ad55e5017..4a9d006cae9 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } From d17b46d69f69229da391e5c062c5f69bead8039e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 26 May 2021 15:06:13 +0300 Subject: [PATCH 213/652] fix mongo logging --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3f0a995c173..e1b37576a18 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1128,7 +1128,7 @@ class ClickHouseCluster: while time.time() - start < timeout: try: connection.list_database_names() - logging.debug("Connected to Mongo dbs: {}", connection.database_names()) + logging.debug(f"Connected to Mongo dbs: {connection.database_names()}") return except Exception as ex: logging.debug("Can't connect to Mongo " + str(ex)) From 99e3be018ea0a8d1718ceb5bf42153f3df46f023 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 26 May 2021 15:06:31 +0300 Subject: [PATCH 214/652] better --- tests/integration/parallel_skip.json | 5 +++++ .../configs/dictionaries/ssd_complex_key_cache_string.xml | 2 +- tests/integration/test_reload_auxiliary_zookeepers/test.py | 4 ++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 4be0c9a77ae..088e3a3ba82 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -1,4 +1,9 @@ [ + "test_host_ip_change/test.py::test_dns_cache_update", + "test_host_ip_change/test.py::test_ip_change_drop_dns_cache", + "test_host_ip_change/test.py::test_ip_change_update_dns_cache", + "test_host_ip_change/test.py::test_user_access_ip_change[node0]", + "test_host_ip_change/test.py::test_user_access_ip_change[node1]", "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", "test_attach_without_fetching/test.py::test_attach_without_fetching", "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml index 7492a4331cb..def138a1c32 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml @@ -41,7 +41,7 @@ 16777216 131072 1048576 - /etc/clickhouse-server/radars + /var/lib/clickhouse/dictionaries/radars 1 diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index 232964931c4..e01331269d0 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -64,6 +64,8 @@ def test_reload_auxiliary_zookeepers(start_cluster): node.query("SYSTEM RELOAD CONFIG") + time.sleep(5) + node.query( "ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';" ) @@ -81,6 +83,8 @@ def test_reload_auxiliary_zookeepers(start_cluster): """ node.replace_config("/etc/clickhouse-server/conf.d/zookeeper_config.xml", new_config) node.query("SYSTEM RELOAD CONFIG") + time.sleep(5) + with pytest.raises(QueryRuntimeException): node.query( "ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';" From 1151f13fed2390c5f6278af9e666fc2815b39cd1 Mon Sep 17 00:00:00 2001 From: Daria Mozhaeva Date: Wed, 26 May 2021 15:38:54 +0300 Subject: [PATCH 215/652] Edit and translate to Russian --- .../functions/date-time-functions.md | 336 ++++-------------- docs/en/sql-reference/statements/rename.md | 2 +- .../functions/date-time-functions.md | 3 + docs/ru/sql-reference/statements/rename.md | 2 +- 4 files changed, 74 insertions(+), 269 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 69baf64ef55..43ce623e69f 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -23,53 +23,13 @@ SELECT └─────────────────────┴────────────┴────────────┴─────────────────────┘ ``` -## timeZone {#timezone} - -Returns the timezone of the server. - -**Syntax** - -``` sql -timeZone() -``` - -Alias: `timezone`. - -**Returned value** - -- Timezone. - -Type: [String](../../sql-reference/data-types/string.md). - ## toTimeZone {#totimezone} -Converts time or date and time to the specified time zone. The time zone is an attribute of the `Date` and `DateTime` data types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly. - -**Syntax** - -``` sql -toTimezone(value, timezone) -``` - -Alias: `toTimezone`. - -**Arguments** - -- `value` — Time or date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). -- `timezone` — Timezone for the returned value. [String](../../sql-reference/data-types/string.md). - -**Returned value** - -- Date and time. - -Type: [DateTime](../../sql-reference/data-types/datetime.md). - -**Example** - -Query: +Convert time or date and time to the specified time zone. The time zone is an attribute of the Date/DateTime types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly. ```sql -SELECT toDateTime('2019-01-01 00:00:00', 'UTC') AS time_utc, +SELECT + toDateTime('2019-01-01 00:00:00', 'UTC') AS time_utc, toTypeName(time_utc) AS type_utc, toInt32(time_utc) AS int32utc, toTimeZone(time_utc, 'Asia/Yekaterinburg') AS time_yekat, @@ -80,7 +40,6 @@ SELECT toDateTime('2019-01-01 00:00:00', 'UTC') AS time_utc, toInt32(time_samoa) AS int32samoa FORMAT Vertical; ``` -Result: ```text Row 1: @@ -98,138 +57,44 @@ int32samoa: 1546300800 `toTimeZone(time_utc, 'Asia/Yekaterinburg')` changes the `DateTime('UTC')` type to `DateTime('Asia/Yekaterinburg')`. The value (Unixtimestamp) 1546300800 stays the same, but the string representation (the result of the toString() function) changes from `time_utc: 2019-01-01 00:00:00` to `time_yekat: 2019-01-01 05:00:00`. -## timeZoneOf {#timezoneof} - -Returns the timezone name of [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md) data types. - -**Syntax** - -``` sql -timeZoneOf(value) -``` - -Alias: `timezoneOf`. - -**Arguments** - -- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md). - -**Returned value** - -- Timezone name. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: -``` sql -SELECT timezoneOf(now()); -``` - -Result: -``` text -┌─timezoneOf(now())─┐ -│ Etc/UTC │ -└───────────────────┘ -``` - -## timeZoneOffset {#timezoneoffset} - -Returns a timezone offset in seconds from [UTC](https://en.wikipedia.org/wiki/Coordinated_Universal_Time). The function takes into account [daylight saving time](https://en.wikipedia.org/wiki/Daylight_saving_time) and historical timezone changes at the specified date and time. -[IANA timezone database](https://www.iana.org/time-zones) is used to calculate the offset. - -**Syntax** - -``` sql -timeZoneOffset(value) -``` - -Alias: `timezoneOffset`. - -**Arguments** - -- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md). - -**Returned value** - -- Offset from UTC in seconds. - -Type: [Int32](../../sql-reference/data-types/int-uint.md). - -**Example** - -Query: - -``` sql -SELECT toDateTime('2021-04-21 10:20:30', 'America/New_York') AS Time, toTypeName(Time) AS Type, - timeZoneOffset(Time) AS Offset_in_seconds, (Offset_in_seconds / 3600) AS Offset_in_hours; -``` - -Result: - -``` text -┌────────────────Time─┬─Type─────────────────────────┬─Offset_in_seconds─┬─Offset_in_hours─┐ -│ 2021-04-21 10:20:30 │ DateTime('America/New_York') │ -14400 │ -4 │ -└─────────────────────┴──────────────────────────────┴───────────────────┴─────────────────┘ -``` - ## toYear {#toyear} Converts a date or date with time to a UInt16 number containing the year number (AD). -Alias: `YEAR`. - ## toQuarter {#toquarter} Converts a date or date with time to a UInt8 number containing the quarter number. -Alias: `QUARTER`. - ## toMonth {#tomonth} Converts a date or date with time to a UInt8 number containing the month number (1-12). -Alias: `MONTH`. - ## toDayOfYear {#todayofyear} Converts a date or date with time to a UInt16 number containing the number of the day of the year (1-366). -Alias: `DAYOFYEAR`. - ## toDayOfMonth {#todayofmonth} Converts a date or date with time to a UInt8 number containing the number of the day of the month (1-31). -Aliases: `DAYOFMONTH`, `DAY`. - ## toDayOfWeek {#todayofweek} Converts a date or date with time to a UInt8 number containing the number of the day of the week (Monday is 1, and Sunday is 7). -Alias: `DAYOFWEEK`. - ## toHour {#tohour} Converts a date with time to a UInt8 number containing the number of the hour in 24-hour time (0-23). This function assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true – even in Moscow the clocks were twice changed at a different time). -Alias: `HOUR`. - ## toMinute {#tominute} Converts a date with time to a UInt8 number containing the number of the minute of the hour (0-59). -Alias: `MINUTE`. - ## toSecond {#tosecond} Converts a date with time to a UInt8 number containing the number of the second in the minute (0-59). Leap seconds are not accounted for. -Alias: `SECOND`. - ## toUnixTimestamp {#to-unix-timestamp} For DateTime argument: converts value to the number with type UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time). @@ -265,7 +130,7 @@ Result: ``` !!! attention "Attention" - The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of normal range (years 1970 - 2105) will give incorrect result. + The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of the normal range (years 1970 - 2105) will give an incorrect result. ## toStartOfYear {#tostartofyear} @@ -324,7 +189,7 @@ Truncates sub-seconds. toStartOfSecond(value[, timezone]) ``` -**Arguments** +**Parameters** - `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). - `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md). @@ -466,7 +331,7 @@ For mode values with a meaning of “contains January 1”, the week contains Ja toWeek(date, [, mode][, Timezone]) ``` -**Arguments** +**Parameters** - `date` – Date or DateTime. - `mode` – Optional parameter, Range of values is \[0,9\], default is 0. @@ -508,15 +373,15 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d Truncates date and time data to the specified part of date. -**Syntax** +**Syntax** ``` sql date_trunc(unit, value[, timezone]) ``` -Alias: `dateTrunc`. +Alias: `dateTrunc`. -**Arguments** +**Parameters** - `unit` — The type of interval to truncate the result. [String Literal](../syntax.md#syntax-string-literal). Possible values: @@ -569,55 +434,41 @@ Result: └─────────────────────┴────────────────────────────────────────────┘ ``` -**See Also** +**See also** - [toStartOfInterval](#tostartofintervaltime-or-data-interval-x-unit-time-zone) ## date\_add {#date_add} -Adds the time interval or date interval to the provided date or date with time. +Adds specified date/time interval to the provided date. -**Syntax** +**Syntax** ``` sql date_add(unit, value, date) ``` -Aliases: `dateAdd`, `DATE_ADD`. +Aliases: `dateAdd`, `DATE_ADD`. -**Arguments** +**Parameters** - `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). - Possible values: - - `second` - - `minute` - - `hour` - - `day` - - `week` - - `month` - - `quarter` - - `year` + Supported values: second, minute, hour, day, week, month, quarter, year. +- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md) +- `date` — [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). -- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md). -- `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Returned value** -Date or date with time obtained by adding `value`, expressed in `unit`, to `date`. - -Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +Returns Date or DateTime with `value` expressed in `unit` added to `date`. **Example** -Query: - ```sql -SELECT date_add(YEAR, 3, toDate('2018-01-01')); +select date_add(YEAR, 3, toDate('2018-01-01')); ``` -Result: - ```text ┌─plus(toDate('2018-01-01'), toIntervalYear(3))─┐ │ 2021-01-01 │ @@ -626,7 +477,7 @@ Result: ## date\_diff {#date_diff} -Returns the difference between two dates or dates with time values. +Returns the difference between two Date or DateTime values. **Syntax** @@ -634,33 +485,25 @@ Returns the difference between two dates or dates with time values. date_diff('unit', startdate, enddate, [timezone]) ``` -Aliases: `dateDiff`, `DATE_DIFF`. +Aliases: `dateDiff`, `DATE_DIFF`. -**Arguments** +**Parameters** -- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). - Possible values: +- `unit` — The type of interval for result [String](../../sql-reference/data-types/string.md). - - `second` - - `minute` - - `hour` - - `day` - - `week` - - `month` - - `quarter` - - `year` + Supported values: second, minute, hour, day, week, month, quarter, year. - `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). - `enddate` — The second time value to subtract from (the minuend). [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). -- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. [String](../../sql-reference/data-types/string.md). +- `timezone` — Optional parameter. If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. **Returned value** Difference between `enddate` and `startdate` expressed in `unit`. -Type: [Int](../../sql-reference/data-types/int-uint.md). +Type: `int`. **Example** @@ -680,7 +523,7 @@ Result: ## date\_sub {#date_sub} -Subtracts the time interval or date interval from the provided date or date with time. +Subtracts a time/date interval from the provided date. **Syntax** @@ -688,30 +531,19 @@ Subtracts the time interval or date interval from the provided date or date with date_sub(unit, value, date) ``` -Aliases: `dateSub`, `DATE_SUB`. +Aliases: `dateSub`, `DATE_SUB`. -**Arguments** +**Parameters** - `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md). - Possible values: - - `second` - - `minute` - - `hour` - - `day` - - `week` - - `month` - - `quarter` - - `year` - -- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md). -- `date` — The date or date with time from which `value` is subtracted. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + Supported values: second, minute, hour, day, week, month, quarter, year. +- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md) +- `date` — [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md) to subtract value from. **Returned value** -Date or date with time obtained by subtracting `value`, expressed in `unit`, from `date`. - -Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +Returns Date or DateTime with `value` expressed in `unit` subtracted from `date`. **Example** @@ -733,46 +565,32 @@ Result: Adds the specified time value with the provided date or date time value. -**Syntax** +**Syntax** ``` sql timestamp_add(date, INTERVAL value unit) ``` -Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. +Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. -**Arguments** - -- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). -- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md). +**Parameters** + +- `date` — Date or Date with time - [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md) - `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). - Possible values: - - `second` - - `minute` - - `hour` - - `day` - - `week` - - `month` - - `quarter` - - `year` + Supported values: second, minute, hour, day, week, month, quarter, year. **Returned value** -Date or date with time with the specified `value` expressed in `unit` added to `date`. - -Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). - +Returns Date or DateTime with the specified `value` expressed in `unit` added to `date`. + **Example** -Query: - ```sql select timestamp_add(toDate('2018-01-01'), INTERVAL 3 MONTH); ``` -Result: - ```text ┌─plus(toDate('2018-01-01'), toIntervalMonth(3))─┐ │ 2018-04-01 │ @@ -781,66 +599,51 @@ Result: ## timestamp\_sub {#timestamp_sub} -Subtracts the time interval from the provided date or date with time. +Returns the difference between two dates in the specified unit. -**Syntax** +**Syntax** ``` sql timestamp_sub(unit, value, date) ``` -Aliases: `timeStampSub`, `TIMESTAMP_SUB`. +Aliases: `timeStampSub`, `TIMESTAMP_SUB`. -**Arguments** +**Parameters** -- `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md). - Possible values: +- `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). - - `second` - - `minute` - - `hour` - - `day` - - `week` - - `month` - - `quarter` - - `year` - -- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md). -- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + Supported values: second, minute, hour, day, week, month, quarter, year. +- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md). +- `date`- [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Returned value** -Date or date with time obtained by subtracting `value`, expressed in `unit`, from `date`. - -Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +Difference between `date` and the specified `value` expressed in `unit`. **Example** -Query: - ```sql select timestamp_sub(MONTH, 5, toDateTime('2018-12-18 01:02:03')); ``` -Result: - ```text ┌─minus(toDateTime('2018-12-18 01:02:03'), toIntervalMonth(5))─┐ │ 2018-07-18 01:02:03 │ └──────────────────────────────────────────────────────────────┘ ``` - + ## now {#now} -Returns the current date and time. +Returns the current date and time. -**Syntax** +**Syntax** ``` sql now([timezone]) ``` -**Arguments** +**Parameters** - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md). @@ -953,7 +756,7 @@ This is necessary for searching for pageviews in the corresponding session. ## formatDateTime {#formatdatetime} -Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. +Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. **Syntax** @@ -973,7 +776,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %C | year divided by 100 and truncated to integer (00-99) | 20 | | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | -| %e | day of the month, space-padded ( 1-31) |   2 | +| %e | day of the month, space-padded ( 1-31) | 2 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | | %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | | %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | @@ -1012,32 +815,31 @@ Result: └────────────────────────────────────────────┘ ``` +[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) + ## FROM\_UNIXTIME {#fromunixfime} -Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. +When there is only single argument of integer type, it act in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md). +type. -**Example:** - -Query: +For example: ```sql -SELECT FROM_UNIXTIME(423543535); +SELECT FROM_UNIXTIME(423543535) ``` -Result: - ```text ┌─FROM_UNIXTIME(423543535)─┐ │ 1983-06-04 10:58:55 │ └──────────────────────────┘ ``` -When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. +When there are two arguments, first is integer or DateTime, second is constant format string, it act in the same way as `formatDateTime` and return `String` type. For example: ```sql -SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; +SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime ``` ```text @@ -1056,7 +858,7 @@ Converts a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Prolepti toModifiedJulianDay(date) ``` -**Arguments** +**Parameters** - `date` — Date in text form. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). @@ -1092,7 +894,7 @@ Similar to [toModifiedJulianDay()](#tomodifiedjulianday), but instead of raising toModifiedJulianDayOrNull(date) ``` -**Arguments** +**Parameters** - `date` — Date in text form. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). @@ -1128,7 +930,7 @@ Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Varian fromModifiedJulianDay(day) ``` -**Arguments** +**Parameters** - `day` — Modified Julian Day number. [Any integral types](../../sql-reference/data-types/int-uint.md). @@ -1164,7 +966,7 @@ Similar to [fromModifiedJulianDayOrNull()](#frommodifiedjuliandayornull), but in fromModifiedJulianDayOrNull(day) ``` -**Arguments** +**Parameters** - `day` — Modified Julian Day number. [Any integral types](../../sql-reference/data-types/int-uint.md). diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index a9dda6ed3b2..6e98b19d9b8 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -6,7 +6,7 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} ## RENAME DATABASE {#misc_operations-rename_database} -Renames database, support only for Atomic database engine +Renames database, it is supported only for Atomic database engine. ``` RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index b442a782100..de71384253b 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -264,6 +264,9 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; └────────────────┘ ``` +!!! attention "Attention" + Возвращаемый тип функций `toStartOf*`, который описан ниже, это `Date` или `DateTime`. Несмотря на то, что эти функции могут принимать `DateTime64` в качестве аргумента, если передача `DateTime64` выходит за пределы нормального диапазона (1970 - 2105 гг.), то это даст неверный результат. + ## toStartOfYear {#tostartofyear} Округляет дату или дату-с-временем вниз до первого дня года. diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md index 192426dbafa..b78505ce9c4 100644 --- a/docs/ru/sql-reference/statements/rename.md +++ b/docs/ru/sql-reference/statements/rename.md @@ -6,7 +6,7 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} ## RENAME DATABASE {#misc_operations-rename_database} -Переименование базы данных +Переименовывает базу данных, поддерживается только для движка базы данных Atomic. ``` RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] From 6bc0167970a4cbf8bef58ead5e325fae89a325e9 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 26 May 2021 17:35:05 +0300 Subject: [PATCH 216/652] Add test --- ...01881_total_bytes_storage_buffer.reference | 5 ++++ .../01881_total_bytes_storage_buffer.sql | 23 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference create mode 100644 tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference new file mode 100644 index 00000000000..eaa9e9259df --- /dev/null +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference @@ -0,0 +1,5 @@ +0 +8192 +32 +8192 +32 diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql new file mode 100644 index 00000000000..d962a6db456 --- /dev/null +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS test_buffer_table; + +CREATE TABLE test_buffer_table +( + `a` Int64 +) +ENGINE = Buffer('', '', 1, 1000000, 1000000, 1000000, 1000000, 1000000, 1000000); + +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +INSERT INTO test_buffer_table SELECT number FROM numbers(1000); +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +OPTIMIZE TABLE test_buffer_table; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +INSERT INTO test_buffer_table SELECT number FROM numbers(1000); +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +OPTIMIZE TABLE test_buffer_table; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; + +DROP TABLE test_buffer_table; From 4f07efe13ce3520c574474938602f57ea91e0a81 Mon Sep 17 00:00:00 2001 From: Daria Mozhaeva Date: Wed, 26 May 2021 18:36:05 +0300 Subject: [PATCH 217/652] v2 --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index de71384253b..214f30409a8 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -265,7 +265,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; ``` !!! attention "Attention" - Возвращаемый тип функций `toStartOf*`, который описан ниже, это `Date` или `DateTime`. Несмотря на то, что эти функции могут принимать `DateTime64` в качестве аргумента, если передача `DateTime64` выходит за пределы нормального диапазона (1970 - 2105 гг.), то это даст неверный результат. + `Date` или `DateTime` это возвращаемый тип функций `toStartOf*`, который описан ниже. Несмотря на то, что эти функции могут принимать `DateTime64` в качестве аргумента, если передача `DateTime64` выходит за пределы нормального диапазона (с 1970 по 2105 год), то это даст неверный результат. ## toStartOfYear {#tostartofyear} From fee5351387ec8f400e7b07e3093c0a59d4348b1c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:10 +0300 Subject: [PATCH 218/652] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 3e3d2a10984..93bb5b73947 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -42,7 +42,7 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_fu Query: ``` sql -SELECT * FROM dictionary('table_function_dictionary_test_dictionary'); +SELECT * FROM dictionary('new_dictionary'); ``` Result: From 8877c3b5c56d5ba26b450961e7e95088bb40bab3 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:20 +0300 Subject: [PATCH 219/652] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 93bb5b73947..4082caf78cd 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -36,7 +36,7 @@ Create a dictionary: ``` sql CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) LAYOUT(DIRECT()); +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); ``` Query: From c8b9a464d71936c1ea5c0a910be95ead1528b3b6 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:30 +0300 Subject: [PATCH 220/652] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index 4082caf78cd..ae9d9af03c0 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -23,7 +23,7 @@ A ClickHouse table. **Example** -Input table: +Input table `dictionary_source_table`: ``` text ┌─id─┬─value─┐ From 2a1c5acd31b724ae457a698ce8bc4e264d0bd639 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:12:40 +0300 Subject: [PATCH 221/652] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index ae9d9af03c0..fca680e5c00 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -5,7 +5,7 @@ toc_title: dictionary function # dictionary {#dictionary-function} -Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. +Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. Works the same way as [Dictionary](../../engines/table-engines/special/dictionary.md) engine. **Syntax** From 620f2755231fff1f1b29b190cbd16a040cc9c470 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 26 May 2021 20:13:05 +0300 Subject: [PATCH 222/652] Update docs/en/sql-reference/table-functions/dictionary.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md index fca680e5c00..675fcb5bfdd 100644 --- a/docs/en/sql-reference/table-functions/dictionary.md +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -35,7 +35,7 @@ Input table `dictionary_source_table`: Create a dictionary: ``` sql -CREATE DICTIONARY table_function_dictionary_test_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); ``` From f8ccd47d75ff02796629718033221e87547acb46 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Wed, 26 May 2021 20:23:54 +0300 Subject: [PATCH 223/652] Added catch to YAML_fuzzer + tested YAMLParser, no other errors found --- src/Parsers/examples/YAML_fuzzer.cpp | 9 ++++++++- tests/fuzz/YAML_fuzzer.options | 2 -- 2 files changed, 8 insertions(+), 3 deletions(-) delete mode 100644 tests/fuzz/YAML_fuzzer.options diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Parsers/examples/YAML_fuzzer.cpp index cf47633cb09..c18b9ffae70 100644 --- a/src/Parsers/examples/YAML_fuzzer.cpp +++ b/src/Parsers/examples/YAML_fuzzer.cpp @@ -26,7 +26,14 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) temp_file << input; temp_file.close(); - DB::YAMLParser::parse(cur_file); + try + { + DB::YAMLParser::parse(cur_file); + } + catch (const DB::Exception&) + { + std::cout << "YAMLParser exception from bad file, etc. OK\n"; + } remove(cur_file.c_str()); return 0; diff --git a/tests/fuzz/YAML_fuzzer.options b/tests/fuzz/YAML_fuzzer.options deleted file mode 100644 index 1a0d84a96c4..00000000000 --- a/tests/fuzz/YAML_fuzzer.options +++ /dev/null @@ -1,2 +0,0 @@ -[libfuzzer] -dict = YAML_fuzzer.dict From 991ae97fb6797d5ab3f1a1e0bfb887fcba2e866f Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 26 May 2021 20:53:51 +0300 Subject: [PATCH 224/652] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../table-engines/special/dictionary.md | 4 +- .../external-dicts-dict-sources.md | 4 ++ .../table-engines/special/dictionary.md | 3 + .../external-dicts-dict-sources.md | 4 ++ .../table-functions/dictionary.md | 59 +++++++++++++++++++ 5 files changed, 73 insertions(+), 1 deletion(-) create mode 100644 docs/ru/sql-reference/table-functions/dictionary.md diff --git a/docs/en/engines/table-engines/special/dictionary.md b/docs/en/engines/table-engines/special/dictionary.md index a6d6f296673..e143f3a8ee1 100644 --- a/docs/en/engines/table-engines/special/dictionary.md +++ b/docs/en/engines/table-engines/special/dictionary.md @@ -94,4 +94,6 @@ select * from products limit 1; └───────────────┴─────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/operations/table_engines/dictionary/) +**See Also** + +- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index dc0b6e17198..3b9a28403f7 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -98,6 +98,10 @@ Setting fields: When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in `user_files` directory, to prevent DB users accessing arbitrary file on ClickHouse node. +**See Also** + +- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) + ## Executable File {#dicts-external_dicts_dict_sources-executable} Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts executable file and treats its output as dictionary data. diff --git a/docs/ru/engines/table-engines/special/dictionary.md b/docs/ru/engines/table-engines/special/dictionary.md index 243fd5395c0..df47ade5a42 100644 --- a/docs/ru/engines/table-engines/special/dictionary.md +++ b/docs/ru/engines/table-engines/special/dictionary.md @@ -90,3 +90,6 @@ select * from products limit 1; └───────────────┴─────────────────┘ ``` +**Смотрите также** + +- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index a7999470330..a9f3857f183 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -97,6 +97,10 @@ SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) Если словарь с источником `FILE` создается с помощью DDL-команды (`CREATE DICTIONARY ...`), источник словаря должен быть расположен в каталоге `user_files`. Иначе пользователи базы данных будут иметь доступ к произвольному файлу на узле ClickHouse. +**Смотрите также** + +- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) + ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} Работа с исполняемым файлом зависит от [размещения словаря в памяти](external-dicts-dict-layout.md). Если тип размещения словаря `cache` и `complex_key_cache`, то ClickHouse запрашивает необходимые ключи, отправляя запрос в `STDIN` исполняемого файла. diff --git a/docs/ru/sql-reference/table-functions/dictionary.md b/docs/ru/sql-reference/table-functions/dictionary.md new file mode 100644 index 00000000000..d4909bf5d9f --- /dev/null +++ b/docs/ru/sql-reference/table-functions/dictionary.md @@ -0,0 +1,59 @@ +--- +toc_priority: 54 +toc_title: dictionary +--- + +# dictionary {#dictionary-function} + +Отображает данные [словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) как таблицу ClickHouse. Работает аналогично движку [Dictionary](../../engines/table-engines/special/dictionary.md). + +**Синтаксис** + +``` sql +dictionary('dict') +``` + +**Аргументы** + +- `dict` — имя словаря. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +Таблица ClickHouse. + +**Пример** + +Входная таблица `dictionary_source_table`: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +Создаем словарь: + +``` sql +CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); +``` + +Запрос: + +``` sql +SELECT * FROM dictionary('new_dictionary'); +``` + +Результат: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +**Смотрите также** + +- [Движок Dictionary](../../engines/table-engines/special/dictionary.md#dictionary) From a51a6ea0b78f305fbcc33703505256ed2f5b0369 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 26 May 2021 21:14:43 +0300 Subject: [PATCH 225/652] Part 2. --- .../QueryPlan/ReadFromMergeTree.cpp | 900 +++++++++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 34 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 265 ++++++ .../MergeTree/MergeTreeDataSelectExecutor.h | 16 +- 4 files changed, 1157 insertions(+), 58 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 771f95cc8fd..67377a54c34 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -2,15 +2,34 @@ #include #include #include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include +#include +#include #include #include +namespace ProfileEvents +{ + extern const Event SelectedParts; + extern const Event SelectedRanges; + extern const Event SelectedMarks; +} + namespace DB { @@ -19,6 +38,55 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; } +namespace +{ + +/// Marks are placed whenever threshold on rows or bytes is met. +/// So we have to return the number of marks on whatever estimate is higher - by rows or by bytes. +size_t roundRowsOrBytesToMarks( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity) +{ + size_t res = (rows_setting + rows_granularity - 1) / rows_granularity; + + if (bytes_granularity == 0) + return res; + else + return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); +} +/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks +size_t minMarksForConcurrentRead( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity, + size_t max_marks) +{ + size_t marks = 1; + + if (rows_setting + rows_granularity <= rows_setting) /// overflow + marks = max_marks; + else if (rows_setting) + marks = (rows_setting + rows_granularity - 1) / rows_granularity; + + if (bytes_granularity == 0) + return marks; + else + { + /// Overflow + if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow + return max_marks; + if (bytes_setting) + return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); + else + return marks; + } +} + +} + ReadFromMergeTree::ReadFromMergeTree( SelectQueryInfo query_info_, const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, @@ -26,15 +94,14 @@ ReadFromMergeTree::ReadFromMergeTree( const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, - String query_id_, + StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, - size_t num_streams_, - ReadType read_type_) + size_t num_streams_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( metadata_snapshot_->getSampleBlockForColumns(real_column_names_, storage_.getVirtuals(), storage_.getStorageID()), prewhere_info_, @@ -46,35 +113,35 @@ ReadFromMergeTree::ReadFromMergeTree( , data(data_) , storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) - , query_id(std::move(query_id_)) + , metadata_snapshot_base(std::move(metadata_snapshot_base_)) , real_column_names(std::move(real_column_names_)) , parts(std::move(parts_)) - //, index_stats(std::move(index_stats_)) , prewhere_info(std::move(prewhere_info_)) , virt_column_names(std::move(virt_column_names_)) , settings(std::move(settings_)) , num_streams(num_streams_) - , read_type(read_type_) { } -Pipe ReadFromMergeTree::readFromPool() +Pipe ReadFromMergeTree::readFromPool( + RangesInDataParts parts_with_range, Names required_columns, + size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { Pipes pipes; size_t sum_marks = 0; size_t total_rows = 0; - for (const auto & part : parts) + for (const auto & part : parts_with_range) { sum_marks += part.getMarksCount(); total_rows += part.getRowsCount(); } auto pool = std::make_shared( - num_streams, + used_max_streams, sum_marks, - settings.min_marks_for_concurrent_read, - std::move(parts), + min_marks_for_concurrent_read, + std::move(parts_with_range), storage, metadata_snapshot, prewhere_info, @@ -85,14 +152,14 @@ Pipe ReadFromMergeTree::readFromPool() false); auto * logger = &Poco::Logger::get(storage.getLogName() + " (SelectExecutor)"); - LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, num_streams); + LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, used_max_streams); - for (size_t i = 0; i < num_streams; ++i) + for (size_t i = 0; i < used_max_streams; ++i) { auto source = std::make_shared( - i, pool, settings.min_marks_for_concurrent_read, settings.max_block_size, + i, pool, min_marks_for_concurrent_read, settings.max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, - storage, metadata_snapshot, settings.use_uncompressed_cache, + storage, metadata_snapshot, use_uncompressed_cache, prewhere_info, settings.reader_settings, virt_column_names); if (i == 0) @@ -108,22 +175,22 @@ Pipe ReadFromMergeTree::readFromPool() } template -ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part) +ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache) { return std::make_shared( storage, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, settings.use_uncompressed_cache, + settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, true, settings.reader_settings, virt_column_names, part.part_index_in_query); } -Pipe ReadFromMergeTree::readInOrder() +Pipe ReadFromMergeTree::readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache) { Pipes pipes; - for (const auto & part : parts) + for (const auto & part : parts_with_range) { auto source = read_type == ReadType::InReverseOrder - ? createSource(part) - : createSource(part); + ? createSource(part, required_columns, use_uncompressed_cache) + : createSource(part, required_columns, use_uncompressed_cache); pipes.emplace_back(std::move(source)); } @@ -141,12 +208,14 @@ Pipe ReadFromMergeTree::readInOrder() return pipe; } -Pipe ReadFromMergeTree::read() +Pipe ReadFromMergeTree::read( + RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, + size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { if (read_type == ReadType::Default && num_streams > 1) - return readFromPool(); + return readFromPool(parts_with_range, required_columns, used_max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); - auto pipe = readInOrder(); + auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache); /// Use ConcatProcessor to concat sources together. /// It is needed to read in parts order (and so in PK order) if single thread is used. @@ -276,8 +345,629 @@ static void filterPartsByPartition( } } +Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( + RangesInDataParts && parts_with_ranges, + const Names & column_names) +{ + const auto & q_settings = context->getSettingsRef(); + + /// Count marks for each part. + std::vector sum_marks_in_parts(parts_with_ranges.size()); + size_t sum_marks = 0; + size_t total_rows = 0; + + const auto data_settings = data.getSettings(); + size_t adaptive_parts = 0; + for (size_t i = 0; i < parts_with_ranges.size(); ++i) + { + total_rows += parts_with_ranges[i].getRowsCount(); + sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); + sum_marks += sum_marks_in_parts[i]; + + if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + size_t index_granularity_bytes = 0; + if (adaptive_parts > parts_with_ranges.size() / 2) + index_granularity_bytes = data_settings->index_granularity_bytes; + + const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + q_settings.merge_tree_max_rows_to_use_cache, + q_settings.merge_tree_max_bytes_to_use_cache, + data_settings->index_granularity, + index_granularity_bytes); + + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + q_settings.merge_tree_min_rows_for_concurrent_read, + q_settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks); + + bool use_uncompressed_cache = q_settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + + if (0 == sum_marks) + return {}; + + size_t used_num_streams = num_streams; + if (used_num_streams > 1) + { + /// Reduce the number of num_streams if the data is small. + if (sum_marks < used_num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < used_num_streams) + used_num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); + } + + return read(std::move(parts_with_ranges), column_names, ReadType::Default, + used_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); +} + +static ActionsDAGPtr createProjection(const Block & header) +{ + auto projection = std::make_shared(header.getNamesAndTypesList()); + projection->removeUnusedActions(header.getNames()); + projection->projectInput(); + return projection; +} + +Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( + RangesInDataParts && parts_with_ranges, + const Names & column_names, + const ActionsDAGPtr & sorting_key_prefix_expr, + ActionsDAGPtr & out_projection, + const InputOrderInfoPtr & input_order_info) +{ + const auto & q_settings = context->getSettingsRef(); + size_t sum_marks = 0; + size_t adaptive_parts = 0; + std::vector sum_marks_in_parts(parts_with_ranges.size()); + const auto data_settings = data.getSettings(); + + for (size_t i = 0; i < parts_with_ranges.size(); ++i) + { + sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); + sum_marks += sum_marks_in_parts[i]; + + if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + size_t index_granularity_bytes = 0; + if (adaptive_parts > parts_with_ranges.size() / 2) + index_granularity_bytes = data_settings->index_granularity_bytes; + + const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + q_settings.merge_tree_max_rows_to_use_cache, + q_settings.merge_tree_max_bytes_to_use_cache, + data_settings->index_granularity, + index_granularity_bytes); + + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + q_settings.merge_tree_min_rows_for_concurrent_read, + q_settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks); + + bool use_uncompressed_cache = settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + + Pipes res; + + if (sum_marks == 0) + return {}; + + /// Let's split ranges to avoid reading much data. + auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = settings.max_block_size] + (const auto & ranges, int direction) + { + MarkRanges new_ranges; + const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; + size_t marks_in_range = 1; + + if (direction == 1) + { + /// Split first few ranges to avoid reading much data. + bool split = false; + for (auto range : ranges) + { + while (!split && range.begin + marks_in_range < range.end) + { + new_ranges.emplace_back(range.begin, range.begin + marks_in_range); + range.begin += marks_in_range; + marks_in_range *= 2; + + if (marks_in_range > max_marks_in_range) + split = true; + } + new_ranges.emplace_back(range.begin, range.end); + } + } + else + { + /// Split all ranges to avoid reading much data, because we have to + /// store whole range in memory to reverse it. + for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) + { + auto range = *it; + while (range.begin + marks_in_range < range.end) + { + new_ranges.emplace_front(range.end - marks_in_range, range.end); + range.end -= marks_in_range; + marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); + } + new_ranges.emplace_front(range.begin, range.end); + } + } + + return new_ranges; + }; + + const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; + bool need_preliminary_merge = (parts_with_ranges.size() > q_settings.read_in_order_two_level_merge_threshold); + + Pipes pipes; + + for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i) + { + size_t need_marks = min_marks_per_stream; + RangesInDataParts new_parts; + + /// Loop over parts. + /// We will iteratively take part or some subrange of a part from the back + /// and assign a stream to read from it. + while (need_marks > 0 && !parts_with_ranges.empty()) + { + RangesInDataPart part = parts_with_ranges.back(); + parts_with_ranges.pop_back(); + + size_t & marks_in_part = sum_marks_in_parts.back(); + + /// We will not take too few rows from a part. + if (marks_in_part >= min_marks_for_concurrent_read && + need_marks < min_marks_for_concurrent_read) + need_marks = min_marks_for_concurrent_read; + + /// Do not leave too few rows in the part. + if (marks_in_part > need_marks && + marks_in_part - need_marks < min_marks_for_concurrent_read) + need_marks = marks_in_part; + + MarkRanges ranges_to_get_from_part; + + /// We take the whole part if it is small enough. + if (marks_in_part <= need_marks) + { + ranges_to_get_from_part = part.ranges; + + need_marks -= marks_in_part; + sum_marks_in_parts.pop_back(); + } + else + { + /// Loop through ranges in part. Take enough ranges to cover "need_marks". + while (need_marks > 0) + { + if (part.ranges.empty()) + throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); + + MarkRange & range = part.ranges.front(); + + const size_t marks_in_range = range.end - range.begin; + const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + + ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); + range.begin += marks_to_get_from_range; + marks_in_part -= marks_to_get_from_range; + need_marks -= marks_to_get_from_range; + if (range.begin == range.end) + part.ranges.pop_front(); + } + parts_with_ranges.emplace_back(part); + } + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); + new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); + } + + auto read_type = input_order_info->direction == 1 + ? ReadFromMergeTree::ReadType::InOrder + : ReadFromMergeTree::ReadType::InReverseOrder; + + pipes.emplace_back(read(std::move(new_parts), column_names, read_type, + num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + } + + if (need_preliminary_merge) + { + SortDescription sort_description; + for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], + input_order_info->direction, 1); + + auto sorting_key_expr = std::make_shared(sorting_key_prefix_expr); + + for (auto & pipe : pipes) + { + /// Drop temporary columns, added by 'sorting_key_prefix_expr' + out_projection = createProjection(pipe.getHeader()); + + pipe.addSimpleTransform([sorting_key_expr](const Block & header) + { + return std::make_shared(header, sorting_key_expr); + }); + + if (pipe.numOutputPorts() > 1) + { + + auto transform = std::make_shared( + pipe.getHeader(), + pipe.numOutputPorts(), + sort_description, + settings.max_block_size); + + pipe.addTransform(std::move(transform)); + } + } + } + + return Pipe::unitePipes(std::move(pipes)); +} + +static void addMergingFinal( + Pipe & pipe, + size_t num_output_streams, + const SortDescription & sort_description, + MergeTreeData::MergingParams merging_params, + Names partition_key_columns, + size_t max_block_size) +{ + const auto & header = pipe.getHeader(); + size_t num_outputs = pipe.numOutputPorts(); + + auto get_merging_processor = [&]() -> MergingTransformPtr + { + switch (merging_params.mode) + { + case MergeTreeData::MergingParams::Ordinary: + { + return std::make_shared(header, num_outputs, + sort_description, max_block_size); + } + + case MergeTreeData::MergingParams::Collapsing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.sign_column, true, max_block_size); + + case MergeTreeData::MergingParams::Summing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); + + case MergeTreeData::MergingParams::Aggregating: + return std::make_shared(header, num_outputs, + sort_description, max_block_size); + + case MergeTreeData::MergingParams::Replacing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.version_column, max_block_size); + + case MergeTreeData::MergingParams::VersionedCollapsing: + return std::make_shared(header, num_outputs, + sort_description, merging_params.sign_column, max_block_size); + + case MergeTreeData::MergingParams::Graphite: + throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); + } + + __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) + { + if (!desc.column_name.empty()) + key_columns.push_back(header.getPositionByName(desc.column_name)); + else + key_columns.emplace_back(desc.column_number); + } + + pipe.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, num_output_streams, key_columns); + }); + + pipe.transform([&](OutputPortRawPtrs ports) + { + Processors transforms; + std::vector output_ports; + transforms.reserve(ports.size() + num_output_streams); + output_ports.reserve(ports.size()); + + for (auto & port : ports) + { + auto copier = std::make_shared(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 ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( + RangesInDataParts && parts_with_range, + const Names & column_names, + ActionsDAGPtr & out_projection) +{ + const auto & q_settings = context->getSettingsRef(); + const auto data_settings = data.getSettings(); + size_t sum_marks = 0; + size_t adaptive_parts = 0; + for (const auto & part : parts_with_range) + { + for (const auto & range : part.ranges) + sum_marks += range.end - range.begin; + + if (part.data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + size_t index_granularity_bytes = 0; + if (adaptive_parts >= parts_with_range.size() / 2) + index_granularity_bytes = data_settings->index_granularity_bytes; + + const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + q_settings.merge_tree_max_rows_to_use_cache, + q_settings.merge_tree_max_bytes_to_use_cache, + data_settings->index_granularity, + index_granularity_bytes); + + bool use_uncompressed_cache = settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + + size_t used_num_streams = num_streams; + if (used_num_streams > q_settings.max_final_threads) + used_num_streams = q_settings.max_final_threads; + + /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. + /// We have all parts in parts vector, where parts with same partition are nearby. + /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), + /// then we will create a pipe for each partition that will run selecting processor and merging processor + /// for the parts with this partition. In the end we will unite all the pipes. + std::vector parts_to_merge_ranges; + auto it = parts_with_range.begin(); + parts_to_merge_ranges.push_back(it); + + if (q_settings.do_not_merge_across_partitions_select_final) + { + while (it != parts_with_range.end()) + { + it = std::find_if( + it, parts_with_range.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); + parts_to_merge_ranges.push_back(it); + } + /// We divide threads for each partition equally. But we will create at least the number of partitions threads. + /// (So, the total number of threads could be more than initial num_streams. + used_num_streams /= (parts_to_merge_ranges.size() - 1); + } + else + { + /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. + parts_to_merge_ranges.push_back(parts_with_range.end()); + } + + Pipes partition_pipes; + + /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 + /// we will store lonely parts with level > 0 to use parallel select on them. + std::vector lonely_parts; + size_t total_rows_in_lonely_parts = 0; + size_t sum_marks_in_lonely_parts = 0; + + for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) + { + Pipe pipe; + + { + RangesInDataParts new_parts; + + /// 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 and if num_streams > 1 we + /// can use parallel select on such parts. We save such parts in one vector and then use + /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. + if (used_num_streams > 1 && q_settings.do_not_merge_across_partitions_select_final && + 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) + { + total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); + sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); + lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); + continue; + } + else + { + for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) + { + new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); + } + } + + if (new_parts.empty()) + continue; + + // ReadFromMergeTree::Settings step_settings + // { + // .max_block_size = max_block_size, + // .preferred_block_size_bytes = settings.preferred_block_size_bytes, + // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, + // .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order + // .use_uncompressed_cache = use_uncompressed_cache, + // .reader_settings = reader_settings, + // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), + // }; + + pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, + used_num_streams, 0, use_uncompressed_cache); + + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe.getHeader()); + } + + auto sorting_expr = std::make_shared( + metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + pipe.addSimpleTransform([sorting_expr](const Block & header) + { + return std::make_shared(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 + if (q_settings.do_not_merge_across_partitions_select_final && + 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)); + continue; + } + + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); + SortDescription sort_description; + size_t sort_columns_size = sort_columns.size(); + sort_description.reserve(sort_columns_size); + + Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + + const auto & header = pipe.getHeader(); + for (size_t i = 0; i < sort_columns_size; ++i) + sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); + + addMergingFinal( + pipe, + std::min(used_num_streams, q_settings.max_final_threads), + sort_description, data.merging_params, partition_key_columns, settings.max_block_size); + + // auto final_step = std::make_unique( + // plan->getCurrentDataStream(), + // std::min(used_num_streams, settings.max_final_threads), + // sort_description, + // data.merging_params, + // partition_key_columns, + // max_block_size); + + // final_step->setStepDescription("Merge rows for FINAL"); + // plan->addStep(std::move(final_step)); + + partition_pipes.emplace_back(std::move(pipe)); + } + + if (!lonely_parts.empty()) + { + RangesInDataParts new_parts; + + size_t num_streams_for_lonely_parts = used_num_streams * lonely_parts.size(); + + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + q_settings.merge_tree_min_rows_for_concurrent_read, + q_settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks_in_lonely_parts); + + /// Reduce the number of num_streams_for_lonely_parts if the data is small. + if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) + num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); + + // ReadFromMergeTree::Settings step_settings + // { + // .max_block_size = max_block_size, + // .preferred_block_size_bytes = settings.preferred_block_size_bytes, + // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, + // .min_marks_for_concurrent_read = min_marks_for_concurrent_read, + // .use_uncompressed_cache = use_uncompressed_cache, + // .reader_settings = reader_settings, + // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), + // }; + + // auto plan = std::make_unique(); + // auto step = std::make_unique( + // data, + // metadata_snapshot, + // query_id, + // column_names, + // std::move(lonely_parts), + // // std::move(index_stats), + // query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, + // virt_columns, + // step_settings, + // num_streams_for_lonely_parts, + // ReadFromMergeTree::ReadType::Default); + + // plan->addStep(std::move(step)); + + auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, + num_streams_for_lonely_parts, min_marks_for_concurrent_read, use_uncompressed_cache); + + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe.getHeader()); + + auto sorting_expr = std::make_shared( + metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + pipe.addSimpleTransform([sorting_expr](const Block & header) + { + return std::make_shared(header, sorting_expr); + }); + + // auto expression_step = std::make_unique( + // plan->getCurrentDataStream(), + // metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + // expression_step->setStepDescription("Calculate sorting key expression"); + // plan->addStep(std::move(expression_step)); + + partition_pipes.emplace_back(std::move(pipe)); + } + + return Pipe::unitePipes(std::move(partition_pipes)); +} + void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { + size_t total_parts = parts.size(); + auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); if (part_values && part_values->empty()) { @@ -320,9 +1010,127 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build sample_factor_column_queried = true; auto sampling = MergeTreeDataSelectExecutor::getSampling( - select, parts, metadata_snapshot, key_condition, settings, data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context) + select, parts, metadata_snapshot, key_condition, + data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); - Pipe pipe = read(); + if (sampling.read_nothing) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + size_t total_marks_pk = 0; + for (const auto & part : parts) + total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); + + auto parts_with_ranges = MergeTreeDataSelectExecutor::filterParts( + parts, + metadata_snapshot, + query_info, + context, + key_condition, + settings.reader_settings, + log, + num_streams, + index_stats); + + size_t sum_marks_pk = total_marks_pk; + for (const auto & stat : index_stats) + if (stat.type == IndexType::PrimaryKey) + sum_marks_pk = stat.num_granules_after; + + size_t sum_marks = 0; + size_t sum_ranges = 0; + + for (const auto & part : parts_with_ranges) + { + sum_ranges += part.ranges.size(); + sum_marks += part.getMarksCount(); + } + + LOG_DEBUG( + log, + "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", + parts.size(), + total_parts, + parts_with_ranges.size(), + sum_marks_pk, + total_marks_pk, + sum_marks, + sum_ranges); + + String query_id = MergeTreeDataSelectExecutor::checkLimits(data, parts_with_ranges, context); + + ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); + ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); + ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); + + /// Projection, that needed to drop columns, which have appeared by execution + /// of some extra expressions, and to allow execute the same expressions later. + /// NOTE: It may lead to double computation of expressions. + ActionsDAGPtr result_projection; + + Names column_names_to_read = real_column_names; + if (!select.final() && sampling.use_sampling) + { + /// Add columns needed for `sample_by_ast` to `column_names_to_read`. + /// Skip this if final was used, because such columns were already added from PK. + std::vector add_columns = sampling.filter_expression->getRequiredColumns().getNames(); + column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); + std::sort(column_names_to_read.begin(), column_names_to_read.end()); + column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), + column_names_to_read.end()); + } + + const auto & input_order_info = query_info.input_order_info + ? query_info.input_order_info + : (query_info.projection ? query_info.projection->input_order_info : nullptr); + + Pipe pipe; + + const auto & q_settings = context->getSettingsRef(); + + if (select.final()) + { + /// Add columns needed to calculate the sorting expression and the sign. + std::vector add_columns = metadata_snapshot->getColumnsRequiredForSortingKey(); + column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); + + if (!data.merging_params.sign_column.empty()) + column_names_to_read.push_back(data.merging_params.sign_column); + if (!data.merging_params.version_column.empty()) + column_names_to_read.push_back(data.merging_params.version_column); + + std::sort(column_names_to_read.begin(), column_names_to_read.end()); + column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); + + pipe = spreadMarkRangesAmongStreamsFinal( + std::move(parts_with_ranges), + column_names_to_read, + result_projection); + } + else if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + { + size_t prefix_size = input_order_info->order_key_prefix_descr.size(); + auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); + order_key_prefix_ast->children.resize(prefix_size); + + auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical()); + auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false); + + pipe = spreadMarkRangesAmongStreamsWithOrder( + std::move(parts_with_ranges), + column_names_to_read, + sorting_key_prefix_expr, + result_projection, + input_order_info); + } + else + { + pipe = spreadMarkRangesAmongStreams( + std::move(parts_with_ranges), + column_names_to_read); + } for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); @@ -371,40 +1179,40 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type) void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const { std::string prefix(format_settings.offset, format_settings.indent_char); - format_settings.out << prefix << "ReadType: " << readTypeToString(read_type) << '\n'; + //format_settings.out << prefix << "ReadType: " << readTypeToString(read_type) << '\n'; - if (index_stats && !index_stats->empty()) + if (!index_stats.empty()) { - format_settings.out << prefix << "Parts: " << index_stats->back().num_parts_after << '\n'; - format_settings.out << prefix << "Granules: " << index_stats->back().num_granules_after << '\n'; + format_settings.out << prefix << "Parts: " << index_stats.back().num_parts_after << '\n'; + format_settings.out << prefix << "Granules: " << index_stats.back().num_granules_after << '\n'; } } void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { - map.add("Read Type", readTypeToString(read_type)); - if (index_stats && !index_stats->empty()) + //map.add("Read Type", readTypeToString(read_type)); + if (!index_stats.empty()) { - map.add("Parts", index_stats->back().num_parts_after); - map.add("Granules", index_stats->back().num_granules_after); + map.add("Parts", index_stats.back().num_parts_after); + map.add("Granules", index_stats.back().num_granules_after); } } void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const { std::string prefix(format_settings.offset, format_settings.indent_char); - if (index_stats && !index_stats->empty()) + if (!index_stats.empty()) { /// Do not print anything if no indexes is applied. - if (index_stats->size() == 1 && index_stats->front().type == IndexType::None) + if (index_stats.size() == 1 && index_stats.front().type == IndexType::None) return; std::string indent(format_settings.indent, format_settings.indent_char); format_settings.out << prefix << "Indexes:\n"; - for (size_t i = 0; i < index_stats->size(); ++i) + for (size_t i = 0; i < index_stats.size(); ++i) { - const auto & stat = (*index_stats)[i]; + const auto & stat = index_stats[i]; if (stat.type == IndexType::None) continue; @@ -428,12 +1236,12 @@ void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const format_settings.out << prefix << indent << indent << "Parts: " << stat.num_parts_after; if (i) - format_settings.out << '/' << (*index_stats)[i - 1].num_parts_after; + format_settings.out << '/' << index_stats[i - 1].num_parts_after; format_settings.out << '\n'; format_settings.out << prefix << indent << indent << "Granules: " << stat.num_granules_after; if (i) - format_settings.out << '/' << (*index_stats)[i - 1].num_granules_after; + format_settings.out << '/' << index_stats[i - 1].num_granules_after; format_settings.out << '\n'; } } @@ -441,17 +1249,17 @@ void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const { - if (index_stats && !index_stats->empty()) + if (!index_stats.empty()) { /// Do not print anything if no indexes is applied. - if (index_stats->size() == 1 && index_stats->front().type == IndexType::None) + if (index_stats.size() == 1 && index_stats.front().type == IndexType::None) return; auto indexes_array = std::make_unique(); - for (size_t i = 0; i < index_stats->size(); ++i) + for (size_t i = 0; i < index_stats.size(); ++i) { - const auto & stat = (*index_stats)[i]; + const auto & stat = index_stats[i]; if (stat.type == IndexType::None) continue; @@ -479,11 +1287,11 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const index_map->add("Condition", stat.condition); if (i) - index_map->add("Initial Parts", (*index_stats)[i - 1].num_parts_after); + index_map->add("Initial Parts", index_stats[i - 1].num_parts_after); index_map->add("Selected Parts", stat.num_parts_after); if (i) - index_map->add("Initial Granules", (*index_stats)[i - 1].num_granules_after); + index_map->add("Initial Granules", index_stats[i - 1].num_granules_after); index_map->add("Selected Granules", stat.num_granules_after); indexes_array->add(std::move(index_map)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 762d7fea0e7..b3366afb118 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -74,7 +74,7 @@ public: const MergeTreeData & data_, const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, - String query_id_, + StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, //IndexStatPtr index_stats_, @@ -82,7 +82,7 @@ public: Names virt_column_names_, Settings settings_, size_t num_streams_, - ReadType read_type_ + //ReadType read_type_ ); String getName() const override { return "ReadFromMergeTree"; } @@ -102,26 +102,42 @@ private: const MergeTreeData & data; const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; - String query_id; + StorageMetadataPtr metadata_snapshot_base; Names real_column_names; MergeTreeData::DataPartsVector parts; - IndexStat index_stats; + IndexStats index_stats; PrewhereInfoPtr prewhere_info; Names virt_column_names; Settings settings; size_t num_streams; - ReadType read_type; + //ReadType read_type; Poco::Logger * log; - Pipe read(); - Pipe readFromPool(); - Pipe readInOrder(); + Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache); template - ProcessorPtr createSource(const RangesInDataPart & part); + ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache); + + Pipe spreadMarkRangesAmongStreams( + RangesInDataParts && parts_with_ranges, + const Names & column_names); + + Pipe spreadMarkRangesAmongStreamsWithOrder( + RangesInDataParts && parts_with_ranges, + const Names & column_names, + const ActionsDAGPtr & sorting_key_prefix_expr, + ActionsDAGPtr & out_projection, + const InputOrderInfoPtr & input_order_info); + + Pipe spreadMarkRangesAmongStreamsFinal( + RangesInDataParts && parts, + const Names & column_names, + ActionsDAGPtr & out_projection); }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 444d074a4db..49bcf751bd1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -633,6 +633,271 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } +RangesInDataParts MergeTreeDataSelectExecutor::filterParts( + MergeTreeData::DataPartsVector & parts, + StorageMetadataPtr metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr & context, + KeyCondition & key_condition, + const MergeTreeReaderSettings & reader_settings, + Poco::Logger * log, + size_t num_streams, + ReadFromMergeTree::IndexStats & index_stats) +{ + RangesInDataParts parts_with_ranges(parts.size()); + const Settings & settings = context->getSettingsRef(); + + /// Let's start analyzing all useful indices + + struct DataSkippingIndexAndCondition + { + MergeTreeIndexPtr index; + MergeTreeIndexConditionPtr condition; + std::atomic total_granules{0}; + std::atomic granules_dropped{0}; + std::atomic total_parts{0}; + std::atomic parts_dropped{0}; + + DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) + : index(index_), condition(condition_) + { + } + }; + std::list useful_indices; + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } + + if (settings.force_data_skipping_indices.changed) + { + const auto & indices = settings.force_data_skipping_indices.toString(); + + Strings forced_indices; + { + Tokens tokens(&indices[0], &indices[indices.size()], settings.max_query_size); + IParser::Pos pos(tokens, settings.max_parser_depth); + Expected expected; + if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); + } + + if (forced_indices.empty()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); + + std::unordered_set useful_indices_names; + for (const auto & useful_index : useful_indices) + useful_indices_names.insert(useful_index.index->index.name); + + for (const auto & index_name : forced_indices) + { + if (!useful_indices_names.count(index_name)) + { + throw Exception( + ErrorCodes::INDEX_NOT_USED, + "Index {} is not used and setting 'force_data_skipping_indices' contains it", + backQuote(index_name)); + } + } + } + + std::atomic sum_marks_pk = 0; + std::atomic sum_parts_pk = 0; + + /// Let's find what range to read from each part. + { + std::atomic total_rows{0}; + + SizeLimits limits; + if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) + limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); + + SizeLimits leaf_limits; + if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); + + auto process_part = [&](size_t part_index) + { + auto & part = parts[part_index]; + + RangesInDataPart ranges(part, part_index); + + size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); + + if (metadata_snapshot->hasPrimaryKey()) + ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings, log); + else if (total_marks_count) + ranges.ranges = MarkRanges{MarkRange{0, total_marks_count}}; + + sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed); + + if (!ranges.ranges.empty()) + sum_parts_pk.fetch_add(1, std::memory_order_relaxed); + + for (auto & index_and_condition : useful_indices) + { + if (ranges.ranges.empty()) + break; + + index_and_condition.total_parts.fetch_add(1, std::memory_order_relaxed); + + size_t total_granules = 0; + size_t granules_dropped = 0; + ranges.ranges = filterMarksUsingIndex( + index_and_condition.index, + index_and_condition.condition, + part, + ranges.ranges, + settings, + reader_settings, + total_granules, + granules_dropped, + log); + + index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + index_and_condition.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + + if (ranges.ranges.empty()) + index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed); + } + + if (!ranges.ranges.empty()) + { + if (limits.max_rows || leaf_limits.max_rows) + { + /// Fail fast if estimated number of rows to read exceeds the limit + auto current_rows_estimate = ranges.getRowsCount(); + size_t prev_total_rows_estimate = total_rows.fetch_add(current_rows_estimate); + size_t total_rows_estimate = current_rows_estimate + prev_total_rows_estimate; + limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); + leaf_limits.check( + total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); + } + + parts_with_ranges[part_index] = std::move(ranges); + } + }; + + size_t num_threads = std::min(size_t(num_streams), parts.size()); + + if (num_threads <= 1) + { + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + process_part(part_index); + } + else + { + /// Parallel loading of data parts. + ThreadPool pool(num_threads); + + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] + { + SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachQueryIfNotDetached();); + if (thread_group) + CurrentThread::attachTo(thread_group); + + process_part(part_index); + }); + + pool.wait(); + } + + /// Skip empty ranges. + size_t next_part = 0; + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + { + auto & part = parts_with_ranges[part_index]; + if (!part.data_part) + continue; + + if (next_part != part_index) + std::swap(parts_with_ranges[next_part], part); + + ++next_part; + } + + parts_with_ranges.resize(next_part); + } + + if (metadata_snapshot->hasPrimaryKey()) + { + auto description = key_condition.getDescription(); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::PrimaryKey, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = sum_parts_pk.load(std::memory_order_relaxed), + .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); + } + + for (const auto & index_and_condition : useful_indices) + { + const auto & index_name = index_and_condition.index->index.name; + LOG_DEBUG( + log, + "Index {} has dropped {}/{} granules.", + backQuote(index_name), + index_and_condition.granules_dropped, + index_and_condition.total_granules); + + std::string description + = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Skip, + .name = index_name, + .description = std::move(description), + .num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped, + .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); + } + + return parts_with_ranges; +} + +void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_range, ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + // Check limitations. query_id is used as the quota RAII's resource key. + String query_id; + { + const auto data_settings = data.getSettings(); + auto max_partitions_to_read + = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; + if (max_partitions_to_read > 0) + { + std::set partitions; + for (auto & part_with_ranges : parts_with_ranges) + partitions.insert(part_with_ranges.data_part->info.partition_id); + if (partitions.size() > size_t(max_partitions_to_read)) + throw Exception( + ErrorCodes::TOO_MANY_PARTITIONS, + "Too many partitions to read. Current {}, max {}", + partitions.size(), + max_partitions_to_read); + } + + if (data_settings->max_concurrent_queries > 0 && data_settings->min_marks_to_honor_max_concurrent_queries > 0) + { + size_t sum_marks = 0; + for (const auto & part : parts_with_ranges) + sum_marks += part.getMarksCount(); + + if (sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) + { + query_id = context->getCurrentQueryId(); + if (!query_id.empty()) + data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries); + } + } + } +} + QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 231908e2ee7..7597af2e173 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -72,7 +72,6 @@ private: QueryPlanPtr spreadMarkRangesAmongStreams( RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, size_t num_streams, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -87,7 +86,6 @@ private: /// out_projection - save projection only with columns, requested to read QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, size_t num_streams, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -104,7 +102,6 @@ private: QueryPlanPtr spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, size_t num_streams, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -178,6 +175,17 @@ public: PartFilterCounters & counters, Poco::Logger * log); + static RangesInDataParts filterParts( + MergeTreeData::DataPartsVector & parts, + StorageMetadataPtr metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr & context, + KeyCondition & key_condition, + const MergeTreeReaderSettings & reader_settings, + Poco::Logger * log, + size_t num_streams, + ReadFromMergeTree::IndexStats & index_stats); + static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, MergeTreeData::DataPartsVector & parts, @@ -188,6 +196,8 @@ public: bool sample_factor_column_queried, NamesAndTypesList available_real_columns, ContextPtr context); + + static String checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); }; } From df400d1437e56fd0d8f9be89691ab6f0c8d874b6 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 26 May 2021 23:01:20 +0300 Subject: [PATCH 226/652] Fix links MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылки. --- docs/en/engines/table-engines/special/dictionary.md | 2 +- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- docs/ru/engines/table-engines/special/dictionary.md | 2 +- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/dictionary.md b/docs/en/engines/table-engines/special/dictionary.md index e143f3a8ee1..d76adebe01e 100644 --- a/docs/en/engines/table-engines/special/dictionary.md +++ b/docs/en/engines/table-engines/special/dictionary.md @@ -96,4 +96,4 @@ select * from products limit 1; **See Also** -- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 3b9a28403f7..533ec6090ff 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -100,7 +100,7 @@ When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONAR **See Also** -- [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) ## Executable File {#dicts-external_dicts_dict_sources-executable} diff --git a/docs/ru/engines/table-engines/special/dictionary.md b/docs/ru/engines/table-engines/special/dictionary.md index df47ade5a42..15d32419472 100644 --- a/docs/ru/engines/table-engines/special/dictionary.md +++ b/docs/ru/engines/table-engines/special/dictionary.md @@ -92,4 +92,4 @@ select * from products limit 1; **Смотрите также** -- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index a9f3857f183..76290c72490 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -99,7 +99,7 @@ SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) **Смотрите также** -- [Функция dictionary](../../sql-reference/table-functions/dictionary.md#dictionary-function) +- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function) ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} From 42b2383e0192afefbaf42da937e293af9c238f99 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 May 2021 23:37:44 +0300 Subject: [PATCH 227/652] Basic throttler implementation --- src/Core/Settings.h | 2 + src/IO/copyData.cpp | 33 +++++++++++---- src/IO/copyData.h | 6 +++ src/Interpreters/Context.cpp | 25 ++++++++++++ src/Interpreters/Context.h | 6 +++ src/Storages/MergeTree/DataPartsExchange.cpp | 43 +++++++++++++------- src/Storages/MergeTree/DataPartsExchange.h | 23 +++++++---- src/Storages/MergeTree/MergeTreeSettings.h | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 12 ++++-- src/Storages/StorageReplicatedMergeTree.h | 14 +++++++ 10 files changed, 133 insertions(+), 33 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d759e87dfbc..1a723df2d48 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -83,6 +83,8 @@ class IColumn; M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, dns cache updates. Only has meaning at server startup.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "Number of threads performing background tasks for message streaming. Only has meaning at server startup.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \ + M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \ + M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.", 0) \ \ M(Milliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ M(Milliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \ diff --git a/src/IO/copyData.cpp b/src/IO/copyData.cpp index c653af761d4..428741f2a47 100644 --- a/src/IO/copyData.cpp +++ b/src/IO/copyData.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -14,7 +15,7 @@ namespace ErrorCodes namespace { -void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, const std::atomic * is_cancelled) +void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, const std::atomic * is_cancelled, ThrottlerPtr throttler) { /// If read to the end of the buffer, eof() either fills the buffer with new data and moves the cursor to the beginning, or returns false. while (bytes > 0 && !from.eof()) @@ -27,13 +28,16 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t to.write(from.position(), count); from.position() += count; bytes -= count; + + if (throttler) + throttler->add(count); } if (check_bytes && bytes > 0) throw Exception("Attempt to read after EOF.", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); } -void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function cancellation_hook) +void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function cancellation_hook, ThrottlerPtr throttler) { /// If read to the end of the buffer, eof() either fills the buffer with new data and moves the cursor to the beginning, or returns false. while (bytes > 0 && !from.eof()) @@ -46,6 +50,9 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t to.write(from.position(), count); from.position() += count; bytes -= count; + + if (throttler) + throttler->add(count); } if (check_bytes && bytes > 0) @@ -56,32 +63,42 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t void copyData(ReadBuffer & from, WriteBuffer & to) { - copyDataImpl(from, to, false, std::numeric_limits::max(), nullptr); + copyDataImpl(from, to, false, std::numeric_limits::max(), nullptr, nullptr); } void copyData(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled) { - copyDataImpl(from, to, false, std::numeric_limits::max(), &is_cancelled); + copyDataImpl(from, to, false, std::numeric_limits::max(), &is_cancelled, nullptr); } void copyData(ReadBuffer & from, WriteBuffer & to, std::function cancellation_hook) { - copyDataImpl(from, to, false, std::numeric_limits::max(), cancellation_hook); + copyDataImpl(from, to, false, std::numeric_limits::max(), cancellation_hook, nullptr); } void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes) { - copyDataImpl(from, to, true, bytes, nullptr); + copyDataImpl(from, to, true, bytes, nullptr, nullptr); } void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled) { - copyDataImpl(from, to, true, bytes, &is_cancelled); + copyDataImpl(from, to, true, bytes, &is_cancelled, nullptr); } void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, std::function cancellation_hook) { - copyDataImpl(from, to, true, bytes, cancellation_hook); + copyDataImpl(from, to, true, bytes, cancellation_hook, nullptr); +} + +void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled, ThrottlerPtr throttler) +{ + copyDataImpl(from, to, true, std::numeric_limits::max(), &is_cancelled, throttler); +} + +void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled, ThrottlerPtr throttler) +{ + copyDataImpl(from, to, true, bytes, &is_cancelled, throttler); } } diff --git a/src/IO/copyData.h b/src/IO/copyData.h index f888a039e9e..56244167861 100644 --- a/src/IO/copyData.h +++ b/src/IO/copyData.h @@ -9,6 +9,9 @@ namespace DB class ReadBuffer; class WriteBuffer; +class Throttler; + +using ThrottlerPtr = std::shared_ptr; /** Copies data from ReadBuffer to WriteBuffer, all that is. @@ -24,6 +27,9 @@ void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes); void copyData(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled); void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled); +void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled, ThrottlerPtr throttler); +void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled, ThrottlerPtr throttler); + void copyData(ReadBuffer & from, WriteBuffer & to, std::function cancellation_hook); void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, std::function cancellation_hook); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7d9c2c27ee2..f1cf89dfaf6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -360,6 +361,10 @@ struct ContextSharedPart mutable std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) mutable std::optional distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) mutable std::optional message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka) + + mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches + mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends + MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. @@ -1625,6 +1630,26 @@ BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const return *shared->message_broker_schedule_pool; } +ThrottlerPtr Context::getReplicatedFetchesThrottler() const +{ + auto lock = getLock(); + if (!shared->replicated_fetches_throttler) + shared->replicated_fetches_throttler = std::make_shared( + settings.max_replicated_fetches_network_bandwidth_for_server); + + return shared->replicated_fetches_throttler; +} + +ThrottlerPtr Context::getReplicatedSendsThrottler() const +{ + auto lock = getLock(); + if (!shared->replicated_sends_throttler) + shared->replicated_sends_throttler = std::make_shared( + settings.max_replicated_sends_network_bandwidth_for_server); + + return shared->replicated_fetches_throttler; +} + bool Context::hasDistributedDDL() const { return getConfigRef().has("distributed_ddl"); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5089d2c0288..4fca9639a50 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -113,6 +113,9 @@ using VolumePtr = std::shared_ptr; struct NamedSession; struct BackgroundTaskSchedulingSettings; +class Throttler; +using ThrottlerPtr = std::shared_ptr; + class ZooKeeperMetadataTransaction; using ZooKeeperMetadataTransactionPtr = std::shared_ptr; @@ -657,6 +660,9 @@ public: BackgroundSchedulePool & getMessageBrokerSchedulePool() const; BackgroundSchedulePool & getDistributedSchedulePool() const; + ThrottlerPtr getReplicatedFetchesThrottler() const; + ThrottlerPtr getReplicatedSendsThrottler() const; + /// Has distributed_ddl configuration or not. bool hasDistributedDDL() const; void setDDLWorker(std::unique_ptr ddl_worker); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 5ccb2ca8c27..9eeac69978f 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +87,10 @@ struct ReplicatedFetchReadCallback } + +Service::Service(StorageReplicatedMergeTree & data_) : + data(data_), log(&Poco::Logger::get(data.getLogName() + " (Replicated PartsService)")) {} + std::string Service::getId(const std::string & node_id) const { return getEndpointId(node_id); @@ -243,6 +248,8 @@ void Service::sendPartFromMemory( NativeBlockOutputStream block_out(out, 0, metadata_snapshot->getSampleBlock()); part->checksums.write(out); block_out.write(part_in_memory->block); + + data.getSendsThrottler()->add(part_in_memory->block.bytes()); } MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( @@ -298,7 +305,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( auto file_in = disk->readFile(path); HashingWriteBuffer hashing_out(out); - copyData(*file_in, hashing_out, blocker.getCounter()); + copyDataWithThrottler(*file_in, hashing_out, blocker.getCounter(), data.getSendsThrottler()); if (blocker.isCancelled()) throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); @@ -354,7 +361,7 @@ void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteB auto file_in = createReadBufferFromFileBase(metadata_file, 0, 0, 0, nullptr, DBMS_DEFAULT_BUFFER_SIZE); HashingWriteBuffer hashing_out(out); - copyData(*file_in, hashing_out, blocker.getCounter()); + copyDataWithThrottler(*file_in, hashing_out, blocker.getCounter(), data.getSendsThrottler()); if (blocker.isCancelled()) throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); @@ -388,6 +395,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( const String & user, const String & password, const String & interserver_scheme, + ThrottlerPtr throttler, bool to_detached, const String & tmp_prefix_, std::optional * tagger_ptr, @@ -514,7 +522,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( try { - return downloadPartToS3(part_name, replica_path, to_detached, tmp_prefix_, std::move(disks_s3), in); + return downloadPartToS3(part_name, replica_path, to_detached, tmp_prefix_, std::move(disks_s3), in, throttler); } catch (const Exception & e) { @@ -522,7 +530,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( throw; /// Try again but without S3 copy return fetchPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts, - user, password, interserver_scheme, to_detached, tmp_prefix_, nullptr, false); + user, password, interserver_scheme, throttler, to_detached, tmp_prefix_, nullptr, false); } } @@ -585,8 +593,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( MergeTreeData::DataPart::Checksums checksums; return part_type == "InMemory" - ? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, context, std::move(reservation), in, projections) - : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, reservation->getDisk(), in, projections, checksums); + ? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, context, std::move(reservation), in, projections, throttler) + : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, reservation->getDisk(), in, projections, checksums, throttler); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( @@ -596,7 +604,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( ContextPtr context, ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in, - size_t projections) + size_t projections, + ThrottlerPtr throttler) { auto volume = std::make_shared("volume_" + part_name, reservation->getDisk(), 0); MergeTreeData::MutableDataPartPtr new_data_part = @@ -612,6 +621,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( NativeBlockInputStream block_in(in, 0); auto block = block_in.read(); + throttler->add(block.bytes()); MergeTreePartInfo new_part_info("all", 0, 0, 0); MergeTreeData::MutableDataPartPtr new_projection_part = @@ -643,6 +653,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( NativeBlockInputStream block_in(in, 0); auto block = block_in.read(); + throttler->add(block.bytes()); new_data_part->uuid = part_uuid; new_data_part->is_temp = true; @@ -666,7 +677,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( bool sync, DiskPtr disk, PooledReadWriteBufferFromHTTP & in, - MergeTreeData::DataPart::Checksums & checksums) const + MergeTreeData::DataPart::Checksums & checksums, + ThrottlerPtr throttler) const { size_t files; readBinary(files, in); @@ -689,7 +701,7 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( auto file_out = disk->writeFile(part_download_path + file_name); HashingWriteBuffer hashing_out(*file_out); - copyData(in, hashing_out, file_size, blocker.getCounter()); + copyDataWithThrottler(in, hashing_out, file_size, blocker.getCounter(), throttler); if (blocker.isCancelled()) { @@ -726,7 +738,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( DiskPtr disk, PooledReadWriteBufferFromHTTP & in, size_t projections, - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, + ThrottlerPtr throttler) { static const String TMP_PREFIX = "tmp_fetch_"; String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_; @@ -763,13 +776,13 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( MergeTreeData::DataPart::Checksums projection_checksum; disk->createDirectories(part_download_path + projection_name + ".proj/"); downloadBaseOrProjectionPartToDisk( - replica_path, part_download_path + projection_name + ".proj/", sync, disk, in, projection_checksum); + replica_path, part_download_path + projection_name + ".proj/", sync, disk, in, projection_checksum, throttler); checksums.addFile( projection_name + ".proj", projection_checksum.getTotalSizeOnDisk(), projection_checksum.getTotalChecksumUInt128()); } // Download the base part - downloadBaseOrProjectionPartToDisk(replica_path, part_download_path, sync, disk, in, checksums); + downloadBaseOrProjectionPartToDisk(replica_path, part_download_path, sync, disk, in, checksums, throttler); assertEOF(in); auto volume = std::make_shared("volume_" + part_name, disk, 0); @@ -787,8 +800,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3( bool to_detached, const String & tmp_prefix_, const Disks & disks_s3, - PooledReadWriteBufferFromHTTP & in - ) + PooledReadWriteBufferFromHTTP & in, + ThrottlerPtr throttler) { if (disks_s3.empty()) throw Exception("No S3 disks anymore", ErrorCodes::LOGICAL_ERROR); @@ -841,7 +854,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3( HashingWriteBuffer hashing_out(*file_out); - copyData(in, hashing_out, file_size, blocker.getCounter()); + copyDataWithThrottler(in, hashing_out, file_size, blocker.getCounter(), throttler); if (blocker.isCancelled()) { diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index f59942ef7f4..eb776c33f0f 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace zkutil @@ -18,15 +19,17 @@ namespace zkutil namespace DB { +class StorageReplicatedMergeTree; + namespace DataPartsExchange { -/** Service for sending parts from the table *MergeTree. +/** Service for sending parts from the table *ReplicatedMergeTree. */ class Service final : public InterserverIOEndpoint { public: - explicit Service(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get(data.getLogName() + " (Replicated PartsService)")) {} + explicit Service(StorageReplicatedMergeTree & data_); Service(const Service &) = delete; Service & operator=(const Service &) = delete; @@ -51,7 +54,7 @@ private: /// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish, /// so Service will never access dangling reference to storage - MergeTreeData & data; + StorageReplicatedMergeTree & data; Poco::Logger * log; }; @@ -74,6 +77,7 @@ public: const String & user, const String & password, const String & interserver_scheme, + ThrottlerPtr throttler, bool to_detached = false, const String & tmp_prefix_ = "", std::optional * tagger_ptr = nullptr, @@ -90,7 +94,9 @@ private: bool sync, DiskPtr disk, PooledReadWriteBufferFromHTTP & in, - MergeTreeData::DataPart::Checksums & checksums) const; + MergeTreeData::DataPart::Checksums & checksums, + ThrottlerPtr throttler) const; + MergeTreeData::MutableDataPartPtr downloadPartToDisk( const String & part_name, @@ -101,7 +107,8 @@ private: DiskPtr disk, PooledReadWriteBufferFromHTTP & in, size_t projections, - MergeTreeData::DataPart::Checksums & checksums); + MergeTreeData::DataPart::Checksums & checksums, + ThrottlerPtr throttler); MergeTreeData::MutableDataPartPtr downloadPartToMemory( const String & part_name, @@ -110,7 +117,8 @@ private: ContextPtr context, ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in, - size_t projections); + size_t projections, + ThrottlerPtr throttler); MergeTreeData::MutableDataPartPtr downloadPartToS3( const String & part_name, @@ -118,7 +126,8 @@ private: bool to_detached, const String & tmp_prefix_, const Disks & disks_s3, - PooledReadWriteBufferFromHTTP & in); + PooledReadWriteBufferFromHTTP & in, + ThrottlerPtr throttler); MergeTreeData & data; Poco::Logger * log; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6e43d0fad77..da953eb0f47 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -91,6 +91,8 @@ struct Settings; M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ M(Bool, detach_old_local_parts_when_cloning_replica, 1, "Do not remove old local parts when repairing lost replica.", 0) \ + M(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ + M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ \ /** Check delay of replicas settings. */ \ M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b3da3e2287b..d776a51b06d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -287,6 +287,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , part_moves_between_shards_orchestrator(*this) , allow_renaming(allow_renaming_) , replicated_fetches_pool_size(getContext()->getSettingsRef().background_fetches_pool_size) + , replicated_fetches_throttler(std::make_shared(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler())) + , replicated_sends_throttler(std::make_shared(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler())) { queue_updating_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); @@ -2501,7 +2503,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) part_desc->res_part = fetcher.fetchPart( metadata_snapshot, getContext(), part_desc->found_new_part_name, source_replica_path, - address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, false, TMP_PREFIX + "fetch_"); + address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), + interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_"); /// TODO: check columns_version of fetched part @@ -2618,7 +2621,8 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr return fetcher.fetchPart( metadata_snapshot, getContext(), entry.new_part_name, source_replica_path, address.host, address.replication_port, - timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, true); + timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, + replicated_fetches_throttler, true); }; part = get_part(); @@ -4025,6 +4029,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora credentials->getUser(), credentials->getPassword(), interserver_scheme, + replicated_fetches_throttler, to_detached, "", &tagger_ptr, @@ -4174,7 +4179,8 @@ bool StorageReplicatedMergeTree::fetchExistsPart(const String & part_name, const return fetcher.fetchPart( metadata_snapshot, getContext(), part_name, source_replica_path, address.host, address.replication_port, - timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, false, "", nullptr, true, + timeouts, credentials->getUser(), credentials->getPassword(), + interserver_scheme, replicated_fetches_throttler, false, "", nullptr, true, replaced_disk); }; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4e697f2d1f2..93f86e8ae7e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -239,6 +240,16 @@ public: /// Get best replica having this partition on S3 String getSharedDataReplica(const IMergeTreeDataPart & part) const; + ThrottlerPtr getFetchesThrottler() const + { + return replicated_fetches_throttler; + } + + ThrottlerPtr getSendsThrottler() const + { + return replicated_sends_throttler; + } + private: /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; @@ -363,6 +374,9 @@ private: const size_t replicated_fetches_pool_size; + ThrottlerPtr replicated_fetches_throttler; + ThrottlerPtr replicated_sends_throttler; + template void foreachCommittedParts(Func && func, bool select_sequential_consistency) const; From ae60a3dd3a787c2c1d590a70edcf8cc65d3681b4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 27 May 2021 07:24:16 +0300 Subject: [PATCH 228/652] new tests from master --- tests/integration/helpers/cluster.py | 2 ++ .../test.py | 28 +++++++++---------- .../integration/test_merge_tree_hdfs/test.py | 18 ++++++------ 3 files changed, 25 insertions(+), 23 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e1b37576a18..bdb5cc78dff 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -254,6 +254,7 @@ class ClickHouseCluster: # available when with_hdfs == True self.hdfs_host = "hdfs1" + self.hdfs_ip = None self.hdfs_name_port = get_free_port() self.hdfs_data_port = get_free_port() self.hdfs_dir = p.abspath(p.join(self.instances_dir, "hdfs")) @@ -1108,6 +1109,7 @@ class ClickHouseCluster: def wait_hdfs_to_start(self, hdfs_api, timeout=300): + self.hdfs_ip = self.get_instance_ip(self.hdfs_host) start = time.time() while time.time() - start < timeout: try: diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index a01e60af47d..b05c4a5f3b4 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -3,20 +3,20 @@ import os import pytest from helpers.cluster import ClickHouseCluster +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +cluster = ClickHouseCluster(__file__) +node_memory = cluster.add_instance('node_memory', main_configs=['configs/enable_dictionaries.xml', + 'configs/dictionaries/complex_key_cache_string.xml']) +node_ssd = cluster.add_instance('node_ssd', main_configs=['configs/enable_dictionaries.xml', + 'configs/dictionaries/ssd_complex_key_cache_string.xml']) -@pytest.fixture(scope="function") -def cluster(request): - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - cluster = ClickHouseCluster(__file__) +@pytest.fixture() +def started_cluster(): try: - if request.param == "memory": - node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/complex_key_cache_string.xml']) - if request.param == "ssd": - node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/ssd_complex_key_cache_string.xml']) cluster.start() - node.query( + node_memory.query( + "create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") + node_ssd.query( "create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") yield cluster @@ -24,9 +24,9 @@ def cluster(request): cluster.shutdown() -@pytest.mark.parametrize("cluster", ["memory", "ssd"], indirect=True) -def test_memory_consumption(cluster): - node = cluster.instances['node'] +@pytest.mark.parametrize("type", ["memory", "ssd"]) +def test_memory_consumption(started_cluster, type): + node = started_cluster.instances[f'node_{type}'] node.query( "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format('w' * 8)) node.query( diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index 2d0d9d9fb1e..0984e4d288a 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -64,7 +64,7 @@ def cluster(): cluster.start() logging.info("Cluster started") - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) fs.mkdirs('/clickhouse') logging.info("Created HDFS directory") @@ -75,7 +75,7 @@ def cluster(): def wait_for_delete_hdfs_objects(cluster, expected, num_tries=30): - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) while num_tries > 0: num_hdfs_objects = len(fs.listdir('/clickhouse')) if num_hdfs_objects == expected: @@ -89,7 +89,7 @@ def wait_for_delete_hdfs_objects(cluster, expected, num_tries=30): def drop_table(cluster): node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') print('Number of hdfs objects to delete:', len(hdfs_objects), sep=' ') @@ -116,7 +116,7 @@ def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part): node.query("INSERT INTO hdfs_test VALUES {}".format(values1)) assert node.query("SELECT * FROM hdfs_test order by dt, id FORMAT Values") == values1 - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') print(hdfs_objects) @@ -136,7 +136,7 @@ def test_alter_table_columns(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096, -1))) @@ -165,7 +165,7 @@ def test_attach_detach_partition(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) @@ -204,7 +204,7 @@ def test_move_partition_to_another_disk(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) @@ -230,7 +230,7 @@ def test_table_manipulations(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) @@ -262,7 +262,7 @@ def test_move_replace_partition_to_another_table(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) From 55dd173535b9efee498844805e2e57ad5feaec87 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 27 May 2021 10:58:12 +0300 Subject: [PATCH 229/652] more --- tests/integration/parallel_skip.json | 3 --- .../integration/test_log_family_hdfs/test.py | 20 +++++++++---------- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 088e3a3ba82..cb6c5f735dc 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -121,7 +121,6 @@ "test_insert_into_distributed/test.py::test_prefer_localhost_replica", "test_insert_into_distributed/test.py::test_reconnect", "test_insert_into_distributed/test.py::test_table_function", - "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED", "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", @@ -192,8 +191,6 @@ "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED", - "test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED", "test_polymorphic_parts/test.py::test_polymorphic_parts_index", "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", diff --git a/tests/integration/test_log_family_hdfs/test.py b/tests/integration/test_log_family_hdfs/test.py index 44f6904e8ea..fcf64381dd9 100644 --- a/tests/integration/test_log_family_hdfs/test.py +++ b/tests/integration/test_log_family_hdfs/test.py @@ -8,7 +8,7 @@ from pyhdfs import HdfsClient @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("node", @@ -18,7 +18,7 @@ def cluster(): cluster.start() logging.info("Cluster started") - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) fs.mkdirs('/clickhouse') yield cluster @@ -26,8 +26,8 @@ def cluster(): cluster.shutdown() -def assert_objects_count(cluster, objects_count, path='data/'): - fs = HdfsClient(hosts='localhost') +def assert_objects_count(started_cluster, objects_count, path='data/'): + fs = HdfsClient(hosts=cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') assert objects_count == len(hdfs_objects) @@ -35,25 +35,25 @@ def assert_objects_count(cluster, objects_count, path='data/'): @pytest.mark.parametrize( "log_engine,files_overhead,files_overhead_per_insert", [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) -def test_log_family_hdfs(cluster, log_engine, files_overhead, files_overhead_per_insert): - node = cluster.instances["node"] +def test_log_family_hdfs(started_cluster, log_engine, files_overhead, files_overhead_per_insert): + node = started_cluster.instances["node"] node.query("CREATE TABLE hdfs_test (id UInt64) ENGINE={} SETTINGS disk = 'hdfs'".format(log_engine)) node.query("INSERT INTO hdfs_test SELECT number FROM numbers(5)") assert node.query("SELECT * FROM hdfs_test") == "0\n1\n2\n3\n4\n" - assert_objects_count(cluster, files_overhead_per_insert + files_overhead) + assert_objects_count(started_cluster, files_overhead_per_insert + files_overhead) node.query("INSERT INTO hdfs_test SELECT number + 5 FROM numbers(3)") assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n" - assert_objects_count(cluster, files_overhead_per_insert * 2 + files_overhead) + assert_objects_count(started_cluster, files_overhead_per_insert * 2 + files_overhead) node.query("INSERT INTO hdfs_test SELECT number + 8 FROM numbers(1)") assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n8\n" - assert_objects_count(cluster, files_overhead_per_insert * 3 + files_overhead) + assert_objects_count(started_cluster, files_overhead_per_insert * 3 + files_overhead) node.query("TRUNCATE TABLE hdfs_test") - assert_objects_count(cluster, 0) + assert_objects_count(started_cluster, 0) node.query("DROP TABLE hdfs_test") From 47e8a53d2a632faa09689f38917f96c3a3c4883a Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 27 May 2021 12:55:15 +0300 Subject: [PATCH 230/652] Fix flaky test --- .../0_stateless/01881_total_bytes_storage_buffer.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql index d962a6db456..1fd9ea56e8f 100644 --- a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql @@ -6,18 +6,18 @@ CREATE TABLE test_buffer_table ) ENGINE = Buffer('', '', 1, 1000000, 1000000, 1000000, 1000000, 1000000, 1000000); -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); INSERT INTO test_buffer_table SELECT number FROM numbers(1000); -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); OPTIMIZE TABLE test_buffer_table; -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); INSERT INTO test_buffer_table SELECT number FROM numbers(1000); -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); OPTIMIZE TABLE test_buffer_table; -SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table'; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); DROP TABLE test_buffer_table; From 2985def91d29e250b6b6b95f15a132acbe2c7c12 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 27 May 2021 15:54:47 +0300 Subject: [PATCH 231/652] Add tests --- src/Common/Throttler.h | 19 ++ src/IO/copyData.cpp | 2 +- src/IO/copyData.h | 16 +- src/Interpreters/Context.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 7 + src/Storages/StorageReplicatedMergeTree.h | 4 + tests/integration/helpers/network.py | 58 +++++ .../__init__.py | 1 + .../configs/limit_replication_config.xml | 9 + .../test_replicated_fetches_bandwidth/test.py | 214 ++++++++++++++++++ 10 files changed, 322 insertions(+), 12 deletions(-) create mode 100644 tests/integration/test_replicated_fetches_bandwidth/__init__.py create mode 100644 tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml create mode 100644 tests/integration/test_replicated_fetches_bandwidth/test.py diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 172cce783d8..41f5661f67e 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -75,8 +75,13 @@ public: if (desired_ns > elapsed_ns) { UInt64 sleep_ns = desired_ns - elapsed_ns; + + accumulated_sleep += sleep_ns; + sleepForNanoseconds(sleep_ns); + accumulated_sleep -= sleep_ns; + ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_ns / 1000UL); } } @@ -97,6 +102,19 @@ public: count = 0; watch.reset(); + accumulated_sleep = 0; + } + + /// Is throttler already accumulated some sleep time and throttling. + bool isThrottling() const + { + if (accumulated_sleep != 0) + return true; + + if (parent) + return parent->isThrottling(); + + return false; } private: @@ -106,6 +124,7 @@ private: const char * limit_exceeded_exception_message = nullptr; Stopwatch watch {CLOCK_MONOTONIC_COARSE}; std::mutex mutex; + std::atomic accumulated_sleep{0}; /// Used to implement a hierarchy of throttlers std::shared_ptr parent; diff --git a/src/IO/copyData.cpp b/src/IO/copyData.cpp index 428741f2a47..8a044b50de9 100644 --- a/src/IO/copyData.cpp +++ b/src/IO/copyData.cpp @@ -93,7 +93,7 @@ void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, std::function & is_cancelled, ThrottlerPtr throttler) { - copyDataImpl(from, to, true, std::numeric_limits::max(), &is_cancelled, throttler); + copyDataImpl(from, to, false, std::numeric_limits::max(), &is_cancelled, throttler); } void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled, ThrottlerPtr throttler) diff --git a/src/IO/copyData.h b/src/IO/copyData.h index 56244167861..2202f36f79e 100644 --- a/src/IO/copyData.h +++ b/src/IO/copyData.h @@ -14,23 +14,21 @@ class Throttler; using ThrottlerPtr = std::shared_ptr; -/** Copies data from ReadBuffer to WriteBuffer, all that is. - */ +/// Copies data from ReadBuffer to WriteBuffer, all that is. void copyData(ReadBuffer & from, WriteBuffer & to); -/** Copies `bytes` bytes from ReadBuffer to WriteBuffer. If there are no `bytes` bytes, then throws an exception. - */ +/// Copies `bytes` bytes from ReadBuffer to WriteBuffer. If there are no `bytes` bytes, then throws an exception. void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes); -/** The same, with the condition to cancel. - */ +/// The same, with the condition to cancel. void copyData(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled); void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled); -void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled, ThrottlerPtr throttler); -void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled, ThrottlerPtr throttler); - void copyData(ReadBuffer & from, WriteBuffer & to, std::function cancellation_hook); void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, std::function cancellation_hook); +/// Same as above but also use throttler to limit maximum speed +void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic & is_cancelled, ThrottlerPtr throttler); +void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic & is_cancelled, ThrottlerPtr throttler); + } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f1cf89dfaf6..7ee2da05b12 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1637,7 +1637,7 @@ ThrottlerPtr Context::getReplicatedFetchesThrottler() const shared->replicated_fetches_throttler = std::make_shared( settings.max_replicated_fetches_network_bandwidth_for_server); - return shared->replicated_fetches_throttler; + return shared->replicated_fetches_throttler; } ThrottlerPtr Context::getReplicatedSendsThrottler() const @@ -1647,7 +1647,7 @@ ThrottlerPtr Context::getReplicatedSendsThrottler() const shared->replicated_sends_throttler = std::make_shared( settings.max_replicated_sends_network_bandwidth_for_server); - return shared->replicated_fetches_throttler; + return shared->replicated_sends_throttler; } bool Context::hasDistributedDDL() const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d776a51b06d..59c9d603963 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3161,6 +3161,13 @@ bool StorageReplicatedMergeTree::canExecuteFetch(const ReplicatedMergeTreeLogEnt return false; } + if (replicated_fetches_throttler->isThrottling()) + { + disable_reason = fmt::format("Not executing fetch of part {} because fetches have already throttled by network settings " + " or .", entry.new_part_name); + return false; + } + return true; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 93f86e8ae7e..7ba524d565b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -240,11 +240,13 @@ public: /// Get best replica having this partition on S3 String getSharedDataReplica(const IMergeTreeDataPart & part) const; + /// Get throttler for replicated fetches ThrottlerPtr getFetchesThrottler() const { return replicated_fetches_throttler; } + /// Get throttler for replicated sends ThrottlerPtr getSendsThrottler() const { return replicated_sends_throttler; @@ -374,6 +376,8 @@ private: const size_t replicated_fetches_pool_size; + /// Throttlers used in DataPartsExchange to lower maximum fetch/sends + /// speed. ThrottlerPtr replicated_fetches_throttler; ThrottlerPtr replicated_sends_throttler; diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index aa697c63d72..4a64ccee809 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -219,3 +219,61 @@ class _NetworkManager: raise subprocess.CalledProcessError(exit_code, cmd) return output + +# Approximately mesure network I/O speed for interface +class NetThroughput(object): + def __init__(self, node, interface="eth0"): + self.interface = interface + self.node = node + try: + check = subprocess.check_output(f'grep "^ *{self.interface}:" /proc/net/dev', shell=True) + if not check: + raise Exception(f"No such interface {self.interface} found in /proc/net/dev") + except: + raise Exception(f"No such interface {self.interface} found in /proc/net/dev") + + self.current_in = self._get_in_bytes() + self.current_out = self._get_out_bytes() + self.measure_time = time.time() + + def _get_in_bytes(self): + try: + result = self.node.exec_in_container(['bash', '-c', f'awk "/^ *{self.interface}:/"\' {{ if ($1 ~ /.*:[0-9][0-9]*/) {{ sub(/^.*:/, "") ; print $1 }} else {{ print $2 }} }}\' /proc/net/dev']) + except: + raise Exception(f"Cannot receive in bytes from /proc/net/dev for interface {self.interface}") + + try: + return int(result) + except: + raise Exception(f"Got non-numeric in bytes '{result}' from /proc/net/dev for interface {self.interface}") + + def _get_out_bytes(self): + try: + result = self.node.exec_in_container(['bash', '-c', f'awk "/^ *{self.interface}:/"\' {{ if ($1 ~ /.*:[0-9][0-9]*/) {{ print $9 }} else {{ print $10 }} }}\' /proc/net/dev']) + except: + raise Exception(f"Cannot receive out bytes from /proc/net/dev for interface {self.interface}") + + try: + return int(result) + except: + raise Exception(f"Got non-numeric out bytes '{result}' from /proc/net/dev for interface {self.interface}") + + def measure_speed(self, measure='bytes'): + new_in = self._get_in_bytes() + new_out = self._get_out_bytes() + current_time = time.time() + in_speed = (new_in - self.current_in) / (current_time - self.measure_time) + out_speed = (new_out - self.current_out) / (current_time - self.measure_time) + + self.current_out = new_out + self.current_in = new_in + self.measure_time = current_time + + if measure == 'bytes': + return in_speed, out_speed + elif measure == 'kilobytes': + return in_speed / 1024., out_speed / 1024. + elif measure == 'megabytes': + return in_speed / (1024 * 1024), out_speed / (1024 * 1024) + else: + raise Exception(f"Unknown measure {measure}") diff --git a/tests/integration/test_replicated_fetches_bandwidth/__init__.py b/tests/integration/test_replicated_fetches_bandwidth/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_replicated_fetches_bandwidth/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml new file mode 100644 index 00000000000..566c7c07f5c --- /dev/null +++ b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml @@ -0,0 +1,9 @@ + + + + + 5242880 + 10485760 + + + diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py new file mode 100644 index 00000000000..ce1628fecd7 --- /dev/null +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -0,0 +1,214 @@ +#!/usr/bin/env python3 +from helpers.cluster import ClickHouseCluster +import pytest +import random +import string +from helpers.network import NetThroughput +import subprocess +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True) +node3 = cluster.add_instance('node3', user_configs=['configs/limit_replication_config.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def get_random_string(length): + return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + +def test_limited_fetch_single_table(start_cluster): + try: + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE limited_fetch_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetch_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_fetches_network_bandwidth=10485760") + + node2.query("SYSTEM STOP FETCHES limited_fetch_table") + + for i in range(5): + node1.query("INSERT INTO limited_fetch_table SELECT {}, '{}' FROM numbers(300)".format(i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n2_net = NetThroughput(node2) + + node2.query("SYSTEM START FETCHES limited_fetch_table") + n2_fetch_speed = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n2_in, n2_out = n2_net.measure_speed('megabytes') + print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical")) + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N2] input:", n2_in, 'MB/s', "output:", n2_out, "MB/s") + n2_fetch_speed.append(n2_in) + time.sleep(0.5) + + for s in n2_fetch_speed: + if s > 20: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert False, "We exceeded max fetch speed for more than 10MB/s. Must be around 10 (+- 5), got " + str(s) + # cannot check lower bound because it can vary depending on host/build etc. + else: + print("Fetch speed OK") + + finally: + for i, node in enumerate([node1, node2]): + node.query(f"DROP TABLE IF EXISTS limited_fetch_table SYNC") + +def test_limited_send_single_table(start_cluster): + try: + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE limited_send_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetch_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_sends_network_bandwidth=5242880") + + node1.query("SYSTEM STOP REPLICATED SENDS limited_send_table") + + for i in range(5): + node1.query("INSERT INTO limited_send_table SELECT {}, '{}' FROM numbers(150)".format(i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n2_net = NetThroughput(node2) + + node1.query("SYSTEM START REPLICATED SENDS limited_send_table") + n1_sends_speed = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n2_in, n2_out = n2_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N2] input:", n2_in, 'MB/s', "output:", n2_out, "MB/s") + n1_sends_speed.append(n1_out) + time.sleep(0.5) + + for s in n1_sends_speed: + if s > 10: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert False, "We exceeded max send speed for more than 5MB/s. Must be around 5 (+- 5), got " + str(s) + # cannot check lower bound because it can vary depending on host/build etc. + else: + print("Send speed OK") + + finally: + for i, node in enumerate([node1, node2]): + node.query(f"DROP TABLE IF EXISTS limited_send_table SYNC") + + +def test_limited_fetches_for_server(start_cluster): + try: + for i, node in enumerate([node1, node3]): + for j in range(5): + node.query(f"CREATE TABLE limited_fetches{j}(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetches{j}', '{i}') ORDER BY tuple() PARTITION BY key") + + for j in range(5): + node3.query(f"SYSTEM STOP FETCHES limited_fetches{j}") + for i in range(5): + node1.query("INSERT INTO limited_fetches{} SELECT {}, '{}' FROM numbers(50)".format(j, i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n3_net = NetThroughput(node3) + + for j in range(5): + node3.query(f"SYSTEM START FETCHES limited_fetches{j}") + + n3_fetches_speed = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n3_in, n3_out = n3_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N3] input:", n3_in, 'MB/s', "output:", n3_out, "MB/s") + n3_fetches_speed.append(n3_in) + time.sleep(0.5) + + for s in n3_fetches_speed: + if s > 20: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert False, "We exceeded max fetch speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(s) + # cannot check lower bound because it can vary depending on host/build etc. + else: + print("Fetch speed OK") + + finally: + for i, node in enumerate([node1, node3]): + for j in range(5): + node.query(f"DROP TABLE IF EXISTS limited_fetches{i} SYNC") + + +def test_limited_sends_for_server(start_cluster): + try: + for i, node in enumerate([node1, node3]): + for j in range(5): + node.query(f"CREATE TABLE limited_sends{j}(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_sends{j}', '{i}') ORDER BY tuple() PARTITION BY key") + + for j in range(5): + node3.query(f"SYSTEM STOP REPLICATED SENDS limited_sends{j}") + for i in range(5): + node3.query("INSERT INTO limited_sends{} SELECT {}, '{}' FROM numbers(50)".format(j, i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n3_net = NetThroughput(node3) + + for j in range(5): + node3.query(f"SYSTEM START REPLICATED SENDS limited_sends{j}") + + n3_sends_speed = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n3_in, n3_out = n3_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N3] input:", n3_in, 'MB/s', "output:", n3_out, "MB/s") + n3_sends_speed.append(n3_out) + time.sleep(0.5) + + for s in n3_sends_speed: + if s > 30: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert False, "We exceeded max send speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(s) + # cannot check lower bound because it can vary depending on host/build etc. + else: + print("Send speed OK") + + finally: + for i, node in enumerate([node1, node3]): + for j in range(5): + node.query(f"DROP TABLE IF EXISTS limited_sends{i} SYNC") + + +def test_should_execute_fetch(start_cluster): + try: + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE should_execute_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/should_execute_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_fetches_network_bandwidth=3505253") + + node2.query("SYSTEM STOP FETCHES should_execute_table") + + for i in range(3): + node1.query("INSERT INTO should_execute_table SELECT {}, '{}' FROM numbers(200)".format(i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n2_net = NetThroughput(node2) + + node2.query("SYSTEM START FETCHES should_execute_table") + + for i in range(10): + node1.query("INSERT INTO should_execute_table SELECT {}, '{}' FROM numbers(3)".format(i, get_random_string(104857))) + + n2_fetch_speed = [] + replication_queue_data = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n2_in, n2_out = n2_net.measure_speed('megabytes') + fetches_count = node2.query("SELECT count() FROM system.replicated_fetches") + if fetches_count == "0\n": + break + + print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical")) + print("Fetches count", fetches_count) + replication_queue_data.append(node2.query("SELECT count() FROM system.replication_queue WHERE postpone_reason like '%fetches have already throttled%'")) + n2_fetch_speed.append(n2_in) + time.sleep(0.5) + + print(node2.query("SELECT * FROM system.replicated_fetches FORMAT Vertical")) + node2.query("SYSTEM SYNC REPLICA should_execute_table") + assert any(int(f.strip()) != 0 for f in replication_queue_data) + assert node2.query("SELECT COUNT() FROM should_execute_table") == "630\n" + finally: + for i, node in enumerate([node1, node2]): + node.query(f"DROP TABLE IF EXISTS should_execute_table SYNC") From 6860946d3c95ccbd2d66b8730359845f0e75d4af Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 27 May 2021 16:00:08 +0300 Subject: [PATCH 232/652] Cleanup in test --- .../test_replicated_fetches_bandwidth/test.py | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py index ce1628fecd7..1d19323d298 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/test.py +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -56,8 +56,8 @@ def test_limited_fetch_single_table(start_cluster): print("Fetch speed OK") finally: - for i, node in enumerate([node1, node2]): - node.query(f"DROP TABLE IF EXISTS limited_fetch_table SYNC") + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS limited_fetch_table SYNC") def test_limited_send_single_table(start_cluster): try: @@ -90,8 +90,8 @@ def test_limited_send_single_table(start_cluster): print("Send speed OK") finally: - for i, node in enumerate([node1, node2]): - node.query(f"DROP TABLE IF EXISTS limited_send_table SYNC") + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS limited_send_table SYNC") def test_limited_fetches_for_server(start_cluster): @@ -128,9 +128,9 @@ def test_limited_fetches_for_server(start_cluster): print("Fetch speed OK") finally: - for i, node in enumerate([node1, node3]): + for node in [node1, node3]: for j in range(5): - node.query(f"DROP TABLE IF EXISTS limited_fetches{i} SYNC") + node.query(f"DROP TABLE IF EXISTS limited_fetches{j} SYNC") def test_limited_sends_for_server(start_cluster): @@ -167,9 +167,9 @@ def test_limited_sends_for_server(start_cluster): print("Send speed OK") finally: - for i, node in enumerate([node1, node3]): + for node in [node1, node3]: for j in range(5): - node.query(f"DROP TABLE IF EXISTS limited_sends{i} SYNC") + node.query(f"DROP TABLE IF EXISTS limited_sends{j} SYNC") def test_should_execute_fetch(start_cluster): @@ -210,5 +210,5 @@ def test_should_execute_fetch(start_cluster): assert any(int(f.strip()) != 0 for f in replication_queue_data) assert node2.query("SELECT COUNT() FROM should_execute_table") == "630\n" finally: - for i, node in enumerate([node1, node2]): - node.query(f"DROP TABLE IF EXISTS should_execute_table SYNC") + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS should_execute_table SYNC") From cbdf3752efc2bebdd1f207e2d2363c9a4d9fa661 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 May 2021 16:40:33 +0300 Subject: [PATCH 233/652] Part 3. --- .../QueryPlan/ReadFromMergeTree.cpp | 261 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 26 +- src/Storages/MergeTree/MergeTreeData.cpp | 53 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2367 +++++++---------- .../MergeTree/MergeTreeDataSelectExecutor.h | 146 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +- src/Storages/SelectQueryInfo.h | 4 +- 7 files changed, 1116 insertions(+), 1746 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 67377a54c34..6d6ee43acb3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -3,6 +3,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -11,8 +14,6 @@ #include #include #include -#include -#include #include #include #include @@ -88,38 +89,35 @@ size_t minMarksForConcurrentRead( } ReadFromMergeTree::ReadFromMergeTree( - SelectQueryInfo query_info_, - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + const SelectQueryInfo & query_info_, + const PartitionIdToMaxBlock * max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, - const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, - //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, - size_t num_streams_) + Poco::Logger * log_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( - metadata_snapshot_->getSampleBlockForColumns(real_column_names_, storage_.getVirtuals(), storage_.getStorageID()), + metadata_snapshot_->getSampleBlockForColumns(real_column_names_, data_.getVirtuals(), data_.getStorageID()), prewhere_info_, - storage_.getPartitionValueType(), + data_.getPartitionValueType(), virt_column_names_)}) , query_info(std::move(query_info_)) , max_block_numbers_to_read(max_block_numbers_to_read_) , context(std::move(context_)) , data(data_) - , storage(storage_) , metadata_snapshot(std::move(metadata_snapshot_)) , metadata_snapshot_base(std::move(metadata_snapshot_base_)) , real_column_names(std::move(real_column_names_)) - , parts(std::move(parts_)) + , prepared_parts(std::move(parts_)) , prewhere_info(std::move(prewhere_info_)) , virt_column_names(std::move(virt_column_names_)) , settings(std::move(settings_)) - , num_streams(num_streams_) + , log(log_) { } @@ -142,7 +140,7 @@ Pipe ReadFromMergeTree::readFromPool( sum_marks, min_marks_for_concurrent_read, std::move(parts_with_range), - storage, + data, metadata_snapshot, prewhere_info, true, @@ -151,7 +149,7 @@ Pipe ReadFromMergeTree::readFromPool( settings.preferred_block_size_bytes, false); - auto * logger = &Poco::Logger::get(storage.getLogName() + " (SelectExecutor)"); + auto * logger = &Poco::Logger::get(data.getLogName() + " (SelectExecutor)"); LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, used_max_streams); for (size_t i = 0; i < used_max_streams; ++i) @@ -159,7 +157,7 @@ Pipe ReadFromMergeTree::readFromPool( auto source = std::make_shared( i, pool, min_marks_for_concurrent_read, settings.max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, - storage, metadata_snapshot, use_uncompressed_cache, + data, metadata_snapshot, use_uncompressed_cache, prewhere_info, settings.reader_settings, virt_column_names); if (i == 0) @@ -178,7 +176,7 @@ template ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache) { return std::make_shared( - storage, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, + data, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, true, settings.reader_settings, virt_column_names, part.part_index_in_query); } @@ -212,7 +210,7 @@ Pipe ReadFromMergeTree::read( RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { - if (read_type == ReadType::Default && num_streams > 1) + if (read_type == ReadType::Default && used_max_streams > 1) return readFromPool(parts_with_range, required_columns, used_max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache); @@ -225,126 +223,6 @@ Pipe ReadFromMergeTree::read( return pipe; } -static std::optional> filterPartsByVirtualColumns( - const MergeTreeData & data, - MergeTreeData::DataPartsVector & parts, - ASTPtr & query, - ContextPtr context) -{ - std::unordered_set part_values; - ASTPtr expression_ast; - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); - - // Generate valid expressions for filtering - VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); - - // If there is still something left, fill the virtual block and do the filtering. - if (expression_ast) - { - virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); - VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); - return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - } - - return {}; -} - -static void filterPartsByPartition( - StorageMetadataPtr & metadata_snapshot, - const MergeTreeData & data, - SelectQueryInfo & query_info, - ContextPtr & context, - ContextPtr & query_context, - MergeTreeData::DataPartsVector & parts, - const std::optional> & part_values, - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read, - Poco::Logger * log, - ReadFromMergeTree::IndexStats & index_stats) -{ - const Settings & settings = context->getSettingsRef(); - std::optional partition_pruner; - std::optional minmax_idx_condition; - DataTypes minmax_columns_types; - if (metadata_snapshot->hasPartitionKey()) - { - const auto & partition_key = metadata_snapshot->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - - minmax_idx_condition.emplace( - query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); - partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); - - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) - { - String msg = "Neither MinMax index by columns ("; - bool first = true; - for (const String & col : minmax_columns_names) - { - if (first) - first = false; - else - msg += ", "; - msg += col; - } - msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; - - throw Exception(msg, ErrorCodes::INDEX_NOT_USED); - } - } - - MergeTreeDataSelectExecutor::PartFilterCounters part_filter_counters; - if (query_context->getSettingsRef().allow_experimental_query_deduplication) - MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( - parts, - part_values, - data.getPinnedPartUUIDs(), - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - query_context, - part_filter_counters, - log); - else - MergeTreeDataSelectExecutor::selectPartsToRead( - parts, - part_values, - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - part_filter_counters); - - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::None, - .num_parts_after = part_filter_counters.num_initial_selected_parts, - .num_granules_after = part_filter_counters.num_initial_selected_granules}); - - if (minmax_idx_condition) - { - auto description = minmax_idx_condition->getDescription(); - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::MinMax, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_minmax, - .num_granules_after = part_filter_counters.num_granules_after_minmax}); - LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); - } - - if (partition_pruner) - { - auto description = partition_pruner->getKeyCondition().getDescription(); - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Partition, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, - .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); - } -} - Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, const Names & column_names) @@ -392,7 +270,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( if (0 == sum_marks) return {}; - size_t used_num_streams = num_streams; + size_t used_num_streams = settings.num_streams; if (used_num_streams > 1) { /// Reduce the number of num_streams if the data is small. @@ -506,12 +384,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return new_ranges; }; - const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; + const size_t min_marks_per_stream = (sum_marks - 1) / settings.num_streams + 1; bool need_preliminary_merge = (parts_with_ranges.size() > q_settings.read_in_order_two_level_merge_threshold); Pipes pipes; - for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i) + for (size_t i = 0; i < settings.num_streams && !parts_with_ranges.empty(); ++i) { size_t need_marks = min_marks_per_stream; RangesInDataParts new_parts; @@ -577,7 +455,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( : ReadFromMergeTree::ReadType::InReverseOrder; pipes.emplace_back(read(std::move(new_parts), column_names, read_type, - num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + settings.num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); } if (need_preliminary_merge) @@ -755,7 +633,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - size_t used_num_streams = num_streams; + size_t used_num_streams = settings.num_streams; if (used_num_streams > q_settings.max_final_threads) used_num_streams = q_settings.max_final_threads; @@ -966,9 +844,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { + auto parts = std::move(prepared_parts); size_t total_parts = parts.size(); - auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); + auto part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); if (part_values && part_values->empty()) { pipeline.init(Pipe(std::make_shared(getOutputStream().header))); @@ -1001,7 +880,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build const auto & select = query_info.query->as(); auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; - filterPartsByPartition( + MergeTreeDataSelectExecutor::filterPartsByPartition( metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); bool sample_factor_column_queried = false; @@ -1023,16 +902,17 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build for (const auto & part : parts) total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); - auto parts_with_ranges = MergeTreeDataSelectExecutor::filterParts( - parts, + auto parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( + std::move(parts), metadata_snapshot, query_info, context, key_condition, settings.reader_settings, log, - num_streams, - index_stats); + settings.num_streams, + index_stats, + true); size_t sum_marks_pk = total_marks_pk; for (const auto & stat : index_stats) @@ -1132,12 +1012,69 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build column_names_to_read); } + if (pipe.empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + if (sampling.use_sampling) + { + auto sampling_actions = std::make_shared(sampling.filter_expression); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + sampling_actions, + sampling.filter_function->getColumnName(), + false); + }); + } + + if (result_projection) + { + auto projection_actions = std::make_shared(result_projection); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, projection_actions); + }); + } + + /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. + if (sample_factor_column_queried) + { + ColumnWithTypeAndName column; + column.name = "_sample_factor"; + column.type = std::make_shared(); + column.column = column.type->createColumnConst(0, Field(sampling.used_sample_factor)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_action = std::make_shared(adding_column_dag); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, adding_column_action); + }); + } + + // TODO There seems to be no place initializing remove_columns_actions + if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) + { + auto remove_columns_action = std::make_shared( + query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, remove_columns_action); + }); + } + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); // Attach QueryIdHolder if needed if (!query_id.empty()) - pipe.addQueryIdHolder(std::make_shared(query_id, storage)); + pipe.addQueryIdHolder(std::make_shared(query_id, data)); pipeline.init(std::move(pipe)); } @@ -1161,20 +1098,20 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type) __builtin_unreachable(); } -static const char * readTypeToString(ReadFromMergeTree::ReadType type) -{ - switch (type) - { - case ReadFromMergeTree::ReadType::Default: - return "Default"; - case ReadFromMergeTree::ReadType::InOrder: - return "InOrder"; - case ReadFromMergeTree::ReadType::InReverseOrder: - return "InReverseOrder"; - } +// static const char * readTypeToString(ReadFromMergeTree::ReadType type) +// { +// switch (type) +// { +// case ReadFromMergeTree::ReadType::Default: +// return "Default"; +// case ReadFromMergeTree::ReadType::InOrder: +// return "InOrder"; +// case ReadFromMergeTree::ReadType::InReverseOrder: +// return "InReverseOrder"; +// } - __builtin_unreachable(); -} +// __builtin_unreachable(); +// } void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index b3366afb118..e9341e46770 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -3,11 +3,13 @@ #include #include #include -#include +//#include namespace DB { +using PartitionIdToMaxBlock = std::unordered_map; + /// This step is created to read from MergeTree* table. /// For now, it takes a list of parts and creates source from it. class ReadFromMergeTree final : public ISourceStep @@ -42,9 +44,10 @@ public: struct Settings { UInt64 max_block_size; + size_t num_streams; size_t preferred_block_size_bytes; size_t preferred_max_column_in_block_size_bytes; - size_t min_marks_for_concurrent_read; + //size_t min_marks_for_concurrent_read; bool use_uncompressed_cache; bool force_primary_key; @@ -68,21 +71,18 @@ public: }; ReadFromMergeTree( - SelectQueryInfo query_info_, - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + const SelectQueryInfo & query_info_, + const PartitionIdToMaxBlock * max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, - const MergeTreeData & storage_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, Names real_column_names_, MergeTreeData::DataPartsVector parts_, - //IndexStatPtr index_stats_, PrewhereInfoPtr prewhere_info_, Names virt_column_names_, Settings settings_, - size_t num_streams_, - //ReadType read_type_ + Poco::Logger * log_ ); String getName() const override { return "ReadFromMergeTree"; } @@ -97,23 +97,19 @@ public: private: SelectQueryInfo query_info; - const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read; + const PartitionIdToMaxBlock * max_block_numbers_to_read; ContextPtr context; const MergeTreeData & data; - const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; StorageMetadataPtr metadata_snapshot_base; Names real_column_names; - MergeTreeData::DataPartsVector parts; - IndexStats index_stats; + MergeTreeData::DataPartsVector prepared_parts; PrewhereInfoPtr prewhere_info; + IndexStats index_stats; Names virt_column_names; Settings settings; - size_t num_streams; - //ReadType read_type; - Poco::Logger * log; Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 41adca37c60..e16bbb640e2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3839,21 +3839,18 @@ static void selectBestProjection( if (projection_parts.empty()) return; - candidate.merge_tree_data_select_base_cache = std::make_unique(); - candidate.merge_tree_data_select_projection_cache = std::make_unique(); - reader.readFromParts( + //candidate.merge_tree_data_select_base_cache = std::make_unique(); + //candidate.merge_tree_data_select_projection_cache = std::make_unique(); + auto sum_marks = reader.estimateNumMarksToRead( projection_parts, candidate.required_columns, metadata_snapshot, candidate.desc->metadata, query_info, // TODO syntax_analysis_result set in index query_context, - 0, // max_block_size is unused when getting cache settings.max_threads, - max_added_blocks, - candidate.merge_tree_data_select_projection_cache.get()); + max_added_blocks); - size_t sum_marks = candidate.merge_tree_data_select_projection_cache->sum_marks; if (normal_parts.empty()) { // All parts are projection parts which allows us to use in_order_optimization. @@ -3862,18 +3859,15 @@ static void selectBestProjection( } else { - reader.readFromParts( + sum_marks += reader.estimateNumMarksToRead( normal_parts, required_columns, metadata_snapshot, metadata_snapshot, query_info, // TODO syntax_analysis_result set in index query_context, - 0, // max_block_size is unused when getting cache settings.max_threads, - max_added_blocks, - candidate.merge_tree_data_select_base_cache.get()); - sum_marks += candidate.merge_tree_data_select_base_cache->sum_marks; + max_added_blocks); } // We choose the projection with least sum_marks to read. @@ -4101,7 +4095,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( if (!candidates.empty()) { // First build a MergeTreeDataSelectCache to check if a projection is indeed better than base - query_info.merge_tree_data_select_cache = std::make_unique(); + // query_info.merge_tree_data_select_cache = std::make_unique(); std::unique_ptr max_added_blocks; if (settings.select_sequential_consistency) @@ -4112,21 +4106,10 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( auto parts = getDataPartsVector(); MergeTreeDataSelectExecutor reader(*this); - reader.readFromParts( - parts, - analysis_result.required_columns, - metadata_snapshot, - metadata_snapshot, - query_info, // TODO syntax_analysis_result set in index - query_context, - 0, // max_block_size is unused when getting cache - settings.max_threads, - max_added_blocks.get(), - query_info.merge_tree_data_select_cache.get()); - // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. - size_t min_sum_marks = query_info.merge_tree_data_select_cache->sum_marks + 1; ProjectionCandidate * selected_candidate = nullptr; + size_t min_sum_marks = std::numeric_limits::max(); + bool has_ordinary_projection = false; /// Favor aggregate projections for (auto & candidate : candidates) { @@ -4145,11 +4128,27 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( selected_candidate, min_sum_marks); } + else + has_ordinary_projection = true; } /// Select the best normal projection if no aggregate projection is available - if (!selected_candidate) + if (!selected_candidate && has_ordinary_projection) { + min_sum_marks = reader.estimateNumMarksToRead( + parts, + analysis_result.required_columns, + metadata_snapshot, + metadata_snapshot, + query_info, // TODO syntax_analysis_result set in index + query_context, + settings.max_threads, + max_added_blocks.get()); + + // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. + // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. + min_sum_marks += 1; + for (auto & candidate : candidates) { if (candidate.desc->type == ProjectionDescription::Type::Normal) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 49bcf751bd1..8a3550fc511 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -47,7 +47,6 @@ #include #include #include -#include #include namespace ProfileEvents @@ -143,6 +142,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const PartitionIdToMaxBlock * max_block_numbers_to_read) const { const auto & settings = context->getSettingsRef(); + auto parts = data.getDataPartsVector(); if (!query_info.projection) { if (settings.allow_experimental_projection_optimization && settings.force_optimize_projection @@ -150,7 +150,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( throw Exception("No projection is used when allow_experimental_projection_optimization = 1", ErrorCodes::PROJECTION_NOT_USED); return readFromParts( - data.getDataPartsVector(), + parts, column_names_to_return, metadata_snapshot, metadata_snapshot, @@ -158,8 +158,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( context, max_block_size, num_streams, - max_block_numbers_to_read, - query_info.merge_tree_data_select_cache.get()); + max_block_numbers_to_read); } LOG_DEBUG( @@ -168,21 +167,33 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( ProjectionDescription::typeToString(query_info.projection->desc->type), query_info.projection->desc->name); - if (query_info.projection->merge_tree_data_select_base_cache->sum_marks - + query_info.projection->merge_tree_data_select_projection_cache->sum_marks - == 0) - return std::make_unique(); + // if (query_info.projection->merge_tree_data_select_base_cache->sum_marks + // + query_info.projection->merge_tree_data_select_projection_cache->sum_marks + // == 0) + // return std::make_unique(); + + MergeTreeData::DataPartsVector projection_parts; + MergeTreeData::DataPartsVector normal_parts; + for (const auto & part : parts) + { + const auto & projections = part->getProjectionParts(); + auto it = projections.find(query_info.projection->desc->name); + if (it != projections.end()) + projection_parts.push_back(it->second); + else + normal_parts.push_back(part); + } Pipes pipes; Pipe projection_pipe; Pipe ordinary_pipe; const auto & given_select = query_info.query->as(); - if (query_info.projection->merge_tree_data_select_projection_cache->sum_marks > 0) + if (!projection_parts.empty()) { LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", ")); auto plan = readFromParts( - {}, + projection_parts, query_info.projection->required_columns, metadata_snapshot, query_info.projection->desc->metadata, @@ -190,8 +201,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( context, max_block_size, num_streams, - max_block_numbers_to_read, - query_info.projection->merge_tree_data_select_projection_cache.get()); + max_block_numbers_to_read); if (plan) { @@ -225,9 +235,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( } } - if (query_info.projection->merge_tree_data_select_base_cache->sum_marks > 0) + if (!normal_parts.empty()) { - auto storage_from_base_parts_of_projection = StorageFromBasePartsOfProjection::create(data, metadata_snapshot); + auto storage_from_base_parts_of_projection = StorageFromMergeTreeDataPart::create(std::move(normal_parts)); auto ast = query_info.projection->desc->query_ast->clone(); auto & select = ast->as(); if (given_select.where()) @@ -382,7 +392,6 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( ContextPtr context) { const Settings & settings = context->getSettingsRef(); - Float64 used_sample_factor = 1; /// Sampling. MergeTreeDataSelectSamplingData sampling; @@ -485,7 +494,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (sampling.use_sampling) { if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) - used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); + sampling.used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; const auto & sampling_key = metadata_snapshot->getSamplingKey(); @@ -633,16 +642,137 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } -RangesInDataParts MergeTreeDataSelectExecutor::filterParts( +std::optional> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( + const MergeTreeData & data, MergeTreeData::DataPartsVector & parts, + const ASTPtr & query, + ContextPtr context) +{ + std::unordered_set part_values; + ASTPtr expression_ast; + auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); + + // Generate valid expressions for filtering + VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); + + // If there is still something left, fill the virtual block and do the filtering. + if (expression_ast) + { + virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); + VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); + return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + } + + return {}; +} + +void MergeTreeDataSelectExecutor::filterPartsByPartition( + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & data, + const SelectQueryInfo & query_info, + ContextPtr & context, + ContextPtr & query_context, + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, + const PartitionIdToMaxBlock * max_block_numbers_to_read, + Poco::Logger * log, + ReadFromMergeTree::IndexStats & index_stats) +{ + const Settings & settings = context->getSettingsRef(); + std::optional partition_pruner; + std::optional minmax_idx_condition; + DataTypes minmax_columns_types; + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + minmax_idx_condition.emplace( + query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); + partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); + + if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) + { + String msg = "Neither MinMax index by columns ("; + bool first = true; + for (const String & col : minmax_columns_names) + { + if (first) + first = false; + else + msg += ", "; + msg += col; + } + msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; + + throw Exception(msg, ErrorCodes::INDEX_NOT_USED); + } + } + + PartFilterCounters part_filter_counters; + if (query_context->getSettingsRef().allow_experimental_query_deduplication) + selectPartsToReadWithUUIDFilter( + parts, + part_values, + data.getPinnedPartUUIDs(), + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + query_context, + part_filter_counters, + log); + else + selectPartsToRead( + parts, + part_values, + minmax_idx_condition, + minmax_columns_types, + partition_pruner, + max_block_numbers_to_read, + part_filter_counters); + + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::None, + .num_parts_after = part_filter_counters.num_initial_selected_parts, + .num_granules_after = part_filter_counters.num_initial_selected_granules}); + + if (minmax_idx_condition) + { + auto description = minmax_idx_condition->getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::MinMax, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_minmax, + .num_granules_after = part_filter_counters.num_granules_after_minmax}); + LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); + } + + if (partition_pruner) + { + auto description = partition_pruner->getKeyCondition().getDescription(); + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Partition, + .condition = std::move(description.condition), + .used_keys = std::move(description.used_keys), + .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, + .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); + } +} + +RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( + MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - SelectQueryInfo & query_info, + const SelectQueryInfo & query_info, ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, - ReadFromMergeTree::IndexStats & index_stats) + ReadFromMergeTree::IndexStats & index_stats, + bool use_skip_indexes) { RangesInDataParts parts_with_ranges(parts.size()); const Settings & settings = context->getSettingsRef(); @@ -665,15 +795,18 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterParts( }; std::list useful_indices; - for (const auto & index : metadata_snapshot->getSecondaryIndices()) + if (use_skip_indexes) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } } - if (settings.force_data_skipping_indices.changed) + if (use_skip_indexes && settings.force_data_skipping_indices.changed) { const auto & indices = settings.force_data_skipping_indices.toString(); @@ -860,7 +993,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterParts( return parts_with_ranges; } -void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_range, ContextPtr & context) +String MergeTreeDataSelectExecutor::checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context) { const auto & settings = context->getSettingsRef(); // Check limitations. query_id is used as the quota RAII's resource key. @@ -872,7 +1005,7 @@ void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_rang if (max_partitions_to_read > 0) { std::set partitions; - for (auto & part_with_ranges : parts_with_ranges) + for (const auto & part_with_ranges : parts_with_ranges) partitions.insert(part_with_ranges.data_part->info.partition_id); if (partitions.size() > size_t(max_partitions_to_read)) throw Exception( @@ -896,33 +1029,18 @@ void checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_rang } } } + + return query_id; } -QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( - MergeTreeData::DataPartsVector parts, +static void selectColumnNames( const Names & column_names_to_return, - const StorageMetadataPtr & metadata_snapshot_base, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - ContextPtr context, - const UInt64 max_block_size, - const unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read, - MergeTreeDataSelectCache * cache) const + const MergeTreeData & data, + Names & real_column_names, + Names & virt_column_names, + bool & sample_factor_column_queried) { - bool use_cache = cache && cache->use_cache; - - /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. - /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. - Names virt_column_names; - Names real_column_names; - - size_t total_parts = parts.size(); - if (!use_cache && total_parts == 0) - return std::make_unique(); - - bool sample_factor_column_queried = false; - Float64 used_sample_factor = 1; + sample_factor_column_queried = false; for (const String & name : column_names_to_return) { @@ -964,395 +1082,112 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( real_column_names.push_back(name); } } +} - // Filter parts by virtual columns. - std::unordered_set part_values; - if (!use_cache) - { - ASTPtr expression_ast; - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); +size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( + MergeTreeData::DataPartsVector parts, + const Names & column_names_to_return, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + unsigned num_streams, + const PartitionIdToMaxBlock * max_block_numbers_to_read) const +{ + size_t total_parts = parts.size(); + if (total_parts == 0) + return 0; - // Generate valid expressions for filtering - VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, context, virtual_columns_block, expression_ast); + Names real_column_names; + Names virt_column_names; + /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. + /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. + bool sample_factor_column_queried = false; - // If there is still something left, fill the virtual block and do the filtering. - if (expression_ast) - { - virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); - VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); - part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - if (part_values.empty()) - return std::make_unique(); - } - } - // At this point, empty `part_values` means all parts. + selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - const Settings & settings = context->getSettingsRef(); - NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); + auto part_values = filterPartsByVirtualColumns(data, parts, query_info.query, context); + if (part_values && part_values->empty()) + return 0; /// If there are only virtual columns in the query, you must request at least one non-virtual one. if (real_column_names.empty()) + { + NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + } metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); - // Build and check if primary key is used when necessary - std::optional key_condition; - if (!use_cache) - { - const auto & primary_key = metadata_snapshot->getPrimaryKey(); - Names primary_key_columns = primary_key.column_names; - key_condition.emplace(query_info, context, primary_key_columns, primary_key.expression); + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + Names primary_key_columns = primary_key.column_names; + KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); - if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) - { - throw Exception( - ErrorCodes::INDEX_NOT_USED, - "Primary key ({}) is not used and setting 'force_primary_key' is set.", - fmt::join(primary_key_columns, ", ")); - } - LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); + if (key_condition.alwaysUnknownOrTrue()) + { + size_t total_marks = 0; + for (const auto & part : parts) + total_marks += part->index_granularity.getMarksCountWithoutFinal(); + + return total_marks; } const auto & select = query_info.query->as(); auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; - auto index_stats = use_cache ? std::move(cache->index_stats) : std::make_unique(); + ReadFromMergeTree::IndexStats index_stats; - // Select parts to read and do partition pruning via partition value and minmax indices - if (!use_cache) - { - std::optional partition_pruner; - std::optional minmax_idx_condition; - DataTypes minmax_columns_types; - if (metadata_snapshot_base->hasPartitionKey()) - { - const auto & partition_key = metadata_snapshot_base->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + filterPartsByPartition( + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); - minmax_idx_condition.emplace( - query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); - partition_pruner.emplace(metadata_snapshot_base->getPartitionKey(), query_info, context, false /* strict */); + auto sampling = MergeTreeDataSelectExecutor::getSampling( + select, parts, metadata_snapshot, key_condition, + data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) - { - String msg = "Neither MinMax index by columns ("; - bool first = true; - for (const String & col : minmax_columns_names) - { - if (first) - first = false; - else - msg += ", "; - msg += col; - } - msg += ") nor partition expr is used and setting 'force_index_by_date' is set"; + if (sampling.read_nothing) + return 0; - throw Exception(msg, ErrorCodes::INDEX_NOT_USED); - } - } + /// Do not init. Ther are not used (cause skip index is ignored) + MergeTreeReaderSettings reader_settings; - PartFilterCounters part_filter_counters; - if (query_context->getSettingsRef().allow_experimental_query_deduplication) - selectPartsToReadWithUUIDFilter( - parts, - part_values, - data.getPinnedPartUUIDs(), - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - query_context, - part_filter_counters, - log); - else - selectPartsToRead( - parts, - part_values, - minmax_idx_condition, - minmax_columns_types, - partition_pruner, - max_block_numbers_to_read, - part_filter_counters); + auto parts_with_ranges = filterPartsByPrimaryKeyAndSkipIndexes( + std::move(parts), + metadata_snapshot, + query_info, + context, + key_condition, + reader_settings, + log, + num_streams, + index_stats, + false); - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::None, - .num_parts_after = part_filter_counters.num_initial_selected_parts, - .num_granules_after = part_filter_counters.num_initial_selected_granules}); + return index_stats.back().num_granules_after; +} - if (minmax_idx_condition) - { - auto description = minmax_idx_condition->getDescription(); - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::MinMax, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_minmax, - .num_granules_after = part_filter_counters.num_granules_after_minmax}); - LOG_DEBUG(log, "MinMax index condition: {}", minmax_idx_condition->toString()); - } +QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( + MergeTreeData::DataPartsVector parts, + const Names & column_names_to_return, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + const UInt64 max_block_size, + const unsigned num_streams, + const PartitionIdToMaxBlock * max_block_numbers_to_read) const +{ + size_t total_parts = parts.size(); + if (total_parts == 0) + return std::make_unique(); - if (partition_pruner) - { - auto description = partition_pruner->getKeyCondition().getDescription(); - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Partition, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = part_filter_counters.num_parts_after_partition_pruner, - .num_granules_after = part_filter_counters.num_granules_after_partition_pruner}); - } - } + Names real_column_names; + Names virt_column_names; + /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. + /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. + bool sample_factor_column_queried = false; - /// Sampling. - MergeTreeDataSelectSamplingData sampling = use_cache ? std::move(cache->sampling) : MergeTreeDataSelectSamplingData{}; - if (!use_cache) - { - assert(key_condition.has_value()); + selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - RelativeSize relative_sample_size = 0; - RelativeSize relative_sample_offset = 0; - - auto select_sample_size = select.sampleSize(); - auto select_sample_offset = select.sampleOffset(); - - if (select_sample_size) - { - relative_sample_size.assign( - select_sample_size->as().ratio.numerator, - select_sample_size->as().ratio.denominator); - - if (relative_sample_size < 0) - throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - relative_sample_offset = 0; - if (select_sample_offset) - relative_sample_offset.assign( - select_sample_offset->as().ratio.numerator, - select_sample_offset->as().ratio.denominator); - - if (relative_sample_offset < 0) - throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - /// Convert absolute value of the sampling (in form `SAMPLE 1000000` - how many rows to - /// read) into the relative `SAMPLE 0.1` (how much data to read). - size_t approx_total_rows = 0; - if (relative_sample_size > 1 || relative_sample_offset > 1) - approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, *key_condition, settings); //-V1007 - - if (relative_sample_size > 1) - { - relative_sample_size = convertAbsoluteSampleSizeToRelative(select_sample_size, approx_total_rows); - LOG_DEBUG(log, "Selected relative sample size: {}", toString(relative_sample_size)); - } - - /// SAMPLE 1 is the same as the absence of SAMPLE. - if (relative_sample_size == RelativeSize(1)) - relative_sample_size = 0; - - if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size) - throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (relative_sample_offset > 1) - { - relative_sample_offset = convertAbsoluteSampleSizeToRelative(select_sample_offset, approx_total_rows); - LOG_DEBUG(log, "Selected relative sample offset: {}", toString(relative_sample_offset)); - } - } - - /** Which range of sampling key values do I need to read? - * First, in the whole range ("universe") we select the interval - * of relative `relative_sample_size` size, offset from the beginning by `relative_sample_offset`. - * - * Example: SAMPLE 0.4 OFFSET 0.3 - * - * [------********------] - * ^ - offset - * <------> - size - * - * If the interval passes through the end of the universe, then cut its right side. - * - * Example: SAMPLE 0.4 OFFSET 0.8 - * - * [----------------****] - * ^ - offset - * <------> - size - * - * Next, if the `parallel_replicas_count`, `parallel_replica_offset` settings are set, - * then it is necessary to break the received interval into pieces of the number `parallel_replicas_count`, - * and select a piece with the number `parallel_replica_offset` (from zero). - * - * Example: SAMPLE 0.4 OFFSET 0.3, parallel_replicas_count = 2, parallel_replica_offset = 1 - * - * [----------****------] - * ^ - offset - * <------> - size - * <--><--> - pieces for different `parallel_replica_offset`, select the second one. - * - * It is very important that the intervals for different `parallel_replica_offset` cover the entire range without gaps and overlaps. - * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. - */ - - /// Parallel replicas has been requested but there is no way to sample data. - /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) - { - LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" - " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); - return std::make_unique(); - } - - sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); - bool no_data = false; /// There is nothing left after sampling. - - if (sampling.use_sampling) - { - if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) - used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); - - RelativeSize size_of_universum = 0; - const auto & sampling_key = metadata_snapshot->getSamplingKey(); - DataTypePtr sampling_column_type = sampling_key.data_types[0]; - - if (sampling_key.data_types.size() == 1) - { - if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(sampling_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - } - - if (size_of_universum == RelativeSize(0)) - throw Exception( - "Invalid sampling column type in storage parameters: " + sampling_column_type->getName() - + ". Must be one unsigned integer type", - ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); - - if (settings.parallel_replicas_count > 1) - { - if (relative_sample_size == RelativeSize(0)) - relative_sample_size = 1; - - relative_sample_size /= settings.parallel_replicas_count.value; - relative_sample_offset += relative_sample_size * RelativeSize(settings.parallel_replica_offset.value); - } - - if (relative_sample_offset >= RelativeSize(1)) - no_data = true; - - /// Calculate the half-interval of `[lower, upper)` column values. - bool has_lower_limit = false; - bool has_upper_limit = false; - - RelativeSize lower_limit_rational = relative_sample_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_sample_offset + relative_sample_size) * size_of_universum; - - UInt64 lower = boost::rational_cast(lower_limit_rational); - UInt64 upper = boost::rational_cast(upper_limit_rational); - - if (lower > 0) - has_lower_limit = true; - - if (upper_limit_rational < size_of_universum) - has_upper_limit = true; - - /*std::cerr << std::fixed << std::setprecision(100) - << "relative_sample_size: " << relative_sample_size << "\n" - << "relative_sample_offset: " << relative_sample_offset << "\n" - << "lower_limit_float: " << lower_limit_rational << "\n" - << "upper_limit_float: " << upper_limit_rational << "\n" - << "lower: " << lower << "\n" - << "upper: " << upper << "\n";*/ - - if ((has_upper_limit && upper == 0) - || (has_lower_limit && has_upper_limit && lower == upper)) - no_data = true; - - if (no_data || (!has_lower_limit && !has_upper_limit)) - { - sampling.use_sampling = false; - } - else - { - /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. - - std::shared_ptr lower_function; - std::shared_ptr upper_function; - - /// If sample and final are used together no need to calculate sampling expression twice. - /// The first time it was calculated for final, because sample key is a part of the PK. - /// So, assume that we already have calculated column. - ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); - - if (select.final()) - { - sampling_key_ast = std::make_shared(sampling_key.column_names[0]); - /// We do spoil available_real_columns here, but it is not used later. - available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type)); - } - - if (has_lower_limit) - { - if (!key_condition->addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) //-V1007 - throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); - - ASTPtr args = std::make_shared(); - args->children.push_back(sampling_key_ast); - args->children.push_back(std::make_shared(lower)); - - lower_function = std::make_shared(); - lower_function->name = "greaterOrEquals"; - lower_function->arguments = args; - lower_function->children.push_back(lower_function->arguments); - - sampling.filter_function = lower_function; - } - - if (has_upper_limit) - { - if (!key_condition->addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) //-V1007 - throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); - - ASTPtr args = std::make_shared(); - args->children.push_back(sampling_key_ast); - args->children.push_back(std::make_shared(upper)); - - upper_function = std::make_shared(); - upper_function->name = "less"; - upper_function->arguments = args; - upper_function->children.push_back(upper_function->arguments); - - sampling.filter_function = upper_function; - } - - if (has_lower_limit && has_upper_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(lower_function); - args->children.push_back(upper_function); - - sampling.filter_function = std::make_shared(); - sampling.filter_function->name = "and"; - sampling.filter_function->arguments = args; - sampling.filter_function->children.push_back(sampling.filter_function->arguments); - } - - ASTPtr query = sampling.filter_function; - auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); - sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); - } - } - - if (no_data) - { - LOG_DEBUG(log, "Sampling yields no data."); - return std::make_unique(); - } - } + const auto & settings = context->getSettingsRef(); MergeTreeReaderSettings reader_settings = { @@ -1364,443 +1199,48 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .checksum_on_read = settings.checksum_on_read, }; - RangesInDataParts parts_with_ranges(parts.size()); - size_t sum_marks = 0; - size_t sum_ranges = 0; - - /// Let's start analyzing all useful indices - if (!use_cache) + ReadFromMergeTree::Settings step_settings { - struct DataSkippingIndexAndCondition - { - MergeTreeIndexPtr index; - MergeTreeIndexConditionPtr condition; - std::atomic total_granules{0}; - std::atomic granules_dropped{0}; - std::atomic total_parts{0}; - std::atomic parts_dropped{0}; - - DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) - : index(index_), condition(condition_) - { - } - }; - std::list useful_indices; - - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); - } - - if (settings.force_data_skipping_indices.changed) - { - const auto & indices = settings.force_data_skipping_indices.toString(); - - Strings forced_indices; - { - Tokens tokens(&indices[0], &indices[indices.size()], settings.max_query_size); - IParser::Pos pos(tokens, settings.max_parser_depth); - Expected expected; - if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); - } - - if (forced_indices.empty()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); - - std::unordered_set useful_indices_names; - for (const auto & useful_index : useful_indices) - useful_indices_names.insert(useful_index.index->index.name); - - for (const auto & index_name : forced_indices) - { - if (!useful_indices_names.count(index_name)) - { - throw Exception( - ErrorCodes::INDEX_NOT_USED, - "Index {} is not used and setting 'force_data_skipping_indices' contains it", - backQuote(index_name)); - } - } - } - - std::atomic sum_marks_pk = 0; - std::atomic sum_parts_pk = 0; - std::atomic total_marks_pk = 0; - - /// Let's find what range to read from each part. - { - std::atomic total_rows{0}; - - SizeLimits limits; - if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) - limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); - - SizeLimits leaf_limits; - if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) - leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); - - auto process_part = [&](size_t part_index) - { - auto & part = parts[part_index]; - - RangesInDataPart ranges(part, part_index); - - size_t total_marks_count = part->getMarksCount(); - if (total_marks_count && part->index_granularity.hasFinalMark()) - --total_marks_count; - - total_marks_pk.fetch_add(total_marks_count, std::memory_order_relaxed); - - if (metadata_snapshot->hasPrimaryKey()) - ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, *key_condition, settings, log); - else if (total_marks_count) - ranges.ranges = MarkRanges{MarkRange{0, total_marks_count}}; - - sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed); - - if (!ranges.ranges.empty()) - sum_parts_pk.fetch_add(1, std::memory_order_relaxed); - - for (auto & index_and_condition : useful_indices) - { - if (ranges.ranges.empty()) - break; - - index_and_condition.total_parts.fetch_add(1, std::memory_order_relaxed); - - size_t total_granules = 0; - size_t granules_dropped = 0; - ranges.ranges = filterMarksUsingIndex( - index_and_condition.index, - index_and_condition.condition, - part, - ranges.ranges, - settings, - reader_settings, - total_granules, - granules_dropped, - log); - - index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - index_and_condition.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); - - if (ranges.ranges.empty()) - index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed); - } - - if (!ranges.ranges.empty()) - { - if (limits.max_rows || leaf_limits.max_rows) - { - /// Fail fast if estimated number of rows to read exceeds the limit - auto current_rows_estimate = ranges.getRowsCount(); - size_t prev_total_rows_estimate = total_rows.fetch_add(current_rows_estimate); - size_t total_rows_estimate = current_rows_estimate + prev_total_rows_estimate; - limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); - leaf_limits.check( - total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); - } - - parts_with_ranges[part_index] = std::move(ranges); - } - }; - - size_t num_threads = std::min(size_t(num_streams), parts.size()); - - if (num_threads <= 1) - { - for (size_t part_index = 0; part_index < parts.size(); ++part_index) - process_part(part_index); - } - else - { - /// Parallel loading of data parts. - ThreadPool pool(num_threads); - - for (size_t part_index = 0; part_index < parts.size(); ++part_index) - pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] - { - SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachQueryIfNotDetached();); - if (thread_group) - CurrentThread::attachTo(thread_group); - - process_part(part_index); - }); - - pool.wait(); - } - - /// Skip empty ranges. - size_t next_part = 0; - for (size_t part_index = 0; part_index < parts.size(); ++part_index) - { - auto & part = parts_with_ranges[part_index]; - if (!part.data_part) - continue; - - sum_ranges += part.ranges.size(); - sum_marks += part.getMarksCount(); - - if (next_part != part_index) - std::swap(parts_with_ranges[next_part], part); - - ++next_part; - } - - parts_with_ranges.resize(next_part); - } - - if (metadata_snapshot->hasPrimaryKey()) - { - auto description = key_condition->getDescription(); - - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::PrimaryKey, - .condition = std::move(description.condition), - .used_keys = std::move(description.used_keys), - .num_parts_after = sum_parts_pk.load(std::memory_order_relaxed), - .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); - } - - for (const auto & index_and_condition : useful_indices) - { - const auto & index_name = index_and_condition.index->index.name; - LOG_DEBUG( - log, - "Index {} has dropped {}/{} granules.", - backQuote(index_name), - index_and_condition.granules_dropped, - index_and_condition.total_granules); - - std::string description - = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); - - index_stats->emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Skip, - .name = index_name, - .description = std::move(description), - .num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped, - .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); - } - - LOG_DEBUG( - log, - "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - parts.size(), - total_parts, - parts_with_ranges.size(), - sum_marks_pk.load(std::memory_order_relaxed), - total_marks_pk.load(std::memory_order_relaxed), - sum_marks, - sum_ranges); - } - - if (cache) - { - if (cache->use_cache) - { - parts_with_ranges = std::move(cache->parts_with_ranges); - sum_marks = cache->sum_marks; - sum_ranges = cache->sum_ranges; - } - else - { - // We are asking for ranges_to_read. Return immediately without further planning. - cache->parts_with_ranges = std::move(parts_with_ranges); - cache->sampling = std::move(sampling); - cache->index_stats = std::move(index_stats); - cache->sum_marks = sum_marks; - cache->sum_ranges = sum_ranges; - cache->use_cache = true; - return std::make_unique(); - } - } - - if (parts_with_ranges.empty()) - return std::make_unique(); - - // Check limitations. query_id is used as the quota RAII's resource key. - String query_id; - { - const auto data_settings = data.getSettings(); - auto max_partitions_to_read - = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; - if (max_partitions_to_read > 0) - { - std::set partitions; - for (auto & part_with_ranges : parts_with_ranges) - partitions.insert(part_with_ranges.data_part->info.partition_id); - if (partitions.size() > size_t(max_partitions_to_read)) - throw Exception( - ErrorCodes::TOO_MANY_PARTITIONS, - "Too many partitions to read. Current {}, max {}", - partitions.size(), - max_partitions_to_read); - } - - if (data_settings->max_concurrent_queries > 0) - { - if (data_settings->min_marks_to_honor_max_concurrent_queries > 0 - && sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) - { - query_id = context->getCurrentQueryId(); - if (!query_id.empty()) - data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries); - } - } - } - - ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); - ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); - ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); - - QueryPlanPtr plan; - - /// Projection, that needed to drop columns, which have appeared by execution - /// of some extra expressions, and to allow execute the same expressions later. - /// NOTE: It may lead to double computation of expressions. - ActionsDAGPtr result_projection; - - Names column_names_to_read = real_column_names; - if (!select.final() && sampling.use_sampling) - { - /// Add columns needed for `sample_by_ast` to `column_names_to_read`. - /// Skip this if final was used, because such columns were already added from PK. - std::vector add_columns = sampling.filter_expression->getRequiredColumns().getNames(); - column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); - std::sort(column_names_to_read.begin(), column_names_to_read.end()); - column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), - column_names_to_read.end()); - } - - const auto & input_order_info = query_info.input_order_info - ? query_info.input_order_info - : (query_info.projection ? query_info.projection->input_order_info : nullptr); - - if (select.final()) - { - /// Add columns needed to calculate the sorting expression and the sign. - std::vector add_columns = metadata_snapshot->getColumnsRequiredForSortingKey(); - column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); - - if (!data.merging_params.sign_column.empty()) - column_names_to_read.push_back(data.merging_params.sign_column); - if (!data.merging_params.version_column.empty()) - column_names_to_read.push_back(data.merging_params.version_column); - - std::sort(column_names_to_read.begin(), column_names_to_read.end()); - column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); - - plan = spreadMarkRangesAmongStreamsFinal( - std::move(parts_with_ranges), - std::move(index_stats), - num_streams, - column_names_to_read, - metadata_snapshot, - max_block_size, - settings.use_uncompressed_cache, - query_info, - virt_column_names, - settings, - reader_settings, - result_projection, - query_id); - } - else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info) - { - size_t prefix_size = input_order_info->order_key_prefix_descr.size(); - auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); - order_key_prefix_ast->children.resize(prefix_size); - - auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical()); - auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false); - - plan = spreadMarkRangesAmongStreamsWithOrder( - std::move(parts_with_ranges), - std::move(index_stats), - num_streams, - column_names_to_read, - metadata_snapshot, - max_block_size, - settings.use_uncompressed_cache, - query_info, - sorting_key_prefix_expr, - virt_column_names, - settings, - reader_settings, - result_projection, - query_id, - input_order_info); - } - else - { - plan = spreadMarkRangesAmongStreams( - std::move(parts_with_ranges), - std::move(index_stats), - num_streams, - column_names_to_read, - metadata_snapshot, - max_block_size, - settings.use_uncompressed_cache, - query_info, - virt_column_names, - settings, - reader_settings, - query_id); - } - - if (!plan) - return std::make_unique(); - - if (sampling.use_sampling) - { - auto sampling_step = std::make_unique( - plan->getCurrentDataStream(), - sampling.filter_expression, - sampling.filter_function->getColumnName(), - false); - - sampling_step->setStepDescription("Sampling"); - plan->addStep(std::move(sampling_step)); - } - - if (result_projection) - { - auto projection_step = std::make_unique(plan->getCurrentDataStream(), result_projection); - projection_step->setStepDescription("Remove unused columns after reading from storage"); - plan->addStep(std::move(projection_step)); - } - - /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. - if (sample_factor_column_queried) - { - ColumnWithTypeAndName column; - column.name = "_sample_factor"; - column.type = std::make_shared(); - column.column = column.type->createColumnConst(0, Field(used_sample_factor)); - - auto adding_column_action = ActionsDAG::makeAddingColumnActions(std::move(column)); - - auto adding_column = std::make_unique(plan->getCurrentDataStream(), std::move(adding_column_action)); - adding_column->setStepDescription("Add _sample_factor column"); - plan->addStep(std::move(adding_column)); - } - - // TODO There seems to be no place initializing remove_columns_actions - if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) - { - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - - expression_step->setStepDescription("Remove unused columns after PREWHERE"); - plan->addStep(std::move(expression_step)); - } - + .max_block_size = max_block_size, + .num_streams = num_streams, + .preferred_block_size_bytes = settings.preferred_block_size_bytes, + .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, + //.min_marks_for_concurrent_read = settings.min_marks_for_concurrent_read, + .use_uncompressed_cache = settings.use_uncompressed_cache, + .reader_settings = reader_settings, + .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), + }; + + // const SelectQueryInfo & query_info_, + // const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, + // ContextPtr context_, + // const MergeTreeData & data_, + // StorageMetadataPtr metadata_snapshot_, + // StorageMetadataPtr metadata_snapshot_base_, + // Names real_column_names_, + // MergeTreeData::DataPartsVector parts_, + // PrewhereInfoPtr prewhere_info_, + // Names virt_column_names_, + // Settings settings_, + // Poco::Logger * log_ + + auto read_from_merge_tree = std::make_unique( + query_info, + max_block_numbers_to_read, + context, + data, + metadata_snapshot, + metadata_snapshot_base, + real_column_names, + parts, + query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, + virt_column_names, + step_settings, + log + ); + + QueryPlanPtr plan = std::make_unique(); + plan->addStep(std::move(read_from_merge_tree)); return plan; } @@ -1822,638 +1262,609 @@ size_t roundRowsOrBytesToMarks( else return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); } -/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks -size_t minMarksForConcurrentRead( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity, - size_t max_marks) -{ - size_t marks = 1; - - if (rows_setting + rows_granularity <= rows_setting) /// overflow - marks = max_marks; - else if (rows_setting) - marks = (rows_setting + rows_granularity - 1) / rows_granularity; - - if (bytes_granularity == 0) - return marks; - else - { - /// Overflow - if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow - return max_marks; - if (bytes_setting) - return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); - else - return marks; - } -} } -QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( - RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - const String & query_id) const -{ - /// Count marks for each part. - std::vector sum_marks_in_parts(parts.size()); - size_t sum_marks = 0; - size_t total_rows = 0; - - const auto data_settings = data.getSettings(); - size_t adaptive_parts = 0; - for (size_t i = 0; i < parts.size(); ++i) - { - total_rows += parts[i].getRowsCount(); - sum_marks_in_parts[i] = parts[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - - if (parts[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - if (0 == sum_marks) - return {}; - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - if (num_streams > 1) - { - /// Reduce the number of num_streams if the data is small. - if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) - num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); - } - - auto plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams, - ReadFromMergeTree::ReadType::Default); - - plan->addStep(std::move(step)); - return plan; -} - -static ActionsDAGPtr createProjection(const Block & header) -{ - auto projection = std::make_shared(header.getNamesAndTypesList()); - projection->removeUnusedActions(header.getNames()); - projection->projectInput(); - return projection; -} - -QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( - RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const ActionsDAGPtr & sorting_key_prefix_expr, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id, - const InputOrderInfoPtr & input_order_info) const -{ - size_t sum_marks = 0; - size_t adaptive_parts = 0; - std::vector sum_marks_in_parts(parts.size()); - const auto data_settings = data.getSettings(); - - for (size_t i = 0; i < parts.size(); ++i) - { - sum_marks_in_parts[i] = parts[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - - if (parts[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - Pipes res; - - if (sum_marks == 0) - return {}; - - /// Let's split ranges to avoid reading much data. - auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) - { - MarkRanges new_ranges; - const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; - size_t marks_in_range = 1; - - if (direction == 1) - { - /// Split first few ranges to avoid reading much data. - bool split = false; - for (auto range : ranges) - { - while (!split && range.begin + marks_in_range < range.end) - { - new_ranges.emplace_back(range.begin, range.begin + marks_in_range); - range.begin += marks_in_range; - marks_in_range *= 2; - - if (marks_in_range > max_marks_in_range) - split = true; - } - new_ranges.emplace_back(range.begin, range.end); - } - } - else - { - /// Split all ranges to avoid reading much data, because we have to - /// store whole range in memory to reverse it. - for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) - { - auto range = *it; - while (range.begin + marks_in_range < range.end) - { - new_ranges.emplace_front(range.end - marks_in_range, range.end); - range.end -= marks_in_range; - marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); - } - new_ranges.emplace_front(range.begin, range.end); - } - } - - return new_ranges; - }; - - const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; - bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); - - std::vector plans; - - for (size_t i = 0; i < num_streams && !parts.empty(); ++i) - { - size_t need_marks = min_marks_per_stream; - RangesInDataParts new_parts; - - /// Loop over parts. - /// We will iteratively take part or some subrange of a part from the back - /// and assign a stream to read from it. - while (need_marks > 0 && !parts.empty()) - { - RangesInDataPart part = parts.back(); - parts.pop_back(); - - size_t & marks_in_part = sum_marks_in_parts.back(); - - /// We will not take too few rows from a part. - if (marks_in_part >= min_marks_for_concurrent_read && - need_marks < min_marks_for_concurrent_read) - need_marks = min_marks_for_concurrent_read; - - /// Do not leave too few rows in the part. - if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) - need_marks = marks_in_part; - - MarkRanges ranges_to_get_from_part; - - /// We take the whole part if it is small enough. - if (marks_in_part <= need_marks) - { - ranges_to_get_from_part = part.ranges; - - need_marks -= marks_in_part; - sum_marks_in_parts.pop_back(); - } - else - { - /// Loop through ranges in part. Take enough ranges to cover "need_marks". - while (need_marks > 0) - { - if (part.ranges.empty()) - throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); - - MarkRange & range = part.ranges.front(); - - const size_t marks_in_range = range.end - range.begin; - const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); - - ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); - range.begin += marks_to_get_from_range; - marks_in_part -= marks_to_get_from_range; - need_marks -= marks_to_get_from_range; - if (range.begin == range.end) - part.ranges.pop_front(); - } - parts.emplace_back(part); - } - ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); - } - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - auto read_type = input_order_info->direction == 1 - ? ReadFromMergeTree::ReadType::InOrder - : ReadFromMergeTree::ReadType::InReverseOrder; - - auto plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(new_parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams, - read_type); - - plan->addStep(std::move(step)); - plans.emplace_back(std::move(plan)); - } - - if (need_preliminary_merge) - { - SortDescription sort_description; - for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], - input_order_info->direction, 1); - - for (auto & plan : plans) - { - /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(plan->getCurrentDataStream().header); - - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - sorting_key_prefix_expr); - - expression_step->setStepDescription("Calculate sorting key prefix"); - plan->addStep(std::move(expression_step)); - - auto merging_sorted = std::make_unique( - plan->getCurrentDataStream(), - sort_description, - max_block_size); - - merging_sorted->setStepDescription("Merge sorting mark ranges"); - plan->addStep(std::move(merging_sorted)); - } - } - - if (plans.size() == 1) - return std::move(plans.front()); - - DataStreams input_streams; - for (const auto & plan : plans) - input_streams.emplace_back(plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams)); - - auto plan = std::make_unique(); - plan->unitePlans(std::move(union_step), std::move(plans)); - - return plan; -} - - -QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, - ReadFromMergeTree::IndexStatPtr index_stats, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id) const -{ - const auto data_settings = data.getSettings(); - size_t sum_marks = 0; - size_t adaptive_parts = 0; - for (const auto & part : parts) - { - for (const auto & range : part.ranges) - sum_marks += range.end - range.begin; - - if (part.data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts >= parts.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - if (num_streams > settings.max_final_threads) - num_streams = settings.max_final_threads; - - /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. - /// We have all parts in parts vector, where parts with same partition are nearby. - /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), - /// then we will create a pipe for each partition that will run selecting processor and merging processor - /// for the parts with this partition. In the end we will unite all the pipes. - std::vector parts_to_merge_ranges; - auto it = parts.begin(); - parts_to_merge_ranges.push_back(it); - - if (settings.do_not_merge_across_partitions_select_final) - { - while (it != parts.end()) - { - it = std::find_if( - it, parts.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); - parts_to_merge_ranges.push_back(it); - } - /// We divide threads for each partition equally. But we will create at least the number of partitions threads. - /// (So, the total number of threads could be more than initial num_streams. - num_streams /= (parts_to_merge_ranges.size() - 1); - } - else - { - /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. - parts_to_merge_ranges.push_back(parts.end()); - } - - std::vector partition_plans; - - /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 - /// we will store lonely parts with level > 0 to use parallel select on them. - std::vector lonely_parts; - size_t total_rows_in_lonely_parts = 0; - size_t sum_marks_in_lonely_parts = 0; - - for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) - { - QueryPlanPtr plan; - - { - RangesInDataParts new_parts; - - /// 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 and if num_streams > 1 we - /// can use parallel select on such parts. We save such parts in one vector and then use - /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. - if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && - 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) - { - total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); - sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); - lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); - continue; - } - else - { - for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) - { - new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); - } - } - - if (new_parts.empty()) - continue; - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(new_parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams, - ReadFromMergeTree::ReadType::InOrder); - - plan->addStep(std::move(step)); - - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(plan->getCurrentDataStream().header); - } - - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - expression_step->setStepDescription("Calculate sorting key expression"); - plan->addStep(std::move(expression_step)); - - /// 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 - if (settings.do_not_merge_across_partitions_select_final && - 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_plans.emplace_back(std::move(plan)); - continue; - } - - Names sort_columns = metadata_snapshot->getSortingKeyColumns(); - SortDescription sort_description; - size_t sort_columns_size = sort_columns.size(); - sort_description.reserve(sort_columns_size); - - Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - - const auto & header = plan->getCurrentDataStream().header; - for (size_t i = 0; i < sort_columns_size; ++i) - sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); - - auto final_step = std::make_unique( - plan->getCurrentDataStream(), - std::min(num_streams, settings.max_final_threads), - sort_description, - data.merging_params, - partition_key_columns, - max_block_size); - - final_step->setStepDescription("Merge rows for FINAL"); - plan->addStep(std::move(final_step)); - - partition_plans.emplace_back(std::move(plan)); - } - - 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 = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks_in_lonely_parts); - - /// Reduce the number of num_streams_for_lonely_parts if the data is small. - if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) - num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - .use_uncompressed_cache = use_uncompressed_cache, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - - auto plan = std::make_unique(); - auto step = std::make_unique( - data, - metadata_snapshot, - query_id, - column_names, - std::move(lonely_parts), - std::move(index_stats), - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_columns, - step_settings, - num_streams_for_lonely_parts, - ReadFromMergeTree::ReadType::Default); - - plan->addStep(std::move(step)); - - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(plan->getCurrentDataStream().header); - - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - expression_step->setStepDescription("Calculate sorting key expression"); - plan->addStep(std::move(expression_step)); - - partition_plans.emplace_back(std::move(plan)); - } - - if (partition_plans.empty()) - return {}; - - if (partition_plans.size() == 1) - return std::move(partition_plans.front()); - - auto result_header = partition_plans.front()->getCurrentDataStream().header; - DataStreams input_streams; - for (const auto & partition_plan : partition_plans) - input_streams.push_back(partition_plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams), result_header); - union_step->setStepDescription("Unite sources after FINAL"); - QueryPlanPtr plan = std::make_unique(); - plan->unitePlans(std::move(union_step), std::move(partition_plans)); - return plan; -} +// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( +// RangesInDataParts && parts, +// // ReadFromMergeTree::IndexStatPtr index_stats, +// size_t num_streams, +// const Names & column_names, +// const StorageMetadataPtr & metadata_snapshot, +// UInt64 max_block_size, +// bool use_uncompressed_cache, +// const SelectQueryInfo & query_info, +// const Names & virt_columns, +// const Settings & settings, +// const MergeTreeReaderSettings & reader_settings, +// const String & query_id) const +// { +// /// Count marks for each part. +// std::vector sum_marks_in_parts(parts.size()); +// size_t sum_marks = 0; +// size_t total_rows = 0; + +// const auto data_settings = data.getSettings(); +// size_t adaptive_parts = 0; +// for (size_t i = 0; i < parts.size(); ++i) +// { +// total_rows += parts[i].getRowsCount(); +// sum_marks_in_parts[i] = parts[i].getMarksCount(); +// sum_marks += sum_marks_in_parts[i]; + +// if (parts[i].data_part->index_granularity_info.is_adaptive) +// ++adaptive_parts; +// } + +// size_t index_granularity_bytes = 0; +// if (adaptive_parts > parts.size() / 2) +// index_granularity_bytes = data_settings->index_granularity_bytes; + +// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( +// settings.merge_tree_max_rows_to_use_cache, +// settings.merge_tree_max_bytes_to_use_cache, +// data_settings->index_granularity, +// index_granularity_bytes); + +// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( +// settings.merge_tree_min_rows_for_concurrent_read, +// settings.merge_tree_min_bytes_for_concurrent_read, +// data_settings->index_granularity, +// index_granularity_bytes, +// sum_marks); + +// if (sum_marks > max_marks_to_use_cache) +// use_uncompressed_cache = false; + +// if (0 == sum_marks) +// return {}; + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// if (num_streams > 1) +// { +// /// Reduce the number of num_streams if the data is small. +// if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) +// num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); +// } + +// auto plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams, +// ReadFromMergeTree::ReadType::Default); + +// plan->addStep(std::move(step)); +// return plan; +// } + +// static ActionsDAGPtr createProjection(const Block & header) +// { +// auto projection = std::make_shared(header.getNamesAndTypesList()); +// projection->removeUnusedActions(header.getNames()); +// projection->projectInput(); +// return projection; +// } + +// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( +// RangesInDataParts && parts, +// // ReadFromMergeTree::IndexStatPtr index_stats, +// size_t num_streams, +// const Names & column_names, +// const StorageMetadataPtr & metadata_snapshot, +// UInt64 max_block_size, +// bool use_uncompressed_cache, +// const SelectQueryInfo & query_info, +// const ActionsDAGPtr & sorting_key_prefix_expr, +// const Names & virt_columns, +// const Settings & settings, +// const MergeTreeReaderSettings & reader_settings, +// ActionsDAGPtr & out_projection, +// const String & query_id, +// const InputOrderInfoPtr & input_order_info) const +// { +// size_t sum_marks = 0; +// size_t adaptive_parts = 0; +// std::vector sum_marks_in_parts(parts.size()); +// const auto data_settings = data.getSettings(); + +// for (size_t i = 0; i < parts.size(); ++i) +// { +// sum_marks_in_parts[i] = parts[i].getMarksCount(); +// sum_marks += sum_marks_in_parts[i]; + +// if (parts[i].data_part->index_granularity_info.is_adaptive) +// ++adaptive_parts; +// } + +// size_t index_granularity_bytes = 0; +// if (adaptive_parts > parts.size() / 2) +// index_granularity_bytes = data_settings->index_granularity_bytes; + +// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( +// settings.merge_tree_max_rows_to_use_cache, +// settings.merge_tree_max_bytes_to_use_cache, +// data_settings->index_granularity, +// index_granularity_bytes); + +// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( +// settings.merge_tree_min_rows_for_concurrent_read, +// settings.merge_tree_min_bytes_for_concurrent_read, +// data_settings->index_granularity, +// index_granularity_bytes, +// sum_marks); + +// if (sum_marks > max_marks_to_use_cache) +// use_uncompressed_cache = false; + +// Pipes res; + +// if (sum_marks == 0) +// return {}; + +// /// Let's split ranges to avoid reading much data. +// auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) +// { +// MarkRanges new_ranges; +// const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; +// size_t marks_in_range = 1; + +// if (direction == 1) +// { +// /// Split first few ranges to avoid reading much data. +// bool split = false; +// for (auto range : ranges) +// { +// while (!split && range.begin + marks_in_range < range.end) +// { +// new_ranges.emplace_back(range.begin, range.begin + marks_in_range); +// range.begin += marks_in_range; +// marks_in_range *= 2; + +// if (marks_in_range > max_marks_in_range) +// split = true; +// } +// new_ranges.emplace_back(range.begin, range.end); +// } +// } +// else +// { +// /// Split all ranges to avoid reading much data, because we have to +// /// store whole range in memory to reverse it. +// for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) +// { +// auto range = *it; +// while (range.begin + marks_in_range < range.end) +// { +// new_ranges.emplace_front(range.end - marks_in_range, range.end); +// range.end -= marks_in_range; +// marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); +// } +// new_ranges.emplace_front(range.begin, range.end); +// } +// } + +// return new_ranges; +// }; + +// const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; +// bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); + +// std::vector plans; + +// for (size_t i = 0; i < num_streams && !parts.empty(); ++i) +// { +// size_t need_marks = min_marks_per_stream; +// RangesInDataParts new_parts; + +// /// Loop over parts. +// /// We will iteratively take part or some subrange of a part from the back +// /// and assign a stream to read from it. +// while (need_marks > 0 && !parts.empty()) +// { +// RangesInDataPart part = parts.back(); +// parts.pop_back(); + +// size_t & marks_in_part = sum_marks_in_parts.back(); + +// /// We will not take too few rows from a part. +// if (marks_in_part >= min_marks_for_concurrent_read && +// need_marks < min_marks_for_concurrent_read) +// need_marks = min_marks_for_concurrent_read; + +// /// Do not leave too few rows in the part. +// if (marks_in_part > need_marks && +// marks_in_part - need_marks < min_marks_for_concurrent_read) +// need_marks = marks_in_part; + +// MarkRanges ranges_to_get_from_part; + +// /// We take the whole part if it is small enough. +// if (marks_in_part <= need_marks) +// { +// ranges_to_get_from_part = part.ranges; + +// need_marks -= marks_in_part; +// sum_marks_in_parts.pop_back(); +// } +// else +// { +// /// Loop through ranges in part. Take enough ranges to cover "need_marks". +// while (need_marks > 0) +// { +// if (part.ranges.empty()) +// throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); + +// MarkRange & range = part.ranges.front(); + +// const size_t marks_in_range = range.end - range.begin; +// const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + +// ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); +// range.begin += marks_to_get_from_range; +// marks_in_part -= marks_to_get_from_range; +// need_marks -= marks_to_get_from_range; +// if (range.begin == range.end) +// part.ranges.pop_front(); +// } +// parts.emplace_back(part); +// } +// ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); +// new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); +// } + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// auto read_type = input_order_info->direction == 1 +// ? ReadFromMergeTree::ReadType::InOrder +// : ReadFromMergeTree::ReadType::InReverseOrder; + +// auto plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(new_parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams, +// read_type); + +// plan->addStep(std::move(step)); +// plans.emplace_back(std::move(plan)); +// } + +// if (need_preliminary_merge) +// { +// SortDescription sort_description; +// for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) +// sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], +// input_order_info->direction, 1); + +// for (auto & plan : plans) +// { +// /// Drop temporary columns, added by 'sorting_key_prefix_expr' +// out_projection = createProjection(plan->getCurrentDataStream().header); + +// auto expression_step = std::make_unique( +// plan->getCurrentDataStream(), +// sorting_key_prefix_expr); + +// expression_step->setStepDescription("Calculate sorting key prefix"); +// plan->addStep(std::move(expression_step)); + +// auto merging_sorted = std::make_unique( +// plan->getCurrentDataStream(), +// sort_description, +// max_block_size); + +// merging_sorted->setStepDescription("Merge sorting mark ranges"); +// plan->addStep(std::move(merging_sorted)); +// } +// } + +// if (plans.size() == 1) +// return std::move(plans.front()); + +// DataStreams input_streams; +// for (const auto & plan : plans) +// input_streams.emplace_back(plan->getCurrentDataStream()); + +// auto union_step = std::make_unique(std::move(input_streams)); + +// auto plan = std::make_unique(); +// plan->unitePlans(std::move(union_step), std::move(plans)); + +// return plan; +// } + + +// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( +// RangesInDataParts && parts, +// size_t num_streams, +// const Names & column_names, +// const StorageMetadataPtr & metadata_snapshot, +// UInt64 max_block_size, +// bool use_uncompressed_cache, +// const SelectQueryInfo & query_info, +// const Names & virt_columns, +// const Settings & settings, +// const MergeTreeReaderSettings & reader_settings, +// ActionsDAGPtr & out_projection, +// const String & query_id) const +// { +// const auto data_settings = data.getSettings(); +// size_t sum_marks = 0; +// size_t adaptive_parts = 0; +// for (const auto & part : parts) +// { +// for (const auto & range : part.ranges) +// sum_marks += range.end - range.begin; + +// if (part.data_part->index_granularity_info.is_adaptive) +// ++adaptive_parts; +// } + +// size_t index_granularity_bytes = 0; +// if (adaptive_parts >= parts.size() / 2) +// index_granularity_bytes = data_settings->index_granularity_bytes; + +// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( +// settings.merge_tree_max_rows_to_use_cache, +// settings.merge_tree_max_bytes_to_use_cache, +// data_settings->index_granularity, +// index_granularity_bytes); + +// if (sum_marks > max_marks_to_use_cache) +// use_uncompressed_cache = false; + +// if (num_streams > settings.max_final_threads) +// num_streams = settings.max_final_threads; + +// /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. +// /// We have all parts in parts vector, where parts with same partition are nearby. +// /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), +// /// then we will create a pipe for each partition that will run selecting processor and merging processor +// /// for the parts with this partition. In the end we will unite all the pipes. +// std::vector parts_to_merge_ranges; +// auto it = parts.begin(); +// parts_to_merge_ranges.push_back(it); + +// if (settings.do_not_merge_across_partitions_select_final) +// { +// while (it != parts.end()) +// { +// it = std::find_if( +// it, parts.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); +// parts_to_merge_ranges.push_back(it); +// } +// /// We divide threads for each partition equally. But we will create at least the number of partitions threads. +// /// (So, the total number of threads could be more than initial num_streams. +// num_streams /= (parts_to_merge_ranges.size() - 1); +// } +// else +// { +// /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. +// parts_to_merge_ranges.push_back(parts.end()); +// } + +// std::vector partition_plans; + +// /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 +// /// we will store lonely parts with level > 0 to use parallel select on them. +// std::vector lonely_parts; +// size_t total_rows_in_lonely_parts = 0; +// size_t sum_marks_in_lonely_parts = 0; + +// for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) +// { +// QueryPlanPtr plan; + +// { +// RangesInDataParts new_parts; + +// /// 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 and if num_streams > 1 we +// /// can use parallel select on such parts. We save such parts in one vector and then use +// /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. +// if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && +// 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) +// { +// total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); +// sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); +// lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); +// continue; +// } +// else +// { +// for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) +// { +// new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); +// } +// } + +// if (new_parts.empty()) +// continue; + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(new_parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams, +// ReadFromMergeTree::ReadType::InOrder); + +// plan->addStep(std::move(step)); + +// /// Drop temporary columns, added by 'sorting_key_expr' +// if (!out_projection) +// out_projection = createProjection(plan->getCurrentDataStream().header); +// } + +// auto expression_step = std::make_unique( +// plan->getCurrentDataStream(), +// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + +// expression_step->setStepDescription("Calculate sorting key expression"); +// plan->addStep(std::move(expression_step)); + +// /// 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 +// if (settings.do_not_merge_across_partitions_select_final && +// 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_plans.emplace_back(std::move(plan)); +// continue; +// } + +// Names sort_columns = metadata_snapshot->getSortingKeyColumns(); +// SortDescription sort_description; +// size_t sort_columns_size = sort_columns.size(); +// sort_description.reserve(sort_columns_size); + +// Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + +// const auto & header = plan->getCurrentDataStream().header; +// for (size_t i = 0; i < sort_columns_size; ++i) +// sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); + +// auto final_step = std::make_unique( +// plan->getCurrentDataStream(), +// std::min(num_streams, settings.max_final_threads), +// sort_description, +// data.merging_params, +// partition_key_columns, +// max_block_size); + +// final_step->setStepDescription("Merge rows for FINAL"); +// plan->addStep(std::move(final_step)); + +// partition_plans.emplace_back(std::move(plan)); +// } + +// 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 = minMarksForConcurrentRead( +// settings.merge_tree_min_rows_for_concurrent_read, +// settings.merge_tree_min_bytes_for_concurrent_read, +// data_settings->index_granularity, +// index_granularity_bytes, +// sum_marks_in_lonely_parts); + +// /// Reduce the number of num_streams_for_lonely_parts if the data is small. +// if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) +// num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); + +// ReadFromMergeTree::Settings step_settings +// { +// .max_block_size = max_block_size, +// .preferred_block_size_bytes = settings.preferred_block_size_bytes, +// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, +// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, +// .use_uncompressed_cache = use_uncompressed_cache, +// .reader_settings = reader_settings, +// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), +// }; + +// auto plan = std::make_unique(); +// auto step = std::make_unique( +// data, +// metadata_snapshot, +// query_id, +// column_names, +// std::move(lonely_parts), +// // std::move(index_stats), +// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, +// virt_columns, +// step_settings, +// num_streams_for_lonely_parts, +// ReadFromMergeTree::ReadType::Default); + +// plan->addStep(std::move(step)); + +// /// Drop temporary columns, added by 'sorting_key_expr' +// if (!out_projection) +// out_projection = createProjection(plan->getCurrentDataStream().header); + +// auto expression_step = std::make_unique( +// plan->getCurrentDataStream(), +// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + +// expression_step->setStepDescription("Calculate sorting key expression"); +// plan->addStep(std::move(expression_step)); + +// partition_plans.emplace_back(std::move(plan)); +// } + +// if (partition_plans.empty()) +// return {}; + +// if (partition_plans.size() == 1) +// return std::move(partition_plans.front()); + +// auto result_header = partition_plans.front()->getCurrentDataStream().header; +// DataStreams input_streams; +// for (const auto & partition_plan : partition_plans) +// input_streams.push_back(partition_plan->getCurrentDataStream()); + +// auto union_step = std::make_unique(std::move(input_streams), result_header); +// union_step->setStepDescription("Unite sources after FINAL"); +// QueryPlanPtr plan = std::make_unique(); +// plan->unitePlans(std::move(union_step), std::move(partition_plans)); +// return plan; +// } /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 7597af2e173..3e8076de8d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -17,19 +17,22 @@ struct MergeTreeDataSelectSamplingData { bool use_sampling = false; bool read_nothing = false; + Float64 used_sample_factor = 1.0; std::shared_ptr filter_function; ActionsDAGPtr filter_expression; }; -struct MergeTreeDataSelectCache -{ - RangesInDataParts parts_with_ranges; - MergeTreeDataSelectSamplingData sampling; - std::unique_ptr index_stats; - size_t sum_marks = 0; - size_t sum_ranges = 0; - bool use_cache = false; -}; +// struct MergeTreeDataSelectCache +// { +// RangesInDataParts parts_with_ranges; +// MergeTreeDataSelectSamplingData sampling; +// std::unique_ptr index_stats; +// size_t sum_marks = 0; +// size_t sum_ranges = 0; +// bool use_cache = false; +// }; + +using PartitionIdToMaxBlock = std::unordered_map; /** Executes SELECT queries on data from the merge tree. */ @@ -41,7 +44,6 @@ public: /** When reading, selects a set of parts that covers the desired range of the index. * max_blocks_number_to_read - if not nullptr, do not read all the parts whose right border is greater than max_block in partition. */ - using PartitionIdToMaxBlock = std::unordered_map; QueryPlanPtr read( const Names & column_names, @@ -53,6 +55,16 @@ public: QueryProcessingStage::Enum processed_stage, const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + size_t estimateNumMarksToRead( + MergeTreeData::DataPartsVector parts, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + unsigned num_streams, + const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, @@ -62,57 +74,56 @@ public: ContextPtr context, UInt64 max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr, - MergeTreeDataSelectCache * cache = nullptr) const; + const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; private: const MergeTreeData & data; Poco::Logger * log; - QueryPlanPtr spreadMarkRangesAmongStreams( - RangesInDataParts && parts, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - const String & query_id) const; + // QueryPlanPtr spreadMarkRangesAmongStreams( + // RangesInDataParts && parts, + // size_t num_streams, + // const Names & column_names, + // const StorageMetadataPtr & metadata_snapshot, + // UInt64 max_block_size, + // bool use_uncompressed_cache, + // const SelectQueryInfo & query_info, + // const Names & virt_columns, + // const Settings & settings, + // const MergeTreeReaderSettings & reader_settings, + // const String & query_id) const; - /// out_projection - save projection only with columns, requested to read - QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( - RangesInDataParts && parts, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const ActionsDAGPtr & sorting_key_prefix_expr, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id, - const InputOrderInfoPtr & input_order_info) const; + // /// out_projection - save projection only with columns, requested to read + // QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( + // RangesInDataParts && parts, + // size_t num_streams, + // const Names & column_names, + // const StorageMetadataPtr & metadata_snapshot, + // UInt64 max_block_size, + // bool use_uncompressed_cache, + // const SelectQueryInfo & query_info, + // const ActionsDAGPtr & sorting_key_prefix_expr, + // const Names & virt_columns, + // const Settings & settings, + // const MergeTreeReaderSettings & reader_settings, + // ActionsDAGPtr & out_projection, + // const String & query_id, + // const InputOrderInfoPtr & input_order_info) const; - QueryPlanPtr spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, - size_t num_streams, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - UInt64 max_block_size, - bool use_uncompressed_cache, - const SelectQueryInfo & query_info, - const Names & virt_columns, - const Settings & settings, - const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection, - const String & query_id) const; + // QueryPlanPtr spreadMarkRangesAmongStreamsFinal( + // RangesInDataParts && parts, + // size_t num_streams, + // const Names & column_names, + // const StorageMetadataPtr & metadata_snapshot, + // UInt64 max_block_size, + // bool use_uncompressed_cache, + // const SelectQueryInfo & query_info, + // const Names & virt_columns, + // const Settings & settings, + // const MergeTreeReaderSettings & reader_settings, + // ActionsDAGPtr & out_projection, + // const String & query_id) const; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. static size_t getApproximateTotalRowsToRead( @@ -140,7 +151,6 @@ private: size_t & granules_dropped, Poco::Logger * log); -public: struct PartFilterCounters { size_t num_initial_selected_parts = 0; @@ -175,16 +185,36 @@ public: PartFilterCounters & counters, Poco::Logger * log); - static RangesInDataParts filterParts( +public: + static std::optional> filterPartsByVirtualColumns( + const MergeTreeData & data, MergeTreeData::DataPartsVector & parts, + const ASTPtr & query, + ContextPtr context); + + static void filterPartsByPartition( + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & data, + const SelectQueryInfo & query_info, + ContextPtr & context, + ContextPtr & query_context, + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, + const PartitionIdToMaxBlock * max_block_numbers_to_read, + Poco::Logger * log, + ReadFromMergeTree::IndexStats & index_stats); + + static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( + MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - SelectQueryInfo & query_info, + const SelectQueryInfo & query_info, ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, - ReadFromMergeTree::IndexStats & index_stats); + ReadFromMergeTree::IndexStats & index_stats, + bool use_skip_indexes); static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, @@ -197,7 +227,7 @@ public: NamesAndTypesList available_real_columns, ContextPtr context); - static String checkLimits(MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); + static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); }; } diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index e8b39c8e28c..dff864bc58c 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -41,10 +41,7 @@ public: query_info, context, max_block_size, - num_streams, - nullptr, - query_info.projection ? query_info.projection->merge_tree_data_select_base_cache.get() - : query_info.merge_tree_data_select_cache.get())); + num_streams)); return query_plan.convertToPipe( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index afed41189c2..73cf3893a89 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -137,8 +137,8 @@ struct ProjectionCandidate ReadInOrderOptimizerPtr order_optimizer; InputOrderInfoPtr input_order_info; ManyExpressionActions group_by_elements_actions; - std::shared_ptr merge_tree_data_select_base_cache; - std::shared_ptr merge_tree_data_select_projection_cache; + // std::shared_ptr merge_tree_data_select_base_cache; + // std::shared_ptr merge_tree_data_select_projection_cache; }; /** Query along with some additional data, From a961100916d6a9c28c3b30590b607c8c87d03f91 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 27 May 2021 18:02:06 +0300 Subject: [PATCH 234/652] More stable test --- .../test_replicated_fetches_bandwidth/test.py | 39 +++++++------------ 1 file changed, 13 insertions(+), 26 deletions(-) diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py index 1d19323d298..30e1162b1d2 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/test.py +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -6,6 +6,7 @@ import string from helpers.network import NetThroughput import subprocess import time +import statistics cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True) @@ -42,18 +43,14 @@ def test_limited_fetch_single_table(start_cluster): for i in range(10): n1_in, n1_out = n1_net.measure_speed('megabytes') n2_in, n2_out = n2_net.measure_speed('megabytes') - print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical")) print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") print("[N2] input:", n2_in, 'MB/s', "output:", n2_out, "MB/s") n2_fetch_speed.append(n2_in) time.sleep(0.5) - for s in n2_fetch_speed: - if s > 20: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert False, "We exceeded max fetch speed for more than 10MB/s. Must be around 10 (+- 5), got " + str(s) - # cannot check lower bound because it can vary depending on host/build etc. - else: - print("Fetch speed OK") + mean_speed = statistics.mean(n2_fetch_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert mean_speed <= 15, "We exceeded max fetch speed for more than 10MB/s. Must be around 10 (+- 5), got " + str(mean_speed) finally: for node in [node1, node2]: @@ -82,12 +79,9 @@ def test_limited_send_single_table(start_cluster): n1_sends_speed.append(n1_out) time.sleep(0.5) - for s in n1_sends_speed: - if s > 10: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert False, "We exceeded max send speed for more than 5MB/s. Must be around 5 (+- 5), got " + str(s) - # cannot check lower bound because it can vary depending on host/build etc. - else: - print("Send speed OK") + mean_speed = statistics.mean(n1_sends_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert mean_speed <= 10, "We exceeded max send speed for more than 5MB/s. Must be around 5 (+- 5), got " + str(mean_speed) finally: for node in [node1, node2]: @@ -120,12 +114,9 @@ def test_limited_fetches_for_server(start_cluster): n3_fetches_speed.append(n3_in) time.sleep(0.5) - for s in n3_fetches_speed: - if s > 20: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert False, "We exceeded max fetch speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(s) - # cannot check lower bound because it can vary depending on host/build etc. - else: - print("Fetch speed OK") + mean_speed = statistics.mean(n3_fetches_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert mean_speed <= 15, "We exceeded max fetch speed for more than 15MB/s. Must be around 5 (+- 10), got " + str(mean_speed) finally: for node in [node1, node3]: @@ -159,12 +150,9 @@ def test_limited_sends_for_server(start_cluster): n3_sends_speed.append(n3_out) time.sleep(0.5) - for s in n3_sends_speed: - if s > 30: # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert False, "We exceeded max send speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(s) - # cannot check lower bound because it can vary depending on host/build etc. - else: - print("Send speed OK") + mean_speed = statistics.mean(n3_sends_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert mean_speed <= 20, "We exceeded max send speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(mean_speed) finally: for node in [node1, node3]: @@ -199,7 +187,6 @@ def test_should_execute_fetch(start_cluster): if fetches_count == "0\n": break - print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical")) print("Fetches count", fetches_count) replication_queue_data.append(node2.query("SELECT count() FROM system.replication_queue WHERE postpone_reason like '%fetches have already throttled%'")) n2_fetch_speed.append(n2_in) From 3ced156df9785e8b2e4cda68e206cda5fa2e2f4c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 27 May 2021 18:20:29 +0300 Subject: [PATCH 235/652] do not auto-apply -OrNull combinator to pure window functions --- src/AggregateFunctions/AggregateFunctionFactory.cpp | 9 +++++++++ .../queries/0_stateless/01591_window_functions.reference | 4 ++++ tests/queries/0_stateless/01591_window_functions.sql | 4 ++++ 3 files changed, 17 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index c84a04cc872..17c3974390b 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -89,6 +89,15 @@ AggregateFunctionPtr AggregateFunctionFactory::get( AggregateFunctionPtr nested_function = getImpl( name, nested_types, nested_parameters, out_properties, has_null_arguments); + + // Pure window functions are not real aggregate functions. Applying + // combinators doesn't make sense for them, they must handle the + // nullability themselves. + if (nested_function->asWindowFunction()) + { + return nested_function; + } + return combinator->transformAggregateFunction(nested_function, out_properties, type_without_low_cardinality, parameters); } diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 24458900350..1defb38c5ae 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -1054,6 +1054,10 @@ settings max_block_size = 3; 13 2 10 12 10 10 143 14 2 10 13 10 10 154 15 3 15 0 15 15 15 +-- careful with auto-application of Null combinator +select lagInFrame(toNullable(1)) over (); +0 +select lagInFrameOrNull(1) over (); -- { serverError 36 } -- case-insensitive SQL-standard synonyms for any and anyLast select number, diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index ec2e463eccd..42216d089e4 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -376,6 +376,10 @@ order by number settings max_block_size = 3; ; +-- careful with auto-application of Null combinator +select lagInFrame(toNullable(1)) over (); +select lagInFrameOrNull(1) over (); -- { serverError 36 } + -- case-insensitive SQL-standard synonyms for any and anyLast select number, From 305134f037bd0e57634e988ef3d23df9081d2817 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 27 May 2021 18:20:50 +0300 Subject: [PATCH 236/652] fix --- tests/integration/test_log_family_hdfs/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_log_family_hdfs/test.py b/tests/integration/test_log_family_hdfs/test.py index fcf64381dd9..3d067f0febe 100644 --- a/tests/integration/test_log_family_hdfs/test.py +++ b/tests/integration/test_log_family_hdfs/test.py @@ -27,7 +27,7 @@ def started_cluster(): def assert_objects_count(started_cluster, objects_count, path='data/'): - fs = HdfsClient(hosts=cluster.hdfs_ip) + fs = HdfsClient(hosts=started_cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') assert objects_count == len(hdfs_objects) From cce04ce9c991c6ed5ec6d975d31483180ee35996 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 27 May 2021 19:05:49 +0300 Subject: [PATCH 237/652] fix ORDER BY after window fuctions over Distributed --- src/Interpreters/InterpreterSelectQuery.cpp | 9 ++++--- ...568_window_functions_distributed.reference | 26 ++++++++++++++++++- .../01568_window_functions_distributed.sql | 9 ++++++- 3 files changed, 39 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 61740284f1b..7407088ba60 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1296,9 +1296,12 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu */ if (from_aggregation_stage) - executeMergeSorted(query_plan, "for ORDER BY"); - else if (!expressions.first_stage && !expressions.need_aggregate && !(query.group_by_with_totals && !aggregate_final)) - executeMergeSorted(query_plan, "for ORDER BY"); + executeMergeSorted(query_plan, "after aggregation stage for ORDER BY"); + else if (!expressions.first_stage + && !expressions.need_aggregate + && !expressions.has_window + && !(query.group_by_with_totals && !aggregate_final)) + executeMergeSorted(query_plan, "for ORDER BY, without aggregation"); else /// Otherwise, just sort. executeOrder( query_plan, diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.reference b/tests/queries/0_stateless/01568_window_functions_distributed.reference index 7d5a95046f7..483e84a2bee 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.reference +++ b/tests/queries/0_stateless/01568_window_functions_distributed.reference @@ -10,7 +10,9 @@ select max(identity(dummy + 1)) over () from remote('127.0.0.{1,2}', system, one 1 1 drop table if exists t_01568; -create table t_01568 engine Memory as select intDiv(number, 3) p, number from numbers(9); +create table t_01568 engine Memory as +select intDiv(number, 3) p, modulo(number, 3) o, number +from numbers(9); select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); 3 2 3 2 @@ -57,4 +59,26 @@ select groupArray(groupArray(number)) over (rows unbounded preceding) from remot [[0,3,6],[1,4,7]] [[0,3,6],[1,4,7],[2,5,8]] select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=2; -- { serverError 48 } +-- proper ORDER BY w/window functions +select p, o, count() over (partition by p) +from remote('127.0.0.{1,2}', '', t_01568) +order by p, o; +0 0 6 +0 0 6 +0 1 6 +0 1 6 +0 2 6 +0 2 6 +1 0 6 +1 0 6 +1 1 6 +1 1 6 +1 2 6 +1 2 6 +2 0 6 +2 0 6 +2 1 6 +2 1 6 +2 2 6 +2 2 6 drop table t_01568; diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.sql b/tests/queries/0_stateless/01568_window_functions_distributed.sql index bc82e1ed6ac..6f38597a7a3 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.sql +++ b/tests/queries/0_stateless/01568_window_functions_distributed.sql @@ -9,7 +9,9 @@ select max(identity(dummy + 1)) over () from remote('127.0.0.{1,2}', system, one drop table if exists t_01568; -create table t_01568 engine Memory as select intDiv(number, 3) p, number from numbers(9); +create table t_01568 engine Memory as +select intDiv(number, 3) p, modulo(number, 3) o, number +from numbers(9); select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); @@ -22,4 +24,9 @@ select groupArray(groupArray(number)) over (rows unbounded preceding) from remot select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=1; select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=2; -- { serverError 48 } +-- proper ORDER BY w/window functions +select p, o, count() over (partition by p) +from remote('127.0.0.{1,2}', '', t_01568) +order by p, o; + drop table t_01568; From 1aeb705b20ef37f0608e6c639dc333b90388bdb4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 May 2021 19:53:58 +0300 Subject: [PATCH 238/652] Fix some tests. --- .../QueryPlan/ReadFromMergeTree.cpp | 165 +++++++++++------- src/Processors/QueryPlan/ReadFromMergeTree.h | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 12 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 20 ++- .../MergeTree/MergeTreeDataSelectExecutor.h | 14 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- .../0_stateless/01651_bugs_from_15889.sql | 10 +- .../01861_explain_pipeline.reference | 10 +- 8 files changed, 141 insertions(+), 102 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6d6ee43acb3..2ad10461613 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -88,9 +88,20 @@ size_t minMarksForConcurrentRead( } +struct ReadFromMergeTree::AnalysisResult +{ + RangesInDataParts parts_with_ranges; + MergeTreeDataSelectSamplingData sampling; + bool sample_factor_column_queried = false; + String query_id; + IndexStats index_stats; + Names column_names_to_read; + ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default; +}; + ReadFromMergeTree::ReadFromMergeTree( const SelectQueryInfo & query_info_, - const PartitionIdToMaxBlock * max_block_numbers_to_read_, + std::shared_ptr max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, StorageMetadataPtr metadata_snapshot_, @@ -107,7 +118,7 @@ ReadFromMergeTree::ReadFromMergeTree( data_.getPartitionValueType(), virt_column_names_)}) , query_info(std::move(query_info_)) - , max_block_numbers_to_read(max_block_numbers_to_read_) + , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , context(std::move(context_)) , data(data_) , metadata_snapshot(std::move(metadata_snapshot_)) @@ -842,26 +853,26 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( return Pipe::unitePipes(std::move(partition_pipes)); } -void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts) const { - auto parts = std::move(prepared_parts); + AnalysisResult result; + size_t total_parts = parts.size(); auto part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); if (part_values && part_values->empty()) - { - pipeline.init(Pipe(std::make_shared(getOutputStream().header))); - return; - } + return result; + + result.column_names_to_read = real_column_names; /// If there are only virtual columns in the query, you must request at least one non-virtual one. - if (real_column_names.empty()) + if (result.column_names_to_read.empty()) { NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); - real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); } - metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); + metadata_snapshot->check(result.column_names_to_read, data.getVirtuals(), data.getStorageID()); // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); @@ -881,28 +892,26 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; MergeTreeDataSelectExecutor::filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, result.index_stats); - bool sample_factor_column_queried = false; for (const auto & col : virt_column_names) if (col == "_sample_factor") - sample_factor_column_queried = true; + result.sample_factor_column_queried = true; - auto sampling = MergeTreeDataSelectExecutor::getSampling( + result.sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, - data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + data, log, result.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); - if (sampling.read_nothing) - { - pipeline.init(Pipe(std::make_shared(getOutputStream().header))); - return; - } + if (result.sampling.read_nothing) + return result; size_t total_marks_pk = 0; for (const auto & part : parts) total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); - auto parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( + size_t parts_before_pk = parts.size(); + + result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), metadata_snapshot, query_info, @@ -911,18 +920,18 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build settings.reader_settings, log, settings.num_streams, - index_stats, + result.index_stats, true); size_t sum_marks_pk = total_marks_pk; - for (const auto & stat : index_stats) + for (const auto & stat : result.index_stats) if (stat.type == IndexType::PrimaryKey) sum_marks_pk = stat.num_granules_after; size_t sum_marks = 0; size_t sum_ranges = 0; - for (const auto & part : parts_with_ranges) + for (const auto & part : result.parts_with_ranges) { sum_ranges += part.ranges.size(); sum_marks += part.getMarksCount(); @@ -931,31 +940,53 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build LOG_DEBUG( log, "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - parts.size(), + parts_before_pk, total_parts, - parts_with_ranges.size(), + result.parts_with_ranges.size(), sum_marks_pk, total_marks_pk, sum_marks, sum_ranges); - String query_id = MergeTreeDataSelectExecutor::checkLimits(data, parts_with_ranges, context); + result.query_id = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); - ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); + ProfileEvents::increment(ProfileEvents::SelectedParts, result.parts_with_ranges.size()); ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); + const auto & input_order_info = query_info.input_order_info + ? query_info.input_order_info + : (query_info.projection ? query_info.projection->input_order_info : nullptr); + + const auto & q_settings = context->getSettingsRef(); + if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + result.read_type = (input_order_info->direction > 0) ? ReadType::InOrder + : ReadType::InReverseOrder; + + return result; +} + +void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +{ + auto result = selectRangesToRead(prepared_parts); + if (result.parts_with_ranges.empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. /// NOTE: It may lead to double computation of expressions. ActionsDAGPtr result_projection; - Names column_names_to_read = real_column_names; - if (!select.final() && sampling.use_sampling) + Names column_names_to_read = std::move(result.column_names_to_read); + const auto & select = query_info.query->as(); + if (!select.final() && result.sampling.use_sampling) { /// Add columns needed for `sample_by_ast` to `column_names_to_read`. /// Skip this if final was used, because such columns were already added from PK. - std::vector add_columns = sampling.filter_expression->getRequiredColumns().getNames(); + std::vector add_columns = result.sampling.filter_expression->getRequiredColumns().getNames(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); std::sort(column_names_to_read.begin(), column_names_to_read.end()); column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), @@ -985,7 +1016,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); pipe = spreadMarkRangesAmongStreamsFinal( - std::move(parts_with_ranges), + std::move(result.parts_with_ranges), column_names_to_read, result_projection); } @@ -999,7 +1030,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false); pipe = spreadMarkRangesAmongStreamsWithOrder( - std::move(parts_with_ranges), + std::move(result.parts_with_ranges), column_names_to_read, sorting_key_prefix_expr, result_projection, @@ -1008,7 +1039,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build else { pipe = spreadMarkRangesAmongStreams( - std::move(parts_with_ranges), + std::move(result.parts_with_ranges), column_names_to_read); } @@ -1018,15 +1049,15 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build return; } - if (sampling.use_sampling) + if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(sampling.filter_expression); + auto sampling_actions = std::make_shared(result.sampling.filter_expression); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( header, sampling_actions, - sampling.filter_function->getColumnName(), + result.sampling.filter_function->getColumnName(), false); }); } @@ -1041,12 +1072,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build } /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. - if (sample_factor_column_queried) + if (result.sample_factor_column_queried) { ColumnWithTypeAndName column; column.name = "_sample_factor"; column.type = std::make_shared(); - column.column = column.type->createColumnConst(0, Field(sampling.used_sample_factor)); + column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); auto adding_column_action = std::make_shared(adding_column_dag); @@ -1073,8 +1104,8 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build processors.emplace_back(processor); // Attach QueryIdHolder if needed - if (!query_id.empty()) - pipe.addQueryIdHolder(std::make_shared(query_id, data)); + if (!result.query_id.empty()) + pipe.addQueryIdHolder(std::make_shared(result.query_id, data)); pipeline.init(std::move(pipe)); } @@ -1098,45 +1129,50 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type) __builtin_unreachable(); } -// static const char * readTypeToString(ReadFromMergeTree::ReadType type) -// { -// switch (type) -// { -// case ReadFromMergeTree::ReadType::Default: -// return "Default"; -// case ReadFromMergeTree::ReadType::InOrder: -// return "InOrder"; -// case ReadFromMergeTree::ReadType::InReverseOrder: -// return "InReverseOrder"; -// } +static const char * readTypeToString(ReadFromMergeTree::ReadType type) +{ + switch (type) + { + case ReadFromMergeTree::ReadType::Default: + return "Default"; + case ReadFromMergeTree::ReadType::InOrder: + return "InOrder"; + case ReadFromMergeTree::ReadType::InReverseOrder: + return "InReverseOrder"; + } -// __builtin_unreachable(); -// } + __builtin_unreachable(); +} void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const { + auto result = selectRangesToRead(prepared_parts); std::string prefix(format_settings.offset, format_settings.indent_char); - //format_settings.out << prefix << "ReadType: " << readTypeToString(read_type) << '\n'; + format_settings.out << prefix << "ReadType: " << readTypeToString(result.read_type) << '\n'; - if (!index_stats.empty()) + if (!result.index_stats.empty()) { - format_settings.out << prefix << "Parts: " << index_stats.back().num_parts_after << '\n'; - format_settings.out << prefix << "Granules: " << index_stats.back().num_granules_after << '\n'; + format_settings.out << prefix << "Parts: " << result.index_stats.back().num_parts_after << '\n'; + format_settings.out << prefix << "Granules: " << result.index_stats.back().num_granules_after << '\n'; } } void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { - //map.add("Read Type", readTypeToString(read_type)); - if (!index_stats.empty()) + auto result = selectRangesToRead(prepared_parts); + map.add("Read Type", readTypeToString(result.read_type)); + if (!result.index_stats.empty()) { - map.add("Parts", index_stats.back().num_parts_after); - map.add("Granules", index_stats.back().num_granules_after); + map.add("Parts", result.index_stats.back().num_parts_after); + map.add("Granules", result.index_stats.back().num_granules_after); } } void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const { + auto result = selectRangesToRead(prepared_parts); + auto index_stats = std::move(result.index_stats); + std::string prefix(format_settings.offset, format_settings.indent_char); if (!index_stats.empty()) { @@ -1186,6 +1222,9 @@ void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const { + auto result = selectRangesToRead(prepared_parts); + auto index_stats = std::move(result.index_stats); + if (!index_stats.empty()) { /// Do not print anything if no indexes is applied. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index e9341e46770..ef5cc5dc70c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -72,7 +72,7 @@ public: ReadFromMergeTree( const SelectQueryInfo & query_info_, - const PartitionIdToMaxBlock * max_block_numbers_to_read_, + std::shared_ptr max_block_numbers_to_read_, ContextPtr context_, const MergeTreeData & data_, StorageMetadataPtr metadata_snapshot_, @@ -97,7 +97,7 @@ public: private: SelectQueryInfo query_info; - const PartitionIdToMaxBlock * max_block_numbers_to_read; + std::shared_ptr max_block_numbers_to_read; ContextPtr context; const MergeTreeData & data; StorageMetadataPtr metadata_snapshot; @@ -106,7 +106,6 @@ private: Names real_column_names; MergeTreeData::DataPartsVector prepared_parts; PrewhereInfoPtr prewhere_info; - IndexStats index_stats; Names virt_column_names; Settings settings; @@ -134,6 +133,9 @@ private: RangesInDataParts && parts, const Names & column_names, ActionsDAGPtr & out_projection); + + struct AnalysisResult; + AnalysisResult selectRangesToRead(MergeTreeData::DataPartsVector parts) const; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e16bbb640e2..3f86432e2ae 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3818,7 +3818,7 @@ static void selectBestProjection( const Names & required_columns, ProjectionCandidate & candidate, ContextPtr query_context, - const PartitionIdToMaxBlock * max_added_blocks, + std::shared_ptr max_added_blocks, const Settings & settings, const MergeTreeData::DataPartsVector & parts, ProjectionCandidate *& selected_candidate, @@ -4097,11 +4097,11 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( // First build a MergeTreeDataSelectCache to check if a projection is indeed better than base // query_info.merge_tree_data_select_cache = std::make_unique(); - std::unique_ptr max_added_blocks; + std::shared_ptr max_added_blocks; if (settings.select_sequential_consistency) { if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) - max_added_blocks = std::make_unique(replicated->getMaxAddedBlocks()); + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); } auto parts = getDataPartsVector(); @@ -4122,7 +4122,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( analysis_result.required_columns, candidate, query_context, - max_added_blocks.get(), + max_added_blocks, settings, parts, selected_candidate, @@ -4143,7 +4143,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( query_info, // TODO syntax_analysis_result set in index query_context, settings.max_threads, - max_added_blocks.get()); + max_added_blocks); // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. @@ -4160,7 +4160,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( analysis_result.required_columns, candidate, query_context, - max_added_blocks.get(), + max_added_blocks, settings, parts, selected_candidate, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8a3550fc511..cb2ead7952a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -139,7 +139,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const UInt64 max_block_size, const unsigned num_streams, QueryProcessingStage::Enum processed_stage, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read) const { const auto & settings = context->getSettingsRef(); auto parts = data.getDataPartsVector(); @@ -670,8 +670,8 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, - ContextPtr & context, - ContextPtr & query_context, + const ContextPtr & context, + const ContextPtr & query_context, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, @@ -766,7 +766,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, - ContextPtr & context, + const ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, @@ -993,7 +993,10 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd return parts_with_ranges; } -String MergeTreeDataSelectExecutor::checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context) +String MergeTreeDataSelectExecutor::checkLimits( + const MergeTreeData & data, + const RangesInDataParts & parts_with_ranges, + const ContextPtr & context) { const auto & settings = context->getSettingsRef(); // Check limitations. query_id is used as the quota RAII's resource key. @@ -1092,7 +1095,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( const SelectQueryInfo & query_info, ContextPtr context, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read) const { size_t total_parts = parts.size(); if (total_parts == 0) @@ -1137,7 +1140,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( ReadFromMergeTree::IndexStats index_stats; filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read, log, index_stats); + metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, index_stats); auto sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, @@ -1173,7 +1176,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( ContextPtr context, const UInt64 max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read) const { size_t total_parts = parts.size(); if (total_parts == 0) @@ -1207,6 +1210,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, //.min_marks_for_concurrent_read = settings.min_marks_for_concurrent_read, .use_uncompressed_cache = settings.use_uncompressed_cache, + .force_primary_key = settings.force_primary_key, .reader_settings = reader_settings, .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), }; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 3e8076de8d3..077584039a1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -53,7 +53,7 @@ public: UInt64 max_block_size, unsigned num_streams, QueryProcessingStage::Enum processed_stage, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr) const; size_t estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, @@ -63,7 +63,7 @@ public: const SelectQueryInfo & query_info, ContextPtr context, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr) const; QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, @@ -74,7 +74,7 @@ public: ContextPtr context, UInt64 max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr) const; private: const MergeTreeData & data; @@ -196,8 +196,8 @@ public: const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, - ContextPtr & context, - ContextPtr & query_context, + const ContextPtr & context, + const ContextPtr & query_context, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, @@ -208,7 +208,7 @@ public: MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, - ContextPtr & context, + const ContextPtr & context, KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, @@ -227,7 +227,7 @@ public: NamesAndTypesList available_real_columns, ContextPtr context); - static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, ContextPtr & context); + static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context); }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3adf3026b23..e38589a5143 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4377,9 +4377,9 @@ void StorageReplicatedMergeTree::read( */ if (local_context->getSettingsRef().select_sequential_consistency) { - auto max_added_blocks = getMaxAddedBlocks(); + auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); if (auto plan = reader.read( - column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, &max_added_blocks)) + column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, std::move(max_added_blocks))) query_plan = std::move(*plan); return; } diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 1fbf669a1b8..2764ed61291 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -8,7 +8,7 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); -SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 20 } +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } @@ -45,7 +45,7 @@ SYSTEM FLUSH LOGS; WITH concat(addressToLine(arrayJoin(trace) AS addr), '#') AS symbol SELECT count() > 7 FROM trace_log AS t -WHERE (query_id = +WHERE (query_id = ( SELECT [NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL], @@ -60,7 +60,7 @@ WHERE (query_id = WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM trace_log AS t -WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterday()) AND (trace_type = 'Memory') AND (query_id = +WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterday()) AND (trace_type = 'Memory') AND (query_id = ( SELECT ignore(ignore(ignore(ignore(65536)), ignore(65537), ignore(2)), ''), @@ -82,7 +82,7 @@ WITH ( WHERE current_database = currentDatabase() ORDER BY query_start_time DESC LIMIT 1 - ) AS time_with_microseconds, + ) AS time_with_microseconds, ( SELECT inf, @@ -101,7 +101,7 @@ WITH ( WHERE current_database = currentDatabase() ORDER BY query_start_time DESC LIMIT 1 - ) AS time_with_microseconds, + ) AS time_with_microseconds, ( SELECT query_start_time FROM system.query_log diff --git a/tests/queries/0_stateless/01861_explain_pipeline.reference b/tests/queries/0_stateless/01861_explain_pipeline.reference index 8d755f807c0..9d62fb9f6b8 100644 --- a/tests/queries/0_stateless/01861_explain_pipeline.reference +++ b/tests/queries/0_stateless/01861_explain_pipeline.reference @@ -1,13 +1,10 @@ (Expression) ExpressionTransform (SettingQuotaAndLimits) - (Expression) + (ReadFromMergeTree) ExpressionTransform - (MergingFinal) ReplacingSorted 2 → 1 - (Expression) ExpressionTransform × 2 - (ReadFromMergeTree) MergeTree × 2 0 → 1 0 0 1 1 @@ -19,13 +16,10 @@ ExpressionTransform (Expression) ExpressionTransform × 2 (SettingQuotaAndLimits) - (Expression) + (ReadFromMergeTree) ExpressionTransform × 2 - (MergingFinal) ReplacingSorted × 2 2 → 1 Copy × 2 1 → 2 AddingSelector × 2 - (Expression) ExpressionTransform × 2 - (ReadFromMergeTree) MergeTree × 2 0 → 1 From 1ed00b06e091c47cdfdf6143cd0bab63d22dec62 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 May 2021 20:56:06 +0300 Subject: [PATCH 239/652] Fix tests. --- ...materialized_view_select_extra_columns.sql | 2 +- ...1_mergetree_read_in_order_spread.reference | 21 +++++-------------- .../01576_alias_column_rewrite.reference | 20 ++++-------------- 3 files changed, 10 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql b/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql index 10bda34e6b9..4b7ea127190 100644 --- a/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql +++ b/tests/queries/0_stateless/01019_materialized_view_select_extra_columns.sql @@ -28,7 +28,7 @@ FROM mv_extra_columns_src; INSERT INTO mv_extra_columns_src VALUES (0, 0), (1, 1), (2, 2); SELECT * FROM mv_extra_columns_dst ORDER by v; -SELECT * FROM mv_extra_columns_view; -- { serverError 16 } +SELECT * FROM mv_extra_columns_view; -- { serverError 10 } DROP TABLE mv_extra_columns_view; DROP TABLE mv_extra_columns_src; diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference index 835e2af269a..2843b305f0a 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -7,22 +7,11 @@ ExpressionTransform (Expression) ExpressionTransform × 3 (SettingQuotaAndLimits) - (Expression) - ExpressionTransform × 3 - (Union) - (MergingSorted) - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTree 0 → 1 - (MergingSorted) + (ReadFromMergeTree) + ExpressionTransform × 4 + MergeTree 0 → 1 MergingSortedTransform 2 → 1 - (Expression) ExpressionTransform × 2 - (ReadFromMergeTree) MergeTree × 2 0 → 1 - (MergingSorted) - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTree 0 → 1 + ExpressionTransform + MergeTree 0 → 1 diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index c5679544e1d..9d3db8c1d00 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -34,19 +34,13 @@ Expression (Projection) FinishSorting Expression ((Before ORDER BY + Add table aliases)) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree Expression (Projection) Limit (preliminary LIMIT) FinishSorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree optimize_aggregation_in_order Expression ((Projection + Before ORDER BY)) Aggregating @@ -57,18 +51,12 @@ Expression ((Projection + Before ORDER BY)) Aggregating Expression ((Before GROUP BY + Add table aliases)) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree Expression ((Projection + Before ORDER BY)) Aggregating Expression (Before GROUP BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - ReadFromMergeTree - ReadFromMergeTree - ReadFromMergeTree + ReadFromMergeTree second-index 1 1 From 6bf62d865330f48648899f452aca4ba4ea218d80 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 27 May 2021 21:39:06 +0300 Subject: [PATCH 240/652] fix --- src/AggregateFunctions/AggregateFunctionFactory.cpp | 6 ++++-- tests/queries/0_stateless/01591_window_functions.reference | 2 ++ tests/queries/0_stateless/01591_window_functions.sql | 1 + 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 17c3974390b..d0e45053695 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -92,8 +92,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get( // Pure window functions are not real aggregate functions. Applying // combinators doesn't make sense for them, they must handle the - // nullability themselves. - if (nested_function->asWindowFunction()) + // nullability themselves. Another special case is functions from Nothing + // that are rewritten to AggregateFunctionNothing, in this case + // nested_function is nullptr. + if (nested_function && nested_function->asWindowFunction()) { return nested_function; } diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 1defb38c5ae..fa909972678 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -1058,6 +1058,8 @@ settings max_block_size = 3; select lagInFrame(toNullable(1)) over (); 0 select lagInFrameOrNull(1) over (); -- { serverError 36 } +select intDiv(1, NULL) x, toTypeName(x), max(x) over (); +\N Nullable(Nothing) \N -- case-insensitive SQL-standard synonyms for any and anyLast select number, diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index 42216d089e4..05f4cb49252 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -379,6 +379,7 @@ settings max_block_size = 3; -- careful with auto-application of Null combinator select lagInFrame(toNullable(1)) over (); select lagInFrameOrNull(1) over (); -- { serverError 36 } +select intDiv(1, NULL) x, toTypeName(x), max(x) over (); -- case-insensitive SQL-standard synonyms for any and anyLast select From 764902c8d3bd44262ae2df36345a5870d15758d8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 28 May 2021 00:25:17 +0300 Subject: [PATCH 241/652] Remove print --- tests/integration/test_replicated_fetches_bandwidth/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py index 30e1162b1d2..80dbd11b41a 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/test.py +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -192,7 +192,6 @@ def test_should_execute_fetch(start_cluster): n2_fetch_speed.append(n2_in) time.sleep(0.5) - print(node2.query("SELECT * FROM system.replicated_fetches FORMAT Vertical")) node2.query("SYSTEM SYNC REPLICA should_execute_table") assert any(int(f.strip()) != 0 for f in replication_queue_data) assert node2.query("SELECT COUNT() FROM should_execute_table") == "630\n" From ce11f35dcc64eeb04f2b5b48931fffe10211d704 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 11:11:17 +0300 Subject: [PATCH 242/652] Convert header from MergeTree to expected from query plan. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 2ad10461613..30281a394e8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1100,6 +1100,20 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }); } + if (!isCompatibleHeader(pipe.getHeader(), getOutputStream().header)) + { + auto converting_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + pipe.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header, converting_actions); + }); + } + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); From 2767dc1988f0b1af9a54111389ecf01caa0d4dcb Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 28 May 2021 17:02:25 +0800 Subject: [PATCH 243/652] Update tests/queries/0_stateless/01872_initial_query_start_time.sh Co-authored-by: Azat Khuzhin --- tests/queries/0_stateless/01872_initial_query_start_time.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01872_initial_query_start_time.sh b/tests/queries/0_stateless/01872_initial_query_start_time.sh index dbfb89a05a1..8b04160812a 100755 --- a/tests/queries/0_stateless/01872_initial_query_start_time.sh +++ b/tests/queries/0_stateless/01872_initial_query_start_time.sh @@ -21,7 +21,7 @@ select anyIf(initial_query_start_time, is_initial_query) = anyIf(initial_query_start_time, not is_initial_query), anyIf(initial_query_start_time_microseconds, is_initial_query) = anyIf(initial_query_start_time_microseconds, not is_initial_query) from system.query_log -where initial_query_id = '$query_id' and type = 'QueryFinish'; +where event_date = today() and event_time >= now() - interval 1 hour and initial_query_id = '$query_id' and type = 'QueryFinish'; " ${CLICKHOUSE_CLIENT} -q "drop table m" From aad9dda479087d522347afa51afd7dfd42111b56 Mon Sep 17 00:00:00 2001 From: Daria Mozhaeva Date: Fri, 28 May 2021 12:14:28 +0300 Subject: [PATCH 244/652] D:/Git/en/date-time-functions --- .../functions/date-time-functions.md | 334 ++++++++++++++---- 1 file changed, 266 insertions(+), 68 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 683fad8564b..2d793875e75 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -23,13 +23,53 @@ SELECT └─────────────────────┴────────────┴────────────┴─────────────────────┘ ``` +## timeZone {#timezone} + +Returns the timezone of the server. + +**Syntax** + +``` sql +timeZone() +``` + +Alias: `timezone`. + +**Returned value** + +- Timezone. + +Type: [String](../../sql-reference/data-types/string.md). + ## toTimeZone {#totimezone} -Convert time or date and time to the specified time zone. The time zone is an attribute of the Date/DateTime types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly. +Converts time or date and time to the specified time zone. The time zone is an attribute of the `Date` and `DateTime` data types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly. + +**Syntax** + +``` sql +toTimezone(value, timezone) +``` + +Alias: `toTimezone`. + +**Arguments** + +- `value` — Time or date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). +- `timezone` — Timezone for the returned value. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- Date and time. + +Type: [DateTime](../../sql-reference/data-types/datetime.md). + +**Example** + +Query: ```sql -SELECT - toDateTime('2019-01-01 00:00:00', 'UTC') AS time_utc, +SELECT toDateTime('2019-01-01 00:00:00', 'UTC') AS time_utc, toTypeName(time_utc) AS type_utc, toInt32(time_utc) AS int32utc, toTimeZone(time_utc, 'Asia/Yekaterinburg') AS time_yekat, @@ -40,6 +80,7 @@ SELECT toInt32(time_samoa) AS int32samoa FORMAT Vertical; ``` +Result: ```text Row 1: @@ -57,44 +98,138 @@ int32samoa: 1546300800 `toTimeZone(time_utc, 'Asia/Yekaterinburg')` changes the `DateTime('UTC')` type to `DateTime('Asia/Yekaterinburg')`. The value (Unixtimestamp) 1546300800 stays the same, but the string representation (the result of the toString() function) changes from `time_utc: 2019-01-01 00:00:00` to `time_yekat: 2019-01-01 05:00:00`. +## timeZoneOf {#timezoneof} + +Returns the timezone name of [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md) data types. + +**Syntax** + +``` sql +timeZoneOf(value) +``` + +Alias: `timezoneOf`. + +**Arguments** + +- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md). + +**Returned value** + +- Timezone name. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: +``` sql +SELECT timezoneOf(now()); +``` + +Result: +``` text +┌─timezoneOf(now())─┐ +│ Etc/UTC │ +└───────────────────┘ +``` + +## timeZoneOffset {#timezoneoffset} + +Returns a timezone offset in seconds from [UTC](https://en.wikipedia.org/wiki/Coordinated_Universal_Time). The function takes into account [daylight saving time](https://en.wikipedia.org/wiki/Daylight_saving_time) and historical timezone changes at the specified date and time. +[IANA timezone database](https://www.iana.org/time-zones) is used to calculate the offset. + +**Syntax** + +``` sql +timeZoneOffset(value) +``` + +Alias: `timezoneOffset`. + +**Arguments** + +- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md). + +**Returned value** + +- Offset from UTC in seconds. + +Type: [Int32](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT toDateTime('2021-04-21 10:20:30', 'America/New_York') AS Time, toTypeName(Time) AS Type, + timeZoneOffset(Time) AS Offset_in_seconds, (Offset_in_seconds / 3600) AS Offset_in_hours; +``` + +Result: + +``` text +┌────────────────Time─┬─Type─────────────────────────┬─Offset_in_seconds─┬─Offset_in_hours─┐ +│ 2021-04-21 10:20:30 │ DateTime('America/New_York') │ -14400 │ -4 │ +└─────────────────────┴──────────────────────────────┴───────────────────┴─────────────────┘ +``` + ## toYear {#toyear} Converts a date or date with time to a UInt16 number containing the year number (AD). +Alias: `YEAR`. + ## toQuarter {#toquarter} Converts a date or date with time to a UInt8 number containing the quarter number. +Alias: `QUARTER`. + ## toMonth {#tomonth} Converts a date or date with time to a UInt8 number containing the month number (1-12). +Alias: `MONTH`. + ## toDayOfYear {#todayofyear} Converts a date or date with time to a UInt16 number containing the number of the day of the year (1-366). +Alias: `DAYOFYEAR`. + ## toDayOfMonth {#todayofmonth} Converts a date or date with time to a UInt8 number containing the number of the day of the month (1-31). +Aliases: `DAYOFMONTH`, `DAY`. + ## toDayOfWeek {#todayofweek} Converts a date or date with time to a UInt8 number containing the number of the day of the week (Monday is 1, and Sunday is 7). +Alias: `DAYOFWEEK`. + ## toHour {#tohour} Converts a date with time to a UInt8 number containing the number of the hour in 24-hour time (0-23). This function assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true – even in Moscow the clocks were twice changed at a different time). +Alias: `HOUR`. + ## toMinute {#tominute} Converts a date with time to a UInt8 number containing the number of the minute of the hour (0-59). +Alias: `MINUTE`. + ## toSecond {#tosecond} Converts a date with time to a UInt8 number containing the number of the second in the minute (0-59). Leap seconds are not accounted for. +Alias: `SECOND`. + ## toUnixTimestamp {#to-unix-timestamp} For DateTime argument: converts value to the number with type UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time). @@ -189,7 +324,7 @@ Truncates sub-seconds. toStartOfSecond(value[, timezone]) ``` -**Parameters** +**Arguments** - `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). - `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md). @@ -331,7 +466,7 @@ For mode values with a meaning of “contains January 1”, the week contains Ja toWeek(date, [, mode][, Timezone]) ``` -**Parameters** +**Arguments** - `date` – Date or DateTime. - `mode` – Optional parameter, Range of values is \[0,9\], default is 0. @@ -373,15 +508,15 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d Truncates date and time data to the specified part of date. -**Syntax** +**Syntax** ``` sql date_trunc(unit, value[, timezone]) ``` -Alias: `dateTrunc`. +Alias: `dateTrunc`. -**Parameters** +**Arguments** - `unit` — The type of interval to truncate the result. [String Literal](../syntax.md#syntax-string-literal). Possible values: @@ -434,41 +569,55 @@ Result: └─────────────────────┴────────────────────────────────────────────┘ ``` -**See also** +**See Also** - [toStartOfInterval](#tostartofintervaltime-or-data-interval-x-unit-time-zone) ## date\_add {#date_add} -Adds specified date/time interval to the provided date. +Adds the time interval or date interval to the provided date or date with time. -**Syntax** +**Syntax** ``` sql date_add(unit, value, date) ``` -Aliases: `dateAdd`, `DATE_ADD`. +Aliases: `dateAdd`, `DATE_ADD`. -**Parameters** +**Arguments** - `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). + Possible values: - Supported values: second, minute, hour, day, week, month, quarter, year. -- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md) -- `date` — [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` +- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md). +- `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Returned value** -Returns Date or DateTime with `value` expressed in `unit` added to `date`. +Date or date with time obtained by adding `value`, expressed in `unit`, to `date`. + +Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Example** +Query: + ```sql -select date_add(YEAR, 3, toDate('2018-01-01')); +SELECT date_add(YEAR, 3, toDate('2018-01-01')); ``` +Result: + ```text ┌─plus(toDate('2018-01-01'), toIntervalYear(3))─┐ │ 2021-01-01 │ @@ -477,7 +626,7 @@ select date_add(YEAR, 3, toDate('2018-01-01')); ## date\_diff {#date_diff} -Returns the difference between two Date or DateTime values. +Returns the difference between two dates or dates with time values. **Syntax** @@ -485,25 +634,33 @@ Returns the difference between two Date or DateTime values. date_diff('unit', startdate, enddate, [timezone]) ``` -Aliases: `dateDiff`, `DATE_DIFF`. +Aliases: `dateDiff`, `DATE_DIFF`. -**Parameters** +**Arguments** -- `unit` — The type of interval for result [String](../../sql-reference/data-types/string.md). +- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). + Possible values: - Supported values: second, minute, hour, day, week, month, quarter, year. + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` - `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). - `enddate` — The second time value to subtract from (the minuend). [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). -- `timezone` — Optional parameter. If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. [String](../../sql-reference/data-types/string.md). **Returned value** Difference between `enddate` and `startdate` expressed in `unit`. -Type: `int`. +Type: [Int](../../sql-reference/data-types/int-uint.md). **Example** @@ -523,7 +680,7 @@ Result: ## date\_sub {#date_sub} -Subtracts a time/date interval from the provided date. +Subtracts the time interval or date interval from the provided date or date with time. **Syntax** @@ -531,19 +688,30 @@ Subtracts a time/date interval from the provided date. date_sub(unit, value, date) ``` -Aliases: `dateSub`, `DATE_SUB`. +Aliases: `dateSub`, `DATE_SUB`. -**Parameters** +**Arguments** - `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md). + Possible values: - Supported values: second, minute, hour, day, week, month, quarter, year. -- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md) -- `date` — [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md) to subtract value from. + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` + +- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md). +- `date` — The date or date with time from which `value` is subtracted. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Returned value** -Returns Date or DateTime with `value` expressed in `unit` subtracted from `date`. +Date or date with time obtained by subtracting `value`, expressed in `unit`, from `date`. + +Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Example** @@ -565,32 +733,46 @@ Result: Adds the specified time value with the provided date or date time value. -**Syntax** +**Syntax** ``` sql timestamp_add(date, INTERVAL value unit) ``` -Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. +Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. -**Parameters** - -- `date` — Date or Date with time - [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). -- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md) +**Arguments** + +- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md). - `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). + Possible values: - Supported values: second, minute, hour, day, week, month, quarter, year. + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` **Returned value** -Returns Date or DateTime with the specified `value` expressed in `unit` added to `date`. - +Date or date with time with the specified `value` expressed in `unit` added to `date`. + +Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + **Example** +Query: + ```sql select timestamp_add(toDate('2018-01-01'), INTERVAL 3 MONTH); ``` +Result: + ```text ┌─plus(toDate('2018-01-01'), toIntervalMonth(3))─┐ │ 2018-04-01 │ @@ -599,51 +781,66 @@ select timestamp_add(toDate('2018-01-01'), INTERVAL 3 MONTH); ## timestamp\_sub {#timestamp_sub} -Returns the difference between two dates in the specified unit. +Subtracts the time interval from the provided date or date with time. -**Syntax** +**Syntax** ``` sql timestamp_sub(unit, value, date) ``` -Aliases: `timeStampSub`, `TIMESTAMP_SUB`. +Aliases: `timeStampSub`, `TIMESTAMP_SUB`. -**Parameters** +**Arguments** -- `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). +- `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md). + Possible values: - Supported values: second, minute, hour, day, week, month, quarter, year. -- `value` - Value in specified unit - [Int](../../sql-reference/data-types/int-uint.md). -- `date`- [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` + +- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md). +- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Returned value** -Difference between `date` and the specified `value` expressed in `unit`. +Date or date with time obtained by subtracting `value`, expressed in `unit`, from `date`. + +Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). **Example** +Query: + ```sql select timestamp_sub(MONTH, 5, toDateTime('2018-12-18 01:02:03')); ``` +Result: + ```text ┌─minus(toDateTime('2018-12-18 01:02:03'), toIntervalMonth(5))─┐ │ 2018-07-18 01:02:03 │ └──────────────────────────────────────────────────────────────┘ ``` - + ## now {#now} -Returns the current date and time. +Returns the current date and time. -**Syntax** +**Syntax** ``` sql now([timezone]) ``` -**Parameters** +**Arguments** - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md). @@ -756,7 +953,7 @@ This is necessary for searching for pageviews in the corresponding session. ## formatDateTime {#formatdatetime} -Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. +Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. **Syntax** @@ -776,7 +973,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %C | year divided by 100 and truncated to integer (00-99) | 20 | | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | -| %e | day of the month, space-padded ( 1-31) | 2 | +| %e | day of the month, space-padded ( 1-31) |   2 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | | %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | | %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | @@ -815,31 +1012,32 @@ Result: └────────────────────────────────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) - ## FROM\_UNIXTIME {#fromunixfime} -When there is only single argument of integer type, it act in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md). -type. +Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. -For example: +**Example:** + +Query: ```sql -SELECT FROM_UNIXTIME(423543535) +SELECT FROM_UNIXTIME(423543535); ``` +Result: + ```text ┌─FROM_UNIXTIME(423543535)─┐ │ 1983-06-04 10:58:55 │ └──────────────────────────┘ ``` -When there are two arguments, first is integer or DateTime, second is constant format string, it act in the same way as `formatDateTime` and return `String` type. +When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. For example: ```sql -SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime +SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; ``` ```text @@ -858,7 +1056,7 @@ Converts a [Proleptic Gregorian calendar](https://en.wikipedia.org/wiki/Prolepti toModifiedJulianDay(date) ``` -**Parameters** +**Arguments** - `date` — Date in text form. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). @@ -894,7 +1092,7 @@ Similar to [toModifiedJulianDay()](#tomodifiedjulianday), but instead of raising toModifiedJulianDayOrNull(date) ``` -**Parameters** +**Arguments** - `date` — Date in text form. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). @@ -930,7 +1128,7 @@ Converts a [Modified Julian Day](https://en.wikipedia.org/wiki/Julian_day#Varian fromModifiedJulianDay(day) ``` -**Parameters** +**Arguments** - `day` — Modified Julian Day number. [Any integral types](../../sql-reference/data-types/int-uint.md). @@ -966,7 +1164,7 @@ Similar to [fromModifiedJulianDayOrNull()](#frommodifiedjuliandayornull), but in fromModifiedJulianDayOrNull(day) ``` -**Parameters** +**Arguments** - `day` — Modified Julian Day number. [Any integral types](../../sql-reference/data-types/int-uint.md). From 4e28b7cb0283f16c467c34d63303aab581b9c81a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:23:46 +0300 Subject: [PATCH 245/652] Fix _sample_factor column. --- .../QueryPlan/ReadFromMergeTree.cpp | 80 ++++++++++--------- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 15 +--- 3 files changed, 46 insertions(+), 50 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 30281a394e8..9241009ee87 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -92,7 +92,6 @@ struct ReadFromMergeTree::AnalysisResult { RangesInDataParts parts_with_ranges; MergeTreeDataSelectSamplingData sampling; - bool sample_factor_column_queried = false; String query_id; IndexStats index_stats; Names column_names_to_read; @@ -130,6 +129,13 @@ ReadFromMergeTree::ReadFromMergeTree( , settings(std::move(settings_)) , log(log_) { + if (settings.sample_factor_column_queried) + { + /// Only _sample_factor virtual column is added by ReadFromMergeTree + /// Other virtual columns are added by MergeTreeBaseSelectProcessor. + auto type = std::make_shared(); + output_stream->header.insert({type->createColumn(), type, "_sample_factor"}); + } } Pipe ReadFromMergeTree::readFromPool( @@ -894,13 +900,9 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre MergeTreeDataSelectExecutor::filterPartsByPartition( metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, result.index_stats); - for (const auto & col : virt_column_names) - if (col == "_sample_factor") - result.sample_factor_column_queried = true; - result.sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, - data, log, result.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + data, log, settings.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); if (result.sampling.read_nothing) return result; @@ -1062,55 +1064,61 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }); } - if (result_projection) + Block cur_header = result_projection ? result_projection->getResultColumns() + : pipe.getHeader(); + + auto append_actions = [&result_projection, &cur_header](ActionsDAGPtr actions) { - auto projection_actions = std::make_shared(result_projection); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, projection_actions); - }); - } + if (!result_projection) + result_projection = std::move(actions); + else + result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); + + cur_header = result_projection->getResultColumns(); + }; /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. - if (result.sample_factor_column_queried) + if (settings.sample_factor_column_queried) { ColumnWithTypeAndName column; column.name = "_sample_factor"; column.type = std::make_shared(); column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_action = std::make_shared(adding_column_dag); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, adding_column_action); - }); + auto adding_column = ActionsDAG::makeAddingColumnActions(std::move(column)); + append_actions(std::move(adding_column)); } // TODO There seems to be no place initializing remove_columns_actions - if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) - { - auto remove_columns_action = std::make_shared( - query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); + // if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) + // { + // auto remove_columns_action = std::make_shared( + // query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, remove_columns_action); - }); - } + // pipe.addSimpleTransform([&](const Block & header) + // { + // return std::make_shared(header, remove_columns_action); + // }); + // } - if (!isCompatibleHeader(pipe.getHeader(), getOutputStream().header)) + /// Extra columns may be returned (for example, if sampling is used). + /// Convert pipe to step header structure. + if (!isCompatibleHeader(cur_header, getOutputStream().header)) { - auto converting_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), + auto converting = ActionsDAG::makeConvertingActions( + cur_header.getColumnsWithTypeAndName(), getOutputStream().header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting_actions = std::make_shared(std::move(converting_dag)); - pipe.addSimpleTransform([&](const Block & cur_header) + append_actions(std::move(converting)); + } + + if (result_projection) + { + auto projection_actions = std::make_shared(result_projection); + pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(cur_header, converting_actions); + return std::make_shared(header, projection_actions); }); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index ef5cc5dc70c..0c3814c507f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -50,6 +50,7 @@ public: //size_t min_marks_for_concurrent_read; bool use_uncompressed_cache; bool force_primary_key; + bool sample_factor_column_queried; MergeTreeReaderSettings reader_settings; MergeTreeReadPool::BackoffSettings backoff_settings; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index db9986dd25d..e3c584a4708 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1208,26 +1208,13 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .num_streams = num_streams, .preferred_block_size_bytes = settings.preferred_block_size_bytes, .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - //.min_marks_for_concurrent_read = settings.min_marks_for_concurrent_read, .use_uncompressed_cache = settings.use_uncompressed_cache, .force_primary_key = settings.force_primary_key, + .sample_factor_column_queried = sample_factor_column_queried, .reader_settings = reader_settings, .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), }; - // const SelectQueryInfo & query_info_, - // const MergeTreeDataSelectExecutor::PartitionIdToMaxBlock * max_block_numbers_to_read_, - // ContextPtr context_, - // const MergeTreeData & data_, - // StorageMetadataPtr metadata_snapshot_, - // StorageMetadataPtr metadata_snapshot_base_, - // Names real_column_names_, - // MergeTreeData::DataPartsVector parts_, - // PrewhereInfoPtr prewhere_info_, - // Names virt_column_names_, - // Settings settings_, - // Poco::Logger * log_ - auto read_from_merge_tree = std::make_unique( query_info, max_block_numbers_to_read, From c80ab6166a74950223da737f360322c11cba23a8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:26:44 +0300 Subject: [PATCH 246/652] Fix style. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 9241009ee87..4af3fec6099 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -37,6 +36,7 @@ namespace DB namespace ErrorCodes { extern const int INDEX_NOT_USED; + extern const int LOGICAL_ERROR; } namespace diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e3c584a4708..299abf66049 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1149,7 +1149,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( if (sampling.read_nothing) return 0; - /// Do not init. Ther are not used (cause skip index is ignored) + /// Do not init. It is not used (cause skip index is ignored) MergeTreeReaderSettings reader_settings; auto parts_with_ranges = filterPartsByPrimaryKeyAndSkipIndexes( From 94f1ac5a167df0ed54c5d4ed76ebb8b55764ea5f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:41:07 +0300 Subject: [PATCH 247/652] Remove some commented code. --- .../QueryPlan/ReadFromMergeTree.cpp | 107 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 631 +----------------- .../MergeTree/MergeTreeDataSelectExecutor.h | 73 +- 3 files changed, 51 insertions(+), 760 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4af3fec6099..7e4a8ba5a55 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -39,55 +39,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - -/// Marks are placed whenever threshold on rows or bytes is met. -/// So we have to return the number of marks on whatever estimate is higher - by rows or by bytes. -size_t roundRowsOrBytesToMarks( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity) -{ - size_t res = (rows_setting + rows_granularity - 1) / rows_granularity; - - if (bytes_granularity == 0) - return res; - else - return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); -} -/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks -size_t minMarksForConcurrentRead( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity, - size_t max_marks) -{ - size_t marks = 1; - - if (rows_setting + rows_granularity <= rows_setting) /// overflow - marks = max_marks; - else if (rows_setting) - marks = (rows_setting + rows_granularity - 1) / rows_granularity; - - if (bytes_granularity == 0) - return marks; - else - { - /// Overflow - if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow - return max_marks; - if (bytes_setting) - return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); - else - return marks; - } -} - -} - struct ReadFromMergeTree::AnalysisResult { RangesInDataParts parts_with_ranges; @@ -267,13 +218,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( if (adaptive_parts > parts_with_ranges.size() / 2) index_granularity_bytes = data_settings->index_granularity_bytes; - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( q_settings.merge_tree_max_rows_to_use_cache, q_settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( q_settings.merge_tree_min_rows_for_concurrent_read, q_settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, @@ -333,13 +284,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (adaptive_parts > parts_with_ranges.size() / 2) index_granularity_bytes = data_settings->index_granularity_bytes; - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( q_settings.merge_tree_max_rows_to_use_cache, q_settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( q_settings.merge_tree_min_rows_for_concurrent_read, q_settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, @@ -496,7 +447,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (pipe.numOutputPorts() > 1) { - auto transform = std::make_shared( pipe.getHeader(), pipe.numOutputPorts(), @@ -640,7 +590,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (adaptive_parts >= parts_with_range.size() / 2) index_granularity_bytes = data_settings->index_granularity_bytes; - const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( + const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( q_settings.merge_tree_max_rows_to_use_cache, q_settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, @@ -773,17 +723,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( std::min(used_num_streams, q_settings.max_final_threads), sort_description, data.merging_params, partition_key_columns, settings.max_block_size); - // auto final_step = std::make_unique( - // plan->getCurrentDataStream(), - // std::min(used_num_streams, settings.max_final_threads), - // sort_description, - // data.merging_params, - // partition_key_columns, - // max_block_size); - - // final_step->setStepDescription("Merge rows for FINAL"); - // plan->addStep(std::move(final_step)); - partition_pipes.emplace_back(std::move(pipe)); } @@ -793,7 +732,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( size_t num_streams_for_lonely_parts = used_num_streams * lonely_parts.size(); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( q_settings.merge_tree_min_rows_for_concurrent_read, q_settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, @@ -804,33 +743,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - // ReadFromMergeTree::Settings step_settings - // { - // .max_block_size = max_block_size, - // .preferred_block_size_bytes = settings.preferred_block_size_bytes, - // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - // .min_marks_for_concurrent_read = min_marks_for_concurrent_read, - // .use_uncompressed_cache = use_uncompressed_cache, - // .reader_settings = reader_settings, - // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - // }; - - // auto plan = std::make_unique(); - // auto step = std::make_unique( - // data, - // metadata_snapshot, - // query_id, - // column_names, - // std::move(lonely_parts), - // // std::move(index_stats), - // query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - // virt_columns, - // step_settings, - // num_streams_for_lonely_parts, - // ReadFromMergeTree::ReadType::Default); - - // plan->addStep(std::move(step)); - auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, num_streams_for_lonely_parts, min_marks_for_concurrent_read, use_uncompressed_cache); @@ -846,13 +758,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( return std::make_shared(header, sorting_expr); }); - // auto expression_step = std::make_unique( - // plan->getCurrentDataStream(), - // metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - // expression_step->setStepDescription("Calculate sorting key expression"); - // plan->addStep(std::move(expression_step)); - partition_pipes.emplace_back(std::move(pipe)); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 299abf66049..baf545f7557 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1235,12 +1235,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( return plan; } -namespace -{ /// Marks are placed whenever threshold on rows or bytes is met. /// So we have to return the number of marks on whatever estimate is higher - by rows or by bytes. -size_t roundRowsOrBytesToMarks( +size_t MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( size_t rows_setting, size_t bytes_setting, size_t rows_granularity, @@ -1254,608 +1252,35 @@ size_t roundRowsOrBytesToMarks( return std::max(res, (bytes_setting + bytes_granularity - 1) / bytes_granularity); } +/// Same as roundRowsOrBytesToMarks() but do not return more then max_marks +size_t MergeTreeDataSelectExecutor::minMarksForConcurrentRead( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity, + size_t max_marks) +{ + size_t marks = 1; + + if (rows_setting + rows_granularity <= rows_setting) /// overflow + marks = max_marks; + else if (rows_setting) + marks = (rows_setting + rows_granularity - 1) / rows_granularity; + + if (bytes_granularity == 0) + return marks; + else + { + /// Overflow + if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow + return max_marks; + if (bytes_setting) + return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); + else + return marks; + } } -// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( -// RangesInDataParts && parts, -// // ReadFromMergeTree::IndexStatPtr index_stats, -// size_t num_streams, -// const Names & column_names, -// const StorageMetadataPtr & metadata_snapshot, -// UInt64 max_block_size, -// bool use_uncompressed_cache, -// const SelectQueryInfo & query_info, -// const Names & virt_columns, -// const Settings & settings, -// const MergeTreeReaderSettings & reader_settings, -// const String & query_id) const -// { -// /// Count marks for each part. -// std::vector sum_marks_in_parts(parts.size()); -// size_t sum_marks = 0; -// size_t total_rows = 0; - -// const auto data_settings = data.getSettings(); -// size_t adaptive_parts = 0; -// for (size_t i = 0; i < parts.size(); ++i) -// { -// total_rows += parts[i].getRowsCount(); -// sum_marks_in_parts[i] = parts[i].getMarksCount(); -// sum_marks += sum_marks_in_parts[i]; - -// if (parts[i].data_part->index_granularity_info.is_adaptive) -// ++adaptive_parts; -// } - -// size_t index_granularity_bytes = 0; -// if (adaptive_parts > parts.size() / 2) -// index_granularity_bytes = data_settings->index_granularity_bytes; - -// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( -// settings.merge_tree_max_rows_to_use_cache, -// settings.merge_tree_max_bytes_to_use_cache, -// data_settings->index_granularity, -// index_granularity_bytes); - -// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( -// settings.merge_tree_min_rows_for_concurrent_read, -// settings.merge_tree_min_bytes_for_concurrent_read, -// data_settings->index_granularity, -// index_granularity_bytes, -// sum_marks); - -// if (sum_marks > max_marks_to_use_cache) -// use_uncompressed_cache = false; - -// if (0 == sum_marks) -// return {}; - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// if (num_streams > 1) -// { -// /// Reduce the number of num_streams if the data is small. -// if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) -// num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); -// } - -// auto plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams, -// ReadFromMergeTree::ReadType::Default); - -// plan->addStep(std::move(step)); -// return plan; -// } - -// static ActionsDAGPtr createProjection(const Block & header) -// { -// auto projection = std::make_shared(header.getNamesAndTypesList()); -// projection->removeUnusedActions(header.getNames()); -// projection->projectInput(); -// return projection; -// } - -// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( -// RangesInDataParts && parts, -// // ReadFromMergeTree::IndexStatPtr index_stats, -// size_t num_streams, -// const Names & column_names, -// const StorageMetadataPtr & metadata_snapshot, -// UInt64 max_block_size, -// bool use_uncompressed_cache, -// const SelectQueryInfo & query_info, -// const ActionsDAGPtr & sorting_key_prefix_expr, -// const Names & virt_columns, -// const Settings & settings, -// const MergeTreeReaderSettings & reader_settings, -// ActionsDAGPtr & out_projection, -// const String & query_id, -// const InputOrderInfoPtr & input_order_info) const -// { -// size_t sum_marks = 0; -// size_t adaptive_parts = 0; -// std::vector sum_marks_in_parts(parts.size()); -// const auto data_settings = data.getSettings(); - -// for (size_t i = 0; i < parts.size(); ++i) -// { -// sum_marks_in_parts[i] = parts[i].getMarksCount(); -// sum_marks += sum_marks_in_parts[i]; - -// if (parts[i].data_part->index_granularity_info.is_adaptive) -// ++adaptive_parts; -// } - -// size_t index_granularity_bytes = 0; -// if (adaptive_parts > parts.size() / 2) -// index_granularity_bytes = data_settings->index_granularity_bytes; - -// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( -// settings.merge_tree_max_rows_to_use_cache, -// settings.merge_tree_max_bytes_to_use_cache, -// data_settings->index_granularity, -// index_granularity_bytes); - -// const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( -// settings.merge_tree_min_rows_for_concurrent_read, -// settings.merge_tree_min_bytes_for_concurrent_read, -// data_settings->index_granularity, -// index_granularity_bytes, -// sum_marks); - -// if (sum_marks > max_marks_to_use_cache) -// use_uncompressed_cache = false; - -// Pipes res; - -// if (sum_marks == 0) -// return {}; - -// /// Let's split ranges to avoid reading much data. -// auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) -// { -// MarkRanges new_ranges; -// const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; -// size_t marks_in_range = 1; - -// if (direction == 1) -// { -// /// Split first few ranges to avoid reading much data. -// bool split = false; -// for (auto range : ranges) -// { -// while (!split && range.begin + marks_in_range < range.end) -// { -// new_ranges.emplace_back(range.begin, range.begin + marks_in_range); -// range.begin += marks_in_range; -// marks_in_range *= 2; - -// if (marks_in_range > max_marks_in_range) -// split = true; -// } -// new_ranges.emplace_back(range.begin, range.end); -// } -// } -// else -// { -// /// Split all ranges to avoid reading much data, because we have to -// /// store whole range in memory to reverse it. -// for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) -// { -// auto range = *it; -// while (range.begin + marks_in_range < range.end) -// { -// new_ranges.emplace_front(range.end - marks_in_range, range.end); -// range.end -= marks_in_range; -// marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); -// } -// new_ranges.emplace_front(range.begin, range.end); -// } -// } - -// return new_ranges; -// }; - -// const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; -// bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); - -// std::vector plans; - -// for (size_t i = 0; i < num_streams && !parts.empty(); ++i) -// { -// size_t need_marks = min_marks_per_stream; -// RangesInDataParts new_parts; - -// /// Loop over parts. -// /// We will iteratively take part or some subrange of a part from the back -// /// and assign a stream to read from it. -// while (need_marks > 0 && !parts.empty()) -// { -// RangesInDataPart part = parts.back(); -// parts.pop_back(); - -// size_t & marks_in_part = sum_marks_in_parts.back(); - -// /// We will not take too few rows from a part. -// if (marks_in_part >= min_marks_for_concurrent_read && -// need_marks < min_marks_for_concurrent_read) -// need_marks = min_marks_for_concurrent_read; - -// /// Do not leave too few rows in the part. -// if (marks_in_part > need_marks && -// marks_in_part - need_marks < min_marks_for_concurrent_read) -// need_marks = marks_in_part; - -// MarkRanges ranges_to_get_from_part; - -// /// We take the whole part if it is small enough. -// if (marks_in_part <= need_marks) -// { -// ranges_to_get_from_part = part.ranges; - -// need_marks -= marks_in_part; -// sum_marks_in_parts.pop_back(); -// } -// else -// { -// /// Loop through ranges in part. Take enough ranges to cover "need_marks". -// while (need_marks > 0) -// { -// if (part.ranges.empty()) -// throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); - -// MarkRange & range = part.ranges.front(); - -// const size_t marks_in_range = range.end - range.begin; -// const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); - -// ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); -// range.begin += marks_to_get_from_range; -// marks_in_part -= marks_to_get_from_range; -// need_marks -= marks_to_get_from_range; -// if (range.begin == range.end) -// part.ranges.pop_front(); -// } -// parts.emplace_back(part); -// } -// ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); -// new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); -// } - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// auto read_type = input_order_info->direction == 1 -// ? ReadFromMergeTree::ReadType::InOrder -// : ReadFromMergeTree::ReadType::InReverseOrder; - -// auto plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(new_parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams, -// read_type); - -// plan->addStep(std::move(step)); -// plans.emplace_back(std::move(plan)); -// } - -// if (need_preliminary_merge) -// { -// SortDescription sort_description; -// for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) -// sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], -// input_order_info->direction, 1); - -// for (auto & plan : plans) -// { -// /// Drop temporary columns, added by 'sorting_key_prefix_expr' -// out_projection = createProjection(plan->getCurrentDataStream().header); - -// auto expression_step = std::make_unique( -// plan->getCurrentDataStream(), -// sorting_key_prefix_expr); - -// expression_step->setStepDescription("Calculate sorting key prefix"); -// plan->addStep(std::move(expression_step)); - -// auto merging_sorted = std::make_unique( -// plan->getCurrentDataStream(), -// sort_description, -// max_block_size); - -// merging_sorted->setStepDescription("Merge sorting mark ranges"); -// plan->addStep(std::move(merging_sorted)); -// } -// } - -// if (plans.size() == 1) -// return std::move(plans.front()); - -// DataStreams input_streams; -// for (const auto & plan : plans) -// input_streams.emplace_back(plan->getCurrentDataStream()); - -// auto union_step = std::make_unique(std::move(input_streams)); - -// auto plan = std::make_unique(); -// plan->unitePlans(std::move(union_step), std::move(plans)); - -// return plan; -// } - - -// QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( -// RangesInDataParts && parts, -// size_t num_streams, -// const Names & column_names, -// const StorageMetadataPtr & metadata_snapshot, -// UInt64 max_block_size, -// bool use_uncompressed_cache, -// const SelectQueryInfo & query_info, -// const Names & virt_columns, -// const Settings & settings, -// const MergeTreeReaderSettings & reader_settings, -// ActionsDAGPtr & out_projection, -// const String & query_id) const -// { -// const auto data_settings = data.getSettings(); -// size_t sum_marks = 0; -// size_t adaptive_parts = 0; -// for (const auto & part : parts) -// { -// for (const auto & range : part.ranges) -// sum_marks += range.end - range.begin; - -// if (part.data_part->index_granularity_info.is_adaptive) -// ++adaptive_parts; -// } - -// size_t index_granularity_bytes = 0; -// if (adaptive_parts >= parts.size() / 2) -// index_granularity_bytes = data_settings->index_granularity_bytes; - -// const size_t max_marks_to_use_cache = roundRowsOrBytesToMarks( -// settings.merge_tree_max_rows_to_use_cache, -// settings.merge_tree_max_bytes_to_use_cache, -// data_settings->index_granularity, -// index_granularity_bytes); - -// if (sum_marks > max_marks_to_use_cache) -// use_uncompressed_cache = false; - -// if (num_streams > settings.max_final_threads) -// num_streams = settings.max_final_threads; - -// /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. -// /// We have all parts in parts vector, where parts with same partition are nearby. -// /// So we will store iterators pointed to the beginning of each partition range (and parts.end()), -// /// then we will create a pipe for each partition that will run selecting processor and merging processor -// /// for the parts with this partition. In the end we will unite all the pipes. -// std::vector parts_to_merge_ranges; -// auto it = parts.begin(); -// parts_to_merge_ranges.push_back(it); - -// if (settings.do_not_merge_across_partitions_select_final) -// { -// while (it != parts.end()) -// { -// it = std::find_if( -// it, parts.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); -// parts_to_merge_ranges.push_back(it); -// } -// /// We divide threads for each partition equally. But we will create at least the number of partitions threads. -// /// (So, the total number of threads could be more than initial num_streams. -// num_streams /= (parts_to_merge_ranges.size() - 1); -// } -// else -// { -// /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. -// parts_to_merge_ranges.push_back(parts.end()); -// } - -// std::vector partition_plans; - -// /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 -// /// we will store lonely parts with level > 0 to use parallel select on them. -// std::vector lonely_parts; -// size_t total_rows_in_lonely_parts = 0; -// size_t sum_marks_in_lonely_parts = 0; - -// for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) -// { -// QueryPlanPtr plan; - -// { -// RangesInDataParts new_parts; - -// /// 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 and if num_streams > 1 we -// /// can use parallel select on such parts. We save such parts in one vector and then use -// /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. -// if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && -// 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) -// { -// total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); -// sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); -// lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); -// continue; -// } -// else -// { -// for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) -// { -// new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); -// } -// } - -// if (new_parts.empty()) -// continue; - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(new_parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams, -// ReadFromMergeTree::ReadType::InOrder); - -// plan->addStep(std::move(step)); - -// /// Drop temporary columns, added by 'sorting_key_expr' -// if (!out_projection) -// out_projection = createProjection(plan->getCurrentDataStream().header); -// } - -// auto expression_step = std::make_unique( -// plan->getCurrentDataStream(), -// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - -// expression_step->setStepDescription("Calculate sorting key expression"); -// plan->addStep(std::move(expression_step)); - -// /// 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 -// if (settings.do_not_merge_across_partitions_select_final && -// 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_plans.emplace_back(std::move(plan)); -// continue; -// } - -// Names sort_columns = metadata_snapshot->getSortingKeyColumns(); -// SortDescription sort_description; -// size_t sort_columns_size = sort_columns.size(); -// sort_description.reserve(sort_columns_size); - -// Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - -// const auto & header = plan->getCurrentDataStream().header; -// for (size_t i = 0; i < sort_columns_size; ++i) -// sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); - -// auto final_step = std::make_unique( -// plan->getCurrentDataStream(), -// std::min(num_streams, settings.max_final_threads), -// sort_description, -// data.merging_params, -// partition_key_columns, -// max_block_size); - -// final_step->setStepDescription("Merge rows for FINAL"); -// plan->addStep(std::move(final_step)); - -// partition_plans.emplace_back(std::move(plan)); -// } - -// 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 = minMarksForConcurrentRead( -// settings.merge_tree_min_rows_for_concurrent_read, -// settings.merge_tree_min_bytes_for_concurrent_read, -// data_settings->index_granularity, -// index_granularity_bytes, -// sum_marks_in_lonely_parts); - -// /// Reduce the number of num_streams_for_lonely_parts if the data is small. -// if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) -// num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - -// ReadFromMergeTree::Settings step_settings -// { -// .max_block_size = max_block_size, -// .preferred_block_size_bytes = settings.preferred_block_size_bytes, -// .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, -// .min_marks_for_concurrent_read = min_marks_for_concurrent_read, -// .use_uncompressed_cache = use_uncompressed_cache, -// .reader_settings = reader_settings, -// .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), -// }; - -// auto plan = std::make_unique(); -// auto step = std::make_unique( -// data, -// metadata_snapshot, -// query_id, -// column_names, -// std::move(lonely_parts), -// // std::move(index_stats), -// query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, -// virt_columns, -// step_settings, -// num_streams_for_lonely_parts, -// ReadFromMergeTree::ReadType::Default); - -// plan->addStep(std::move(step)); - -// /// Drop temporary columns, added by 'sorting_key_expr' -// if (!out_projection) -// out_projection = createProjection(plan->getCurrentDataStream().header); - -// auto expression_step = std::make_unique( -// plan->getCurrentDataStream(), -// metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - -// expression_step->setStepDescription("Calculate sorting key expression"); -// plan->addStep(std::move(expression_step)); - -// partition_plans.emplace_back(std::move(plan)); -// } - -// if (partition_plans.empty()) -// return {}; - -// if (partition_plans.size() == 1) -// return std::move(partition_plans.front()); - -// auto result_header = partition_plans.front()->getCurrentDataStream().header; -// DataStreams input_streams; -// for (const auto & partition_plan : partition_plans) -// input_streams.push_back(partition_plan->getCurrentDataStream()); - -// auto union_step = std::make_unique(std::move(input_streams), result_header); -// union_step->setStepDescription("Unite sources after FINAL"); -// QueryPlanPtr plan = std::make_unique(); -// plan->unitePlans(std::move(union_step), std::move(partition_plans)); -// return plan; -// } /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 077584039a1..00a347790a7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -22,16 +22,6 @@ struct MergeTreeDataSelectSamplingData ActionsDAGPtr filter_expression; }; -// struct MergeTreeDataSelectCache -// { -// RangesInDataParts parts_with_ranges; -// MergeTreeDataSelectSamplingData sampling; -// std::unique_ptr index_stats; -// size_t sum_marks = 0; -// size_t sum_ranges = 0; -// bool use_cache = false; -// }; - using PartitionIdToMaxBlock = std::unordered_map; /** Executes SELECT queries on data from the merge tree. @@ -78,53 +68,8 @@ public: private: const MergeTreeData & data; - Poco::Logger * log; - // QueryPlanPtr spreadMarkRangesAmongStreams( - // RangesInDataParts && parts, - // size_t num_streams, - // const Names & column_names, - // const StorageMetadataPtr & metadata_snapshot, - // UInt64 max_block_size, - // bool use_uncompressed_cache, - // const SelectQueryInfo & query_info, - // const Names & virt_columns, - // const Settings & settings, - // const MergeTreeReaderSettings & reader_settings, - // const String & query_id) const; - - // /// out_projection - save projection only with columns, requested to read - // QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( - // RangesInDataParts && parts, - // size_t num_streams, - // const Names & column_names, - // const StorageMetadataPtr & metadata_snapshot, - // UInt64 max_block_size, - // bool use_uncompressed_cache, - // const SelectQueryInfo & query_info, - // const ActionsDAGPtr & sorting_key_prefix_expr, - // const Names & virt_columns, - // const Settings & settings, - // const MergeTreeReaderSettings & reader_settings, - // ActionsDAGPtr & out_projection, - // const String & query_id, - // const InputOrderInfoPtr & input_order_info) const; - - // QueryPlanPtr spreadMarkRangesAmongStreamsFinal( - // RangesInDataParts && parts, - // size_t num_streams, - // const Names & column_names, - // const StorageMetadataPtr & metadata_snapshot, - // UInt64 max_block_size, - // bool use_uncompressed_cache, - // const SelectQueryInfo & query_info, - // const Names & virt_columns, - // const Settings & settings, - // const MergeTreeReaderSettings & reader_settings, - // ActionsDAGPtr & out_projection, - // const String & query_id) const; - /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. static size_t getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, @@ -186,6 +131,19 @@ private: Poco::Logger * log); public: + static size_t roundRowsOrBytesToMarks( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity); + + static size_t minMarksForConcurrentRead( + size_t rows_setting, + size_t bytes_setting, + size_t rows_granularity, + size_t bytes_granularity, + size_t max_marks); + static std::optional> filterPartsByVirtualColumns( const MergeTreeData & data, MergeTreeData::DataPartsVector & parts, @@ -227,7 +185,10 @@ public: NamesAndTypesList available_real_columns, ContextPtr context); - static String checkLimits(const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context); + static String checkLimits( + const MergeTreeData & data, + const RangesInDataParts & parts_with_ranges, + const ContextPtr & context); }; } From 91e8c2c75dbc598b23880c593fa09171e3dacaaa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 12:52:33 +0300 Subject: [PATCH 248/652] Remove MergingFinal step. --- src/Processors/QueryPlan/MergingFinal.cpp | 170 ------------------ src/Processors/QueryPlan/MergingFinal.h | 36 ---- src/Processors/ya.make | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - 4 files changed, 208 deletions(-) delete mode 100644 src/Processors/QueryPlan/MergingFinal.cpp delete mode 100644 src/Processors/QueryPlan/MergingFinal.h diff --git a/src/Processors/QueryPlan/MergingFinal.cpp b/src/Processors/QueryPlan/MergingFinal.cpp deleted file mode 100644 index c564a28d377..00000000000 --- a/src/Processors/QueryPlan/MergingFinal.cpp +++ /dev/null @@ -1,170 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -static ITransformingStep::Traits getTraits() -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = true, - } - }; -} - -MergingFinal::MergingFinal( - const DataStream & input_stream, - size_t num_output_streams_, - SortDescription sort_description_, - MergeTreeData::MergingParams params_, - Names partition_key_columns_, - size_t max_block_size_) - : ITransformingStep(input_stream, input_stream.header, getTraits()) - , num_output_streams(num_output_streams_) - , sort_description(std::move(sort_description_)) - , merging_params(std::move(params_)) - , partition_key_columns(std::move(partition_key_columns_)) - , max_block_size(max_block_size_) -{ - /// TODO: check input_stream is partially sorted (each port) by the same description. -// output_stream->sort_description = sort_description; -// output_stream->sort_mode = DataStream::SortMode::Stream; -} - -void MergingFinal::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) -{ - const auto & header = pipeline.getHeader(); - size_t num_outputs = pipeline.getNumStreams(); - - auto get_merging_processor = [&]() -> MergingTransformPtr - { - switch (merging_params.mode) - { - case MergeTreeData::MergingParams::Ordinary: - { - return std::make_shared(header, num_outputs, - sort_description, max_block_size); - } - - case MergeTreeData::MergingParams::Collapsing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, true, max_block_size); - - case MergeTreeData::MergingParams::Summing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); - - case MergeTreeData::MergingParams::Aggregating: - return std::make_shared(header, num_outputs, - sort_description, max_block_size); - - case MergeTreeData::MergingParams::Replacing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.version_column, max_block_size); - - case MergeTreeData::MergingParams::VersionedCollapsing: - return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, max_block_size); - - case MergeTreeData::MergingParams::Graphite: - throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); - } - - __builtin_unreachable(); - }; - - if (num_output_streams <= 1 || sort_description.empty()) - { - pipeline.addTransform(get_merging_processor()); - return; - } - - ColumnNumbers key_columns; - key_columns.reserve(sort_description.size()); - - for (auto & desc : sort_description) - { - if (!desc.column_name.empty()) - key_columns.push_back(header.getPositionByName(desc.column_name)); - else - key_columns.emplace_back(desc.column_number); - } - - pipeline.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, num_output_streams, key_columns); - }); - - pipeline.transform([&](OutputPortRawPtrs ports) - { - Processors transforms; - std::vector output_ports; - transforms.reserve(ports.size() + num_output_streams); - output_ports.reserve(ports.size()); - - for (auto & port : ports) - { - auto copier = std::make_shared(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; - }); -} - -void MergingFinal::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(sort_description, input_streams.front().header, settings.out); - settings.out << '\n'; -} - -void MergingFinal::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); -} - -} diff --git a/src/Processors/QueryPlan/MergingFinal.h b/src/Processors/QueryPlan/MergingFinal.h deleted file mode 100644 index ed0394a62f4..00000000000 --- a/src/Processors/QueryPlan/MergingFinal.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -/// Merge streams of data into single sorted stream. -class MergingFinal : public ITransformingStep -{ -public: - explicit MergingFinal( - const DataStream & input_stream, - size_t num_output_streams_, - SortDescription sort_description_, - MergeTreeData::MergingParams params_, - Names partition_key_columns_, - size_t max_block_size_); - - String getName() const override { return "MergingFinal"; } - - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - -private: - size_t num_output_streams; - SortDescription sort_description; - MergeTreeData::MergingParams merging_params; - Names partition_key_columns; - size_t max_block_size; -}; - -} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 5ab9c79511f..86a40685d1f 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -112,7 +112,6 @@ SRCS( QueryPlan/LimitStep.cpp QueryPlan/MergeSortingStep.cpp QueryPlan/MergingAggregatedStep.cpp - QueryPlan/MergingFinal.cpp QueryPlan/MergingSortedStep.cpp QueryPlan/OffsetStep.cpp QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index baf545f7557..c719c7f996c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include From c34d8ae7e71ad8bd42acf0f86551a84965d8d274 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 21 May 2021 14:31:12 +0300 Subject: [PATCH 249/652] Fix logical error AggregateFunctionFactory returned nullptr --- .../AggregateFunctionDeltaSumTimestamp.cpp | 4 ++-- .../01762_deltasumtimestamp_datetime64.reference | 1 + .../0_stateless/01762_deltasumtimestamp_datetime64.sh | 8 ++++++++ 3 files changed, 11 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference create mode 100755 tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp index 10195ae33fd..719729048ef 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp +++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp @@ -29,11 +29,11 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp( throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDateOrDateTime(arguments[0])) + if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDate(arguments[0]) && !isDateTime(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDateOrDateTime(arguments[1])) + if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDate(arguments[1]) && !isDateTime(arguments[1])) throw Exception("Illegal type " + arguments[1]->getName() + " of argument for aggregate function " + name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh new file mode 100755 index 00000000000..9181cc94564 --- /dev/null +++ b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="SELECT deltaSumTimestamp(1, now64());" 2>&1 | grep -q "Code: 43.*Illegal type DateTime64" && echo 'OK' || echo 'FAIL'; + From 42d57138f20622e1aaf4a0693356de5b53a44722 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 28 May 2021 15:34:49 +0300 Subject: [PATCH 250/652] replace all isDateOrDateTime invocations to explicit calls to isDate and isDateTime and isDateTime64 --- src/AggregateFunctions/AggregateFunctionResample.cpp | 2 +- .../AggregateFunctionSegmentLengthSum.cpp | 2 +- src/DataTypes/IDataType.h | 5 +---- src/DataTypes/Native.h | 2 +- src/DataTypes/getMostSubtype.cpp | 2 +- src/Functions/FunctionBinaryArithmetic.h | 8 ++++---- src/Functions/FunctionCustomWeekToSomething.h | 6 +++--- src/Functions/FunctionDateOrDateTimeAddInterval.h | 2 +- src/Functions/FunctionDateOrDateTimeToSomething.h | 4 ++-- src/Functions/FunctionsCoding.h | 4 +++- src/Functions/FunctionsComparison.h | 6 +++--- src/Functions/FunctionsConversion.h | 4 ++-- src/Functions/array/arrayIntersect.cpp | 6 ++++-- src/Functions/dateDiff.cpp | 4 ++-- src/Functions/date_trunc.cpp | 2 +- src/Functions/formatDateTime.cpp | 4 ++-- src/Functions/reinterpretAs.cpp | 4 +++- src/Functions/toStartOfInterval.cpp | 2 +- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/Interpreters/convertFieldToType.cpp | 4 ++-- .../Formats/RowInputFormatWithDiagnosticInfo.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp | 2 +- 22 files changed, 41 insertions(+), 38 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionResample.cpp b/src/AggregateFunctions/AggregateFunctionResample.cpp index fe818acd8d7..ba2d4a58b4b 100644 --- a/src/AggregateFunctions/AggregateFunctionResample.cpp +++ b/src/AggregateFunctions/AggregateFunctionResample.cpp @@ -53,7 +53,7 @@ public: { WhichDataType which{arguments.back()}; - if (which.isNativeUInt() || which.isDateOrDateTime()) + if (which.isNativeUInt() || which.isDate() || which.isDateTime() || which.isDateTime64()) { UInt64 begin = params[params.size() - 3].safeGet(); UInt64 end = params[params.size() - 2].safeGet(); diff --git a/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.cpp b/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.cpp index fad60eacc04..6817d1f0666 100644 --- a/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.cpp +++ b/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.cpp @@ -38,7 +38,7 @@ namespace for (const auto & arg : args) { - if (!isNativeNumber(arg) && !isDateOrDateTime(arg)) + if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg) && !isDateTime64(arg)) throw Exception( "Illegal type " + arg->getName() + " of argument of aggregate function " + name + ", must be Number, Date, DateTime or DateTime64", diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 85526cd98de..04657b3c8c7 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -324,7 +324,6 @@ struct WhichDataType constexpr bool isDate() const { return idx == TypeIndex::Date; } constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; } constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; } - constexpr bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); } constexpr bool isString() const { return idx == TypeIndex::String; } constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; } @@ -350,8 +349,6 @@ struct WhichDataType template inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); } template -inline bool isDateOrDateTime(const T & data_type) { return WhichDataType(data_type).isDateOrDateTime(); } -template inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); } template inline bool isDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTime64(); } @@ -415,7 +412,7 @@ template inline bool isColumnedAsNumber(const T & data_type) { WhichDataType which(data_type); - return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDateTime() || which.isUUID(); + return which.isInt() || which.isUInt() || which.isFloat() || which.isDate() || which.isDateTime() || which.isDateTime64() || which.isUUID(); } template diff --git a/src/DataTypes/Native.h b/src/DataTypes/Native.h index a1d07b54c8e..0a13ce83590 100644 --- a/src/DataTypes/Native.h +++ b/src/DataTypes/Native.h @@ -171,7 +171,7 @@ static inline llvm::Constant * getColumnNativeValue(llvm::IRBuilderBase & builde { return llvm::ConstantFP::get(type, assert_cast &>(column).getElement(index)); } - else if (column_data_type.isNativeUInt() || column_data_type.isDateOrDateTime()) + else if (column_data_type.isNativeUInt() || column_data_type.isDate() || column_data_type.isDateTime() || column_data_type.isDateTime64()) { return llvm::ConstantInt::get(type, column.getUInt(index)); } diff --git a/src/DataTypes/getMostSubtype.cpp b/src/DataTypes/getMostSubtype.cpp index 4a73f0e2de2..fcf966780eb 100644 --- a/src/DataTypes/getMostSubtype.cpp +++ b/src/DataTypes/getMostSubtype.cpp @@ -241,7 +241,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth for (const auto & type : types) { - if (isDateOrDateTime(type)) + if (isDate(type) || isDateTime(type) || isDateTime64(type)) have_date_or_datetime = true; else all_date_or_datetime = false; diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 927b870891f..c8cd8536f3a 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -599,8 +599,8 @@ class FunctionBinaryArithmetic : public IFunction static FunctionOverloadResolverPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_is_date_or_datetime = isDateOrDateTime(type0); - bool second_is_date_or_datetime = isDateOrDateTime(type1); + bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); /// Exactly one argument must be Date or DateTime if (first_is_date_or_datetime == second_is_date_or_datetime) @@ -775,7 +775,7 @@ class FunctionBinaryArithmetic : public IFunction ColumnsWithTypeAndName new_arguments = arguments; /// Interval argument must be second. - if (WhichDataType(arguments[1].type).isDateOrDateTime()) + if (isDate(arguments[1].type) || isDateTime(arguments[1].type) || isDateTime64(arguments[1].type)) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument type to its representation @@ -990,7 +990,7 @@ public: new_arguments[i].type = arguments[i]; /// Interval argument must be second. - if (WhichDataType(new_arguments[1].type).isDateOrDateTime()) + if (isDate(new_arguments[1].type) || isDateTime(new_arguments[1].type) || isDateTime64(new_arguments[1].type)) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument to its representation diff --git a/src/Functions/FunctionCustomWeekToSomething.h b/src/Functions/FunctionCustomWeekToSomething.h index 84ecd4d7a5a..2bcf3a3e5ab 100644 --- a/src/Functions/FunctionCustomWeekToSomething.h +++ b/src/Functions/FunctionCustomWeekToSomething.h @@ -35,7 +35,7 @@ public: { if (arguments.size() == 1) { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", @@ -43,7 +43,7 @@ public: } else if (arguments.size() == 2) { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", @@ -59,7 +59,7 @@ public: } else if (arguments.size() == 3) { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 14ab9f00452..c87cf863154 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -375,7 +375,7 @@ public: if (arguments.size() == 2) { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index 8b8f1b483c8..43b1c8e4aae 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -38,7 +38,7 @@ public: { if (arguments.size() == 1) { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", @@ -46,7 +46,7 @@ public: } else if (arguments.size() == 2) { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", diff --git a/src/Functions/FunctionsCoding.h b/src/Functions/FunctionsCoding.h index 01602cf1175..b45eca565d7 100644 --- a/src/Functions/FunctionsCoding.h +++ b/src/Functions/FunctionsCoding.h @@ -970,7 +970,9 @@ public: WhichDataType which(arguments[0]); if (!which.isStringOrFixedString() && - !which.isDateOrDateTime() && + !which.isDate() && + !which.isDateTime() && + !which.isDateTime64() && !which.isUInt() && !which.isFloat() && !which.isDecimal()) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 9ffb0cd0fc3..ce0f580e6f1 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1086,7 +1086,7 @@ public: if (!((both_represented_by_number && !has_date) /// Do not allow to compare date and number. || (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion. /// You can compare the date, datetime, or datatime64 and an enumeration with a constant string. - || (left.isDateOrDateTime() && right.isDateOrDateTime() && left.idx == right.idx) /// only date vs date, or datetime vs datetime + || ((left.isDate() || left.isDateTime() || left.isDateTime64()) && (right.isDate() || right.isDateTime() || right.isDateTime64()) && left.idx == right.idx) /// only date vs date, or datetime vs datetime || (left.isUUID() && right.isUUID()) || (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()) @@ -1176,8 +1176,8 @@ public: const bool left_is_string = isStringOrFixedString(which_left); const bool right_is_string = isStringOrFixedString(which_right); - bool date_and_datetime = (which_left.idx != which_right.idx) && - which_left.isDateOrDateTime() && which_right.isDateOrDateTime(); + bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDateTime() || which_left.isDateTime64()) + && (which_right.isDate() || which_right.isDateTime() || which_right.isDateTime64()); ColumnPtr res; if (left_is_num && right_is_num && !date_and_datetime) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 589bd4cc16e..a2202b9d8a4 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1890,7 +1890,7 @@ struct ToDateMonotonicity static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) { auto which = WhichDataType(type); - if (which.isDateOrDateTime() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16()) + if (which.isDate() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16()) return {true, true, true}; else if ( (which.isUInt() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) @@ -2408,7 +2408,7 @@ private: UInt32 scale = to_type->getScale(); WhichDataType which(type_index); - bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDateOrDateTime() + bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDate() || which.isDateTime() || which.isDateTime64() || which.isStringOrFixedString(); if (!ok) { diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 93c1ee06403..5a6dac0a8a1 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -217,7 +217,9 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns( auto type_not_nullable_nested = removeNullable(type_nested); const bool is_numeric_or_string = isNativeNumber(type_not_nullable_nested) - || isDateOrDateTime(type_not_nullable_nested) + || isDate(type_not_nullable_nested) + || isDateTime(type_not_nullable_nested) + || isDateTime64(type_not_nullable_nested) || isStringOrFixedString(type_not_nullable_nested); DataTypePtr nullable_return_type; @@ -334,7 +336,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays( const auto & nested_init_type = typeid_cast(removeNullable(initial_columns[i].type).get())->getNestedType(); const auto & nested_cast_type = typeid_cast(removeNullable(columns[i].type).get())->getNestedType(); - if (isInteger(nested_init_type) || isDateOrDateTime(nested_init_type)) + if (isInteger(nested_init_type) || isDate(nested_init_type) || isDateTime(nested_init_type) || isDateTime64(nested_init_type)) { /// Compare original and casted columns. It seem to be the easiest way. auto overflow_mask = callFunctionNotEquals( diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index cbad9dc1ce4..d2d1d7c6e0d 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -68,11 +68,11 @@ public: throw Exception("First argument for function " + getName() + " (unit) must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!isDateOrDateTime(arguments[1])) + if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[1])) throw Exception("Second argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!isDateOrDateTime(arguments[2])) + if (!isDate(arguments[2]) && !isDateTime(arguments[2]) && !isDateTime64(arguments[2])) throw Exception("Third argument for function " + getName() + " must be Date or DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 7952eda4f2b..1f2bcabf1f9 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -62,7 +62,7 @@ public: bool second_argument_is_date = false; auto check_second_argument = [&] { - if (!isDateOrDateTime(arguments[1].type)) + if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) throw Exception( "Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName() + ". Should be a date or a date with time", diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index d16985b5b34..12c2932c54f 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -309,7 +309,7 @@ public: "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + " when arguments size is 1. Should be integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (arguments.size() > 1 && !(isInteger(arguments[0].type) || WhichDataType(arguments[0].type).isDateOrDateTime())) + if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + " when arguments size is 2 or 3. Should be a integer or a date with time", @@ -322,7 +322,7 @@ public: "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2 or 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!WhichDataType(arguments[0].type).isDateOrDateTime()) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Should be a date or a date with time", diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 94c9eec5868..460651ac5c1 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -259,7 +259,9 @@ private: { return type.isUInt() || type.isInt() || - type.isDateOrDateTime() || + type.isDate() || + type.isDateTime() || + type.isDateTime64() || type.isFloat() || type.isUUID() || type.isDecimal(); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index ee592fbb1e3..0169d50e1cf 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -197,7 +197,7 @@ public: bool first_argument_is_date = false; auto check_first_argument = [&] { - if (!isDateOrDateTime(arguments[0].type)) + if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index d50aa323ba3..cd2774de94a 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -281,7 +281,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) if (which.isNullable()) throw Exception("LOGICAL ERROR: MySQL primary key must be not null, it is a bug.", ErrorCodes::LOGICAL_ERROR); - if (which.isDateOrDateTime()) + if (which.isDate() || which.isDateTime() || which.isDateTime64()) { /// In any case, date or datetime is always the best partitioning key return makeASTFunction("toYYYYMM", std::make_shared(primary_key.name)); diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 0b124634fec..08bfb1e175b 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -189,7 +189,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return dynamic_cast(type).castToValue(src); } - if (which_type.isDateOrDateTime() && !which_type.isDateTime64() && src.getType() == Field::Types::UInt64) + if ((which_type.isDate() || which_type.isDateTime()) && src.getType() == Field::Types::UInt64) { /// We don't need any conversion UInt64 is under type of Date and DateTime return src; @@ -202,7 +202,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } if (which_type.isDateTime64() - && (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDateOrDateTime())) + && (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDateTime() || which_from_type.isDateTime64())) { const auto scale = static_cast(type).getScale(); const auto decimal_value = DecimalUtils::decimalFromComponents(src.reinterpret(), 0, scale); diff --git a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp index 86f8bb36bb2..4dddc9fff9b 100644 --- a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp +++ b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp @@ -117,7 +117,7 @@ bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(co if (curr_position < prev_position) throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - if (isNativeNumber(type) || isDateOrDateTime(type)) + if (isNativeNumber(type) || isDate(type) || isDateTime(type) || isDateTime64(type)) { /// An empty string instead of a value. if (curr_position == prev_position) diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index c37d710ec8f..79581fb3ab4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -85,7 +85,7 @@ static void assertIndexColumnsType(const Block & header) WhichDataType which(actual_type); if (!which.isUInt() && !which.isInt() && !which.isString() && !which.isFixedString() && !which.isFloat() && - !which.isDateOrDateTime() && !which.isEnum() && !which.isUUID()) + !which.isDate() && !which.isDateTime() && !which.isDateTime64() && !which.isEnum() && !which.isUUID()) throw Exception("Unexpected type " + type->getName() + " of bloom filter index.", ErrorCodes::ILLEGAL_COLUMN); } From d396f802c61195b1cdef0b4ee4642da27d603f4c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 28 May 2021 15:39:36 +0300 Subject: [PATCH 251/652] Fix typo --- src/Functions/toStartOfInterval.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 0169d50e1cf..6d5f880f744 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -197,7 +197,7 @@ public: bool first_argument_is_date = false; auto check_first_argument = [&] { - if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", From 295a302bc8e3e22b2b5a7dff92bdbb6e7e113557 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 17:34:02 +0300 Subject: [PATCH 252/652] Remove settings from ReadFromMergeTree. --- .../QueryPlan/ReadFromMergeTree.cpp | 225 ++++++++++-------- src/Processors/QueryPlan/ReadFromMergeTree.h | 63 +++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 44 +--- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- 4 files changed, 161 insertions(+), 173 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7e4a8ba5a55..4515db11067 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -49,38 +49,63 @@ struct ReadFromMergeTree::AnalysisResult ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default; }; +static MergeTreeReaderSettings getMergeTreeReaderSettings(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + return { + .min_bytes_to_use_direct_io = settings.min_bytes_to_use_direct_io, + .min_bytes_to_use_mmap_io = settings.min_bytes_to_use_mmap_io, + .mmap_cache = context->getMMappedFileCache(), + .max_read_buffer_size = settings.max_read_buffer_size, + .save_marks_in_cache = true, + .checksum_on_read = settings.checksum_on_read, + }; +} + +static const PrewhereInfoPtr & getPrewhereInfo(const SelectQueryInfo & query_info) +{ + return query_info.projection ? query_info.projection->prewhere_info + : query_info.prewhere_info; +} + ReadFromMergeTree::ReadFromMergeTree( - const SelectQueryInfo & query_info_, - std::shared_ptr max_block_numbers_to_read_, - ContextPtr context_, + MergeTreeData::DataPartsVector parts_, + Names real_column_names_, + Names virt_column_names_, const MergeTreeData & data_, + const SelectQueryInfo & query_info_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, - Names real_column_names_, - MergeTreeData::DataPartsVector parts_, - PrewhereInfoPtr prewhere_info_, - Names virt_column_names_, - Settings settings_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_, + bool sample_factor_column_queried_, + std::shared_ptr max_block_numbers_to_read_, Poco::Logger * log_) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( metadata_snapshot_->getSampleBlockForColumns(real_column_names_, data_.getVirtuals(), data_.getStorageID()), - prewhere_info_, + getPrewhereInfo(query_info_), data_.getPartitionValueType(), virt_column_names_)}) - , query_info(std::move(query_info_)) - , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) - , context(std::move(context_)) + , reader_settings(getMergeTreeReaderSettings(context_)) + , prepared_parts(std::move(parts_)) + , real_column_names(std::move(real_column_names_)) + , virt_column_names(std::move(virt_column_names_)) , data(data_) + , query_info(query_info_) + , prewhere_info(getPrewhereInfo(query_info)) , metadata_snapshot(std::move(metadata_snapshot_)) , metadata_snapshot_base(std::move(metadata_snapshot_base_)) - , real_column_names(std::move(real_column_names_)) - , prepared_parts(std::move(parts_)) - , prewhere_info(std::move(prewhere_info_)) - , virt_column_names(std::move(virt_column_names_)) - , settings(std::move(settings_)) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , requested_num_streams(num_streams_) + , preferred_block_size_bytes(context->getSettingsRef().preferred_block_size_bytes) + , preferred_max_column_in_block_size_bytes(context->getSettingsRef().preferred_max_column_in_block_size_bytes) + , sample_factor_column_queried(sample_factor_column_queried_) + , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(log_) { - if (settings.sample_factor_column_queried) + if (sample_factor_column_queried) { /// Only _sample_factor virtual column is added by ReadFromMergeTree /// Other virtual columns are added by MergeTreeBaseSelectProcessor. @@ -90,8 +115,11 @@ ReadFromMergeTree::ReadFromMergeTree( } Pipe ReadFromMergeTree::readFromPool( - RangesInDataParts parts_with_range, Names required_columns, - size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) + RangesInDataParts parts_with_range, + Names required_columns, + size_t max_streams, + size_t min_marks_for_concurrent_read, + bool use_uncompressed_cache) { Pipes pipes; size_t sum_marks = 0; @@ -103,8 +131,11 @@ Pipe ReadFromMergeTree::readFromPool( total_rows += part.getRowsCount(); } + const auto & settings = context->getSettingsRef(); + MergeTreeReadPool::BackoffSettings backoff_settings(settings); + auto pool = std::make_shared( - used_max_streams, + max_streams, sum_marks, min_marks_for_concurrent_read, std::move(parts_with_range), @@ -113,20 +144,20 @@ Pipe ReadFromMergeTree::readFromPool( prewhere_info, true, required_columns, - settings.backoff_settings, + backoff_settings, settings.preferred_block_size_bytes, false); auto * logger = &Poco::Logger::get(data.getLogName() + " (SelectExecutor)"); - LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, used_max_streams); + LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, max_streams); - for (size_t i = 0; i < used_max_streams; ++i) + for (size_t i = 0; i < max_streams; ++i) { auto source = std::make_shared( - i, pool, min_marks_for_concurrent_read, settings.max_block_size, + i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, metadata_snapshot, use_uncompressed_cache, - prewhere_info, settings.reader_settings, virt_column_names); + prewhere_info, reader_settings, virt_column_names); if (i == 0) { @@ -141,15 +172,22 @@ Pipe ReadFromMergeTree::readFromPool( } template -ProcessorPtr ReadFromMergeTree::createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache) +ProcessorPtr ReadFromMergeTree::createSource( + const RangesInDataPart & part, + const Names & required_columns, + bool use_uncompressed_cache) { return std::make_shared( - data, metadata_snapshot, part.data_part, settings.max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, - prewhere_info, true, settings.reader_settings, virt_column_names, part.part_index_in_query); + data, metadata_snapshot, part.data_part, max_block_size, preferred_block_size_bytes, + preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, + prewhere_info, true, reader_settings, virt_column_names, part.part_index_in_query); } -Pipe ReadFromMergeTree::readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache) +Pipe ReadFromMergeTree::readInOrder( + RangesInDataParts parts_with_range, + Names required_columns, + ReadType read_type, + bool use_uncompressed_cache) { Pipes pipes; for (const auto & part : parts_with_range) @@ -176,10 +214,11 @@ Pipe ReadFromMergeTree::readInOrder(RangesInDataParts parts_with_range, Names re Pipe ReadFromMergeTree::read( RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, - size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) + size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { - if (read_type == ReadType::Default && used_max_streams > 1) - return readFromPool(parts_with_range, required_columns, used_max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + if (read_type == ReadType::Default && max_streams > 1) + return readFromPool(parts_with_range, required_columns, max_streams, + min_marks_for_concurrent_read, use_uncompressed_cache); auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache); @@ -195,7 +234,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, const Names & column_names) { - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); /// Count marks for each part. std::vector sum_marks_in_parts(parts_with_ranges.size()); @@ -219,35 +258,35 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - q_settings.merge_tree_max_rows_to_use_cache, - q_settings.merge_tree_max_bytes_to_use_cache, + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - q_settings.merge_tree_min_rows_for_concurrent_read, - q_settings.merge_tree_min_bytes_for_concurrent_read, + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, index_granularity_bytes, sum_marks); - bool use_uncompressed_cache = q_settings.use_uncompressed_cache; + bool use_uncompressed_cache = settings.use_uncompressed_cache; if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; if (0 == sum_marks) return {}; - size_t used_num_streams = settings.num_streams; - if (used_num_streams > 1) + size_t num_streams = requested_num_streams; + if (num_streams > 1) { /// Reduce the number of num_streams if the data is small. - if (sum_marks < used_num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < used_num_streams) - used_num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); + if (sum_marks < num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) + num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); } return read(std::move(parts_with_ranges), column_names, ReadType::Default, - used_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); } static ActionsDAGPtr createProjection(const Block & header) @@ -265,7 +304,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( ActionsDAGPtr & out_projection, const InputOrderInfoPtr & input_order_info) { - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); size_t sum_marks = 0; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts_with_ranges.size()); @@ -285,14 +324,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - q_settings.merge_tree_max_rows_to_use_cache, - q_settings.merge_tree_max_bytes_to_use_cache, + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - q_settings.merge_tree_min_rows_for_concurrent_read, - q_settings.merge_tree_min_bytes_for_concurrent_read, + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, index_granularity_bytes, sum_marks); @@ -307,7 +346,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return {}; /// Let's split ranges to avoid reading much data. - auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = settings.max_block_size] + auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = max_block_size] (const auto & ranges, int direction) { MarkRanges new_ranges; @@ -352,12 +391,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return new_ranges; }; - const size_t min_marks_per_stream = (sum_marks - 1) / settings.num_streams + 1; - bool need_preliminary_merge = (parts_with_ranges.size() > q_settings.read_in_order_two_level_merge_threshold); + const size_t min_marks_per_stream = (sum_marks - 1) / requested_num_streams + 1; + bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold); Pipes pipes; - for (size_t i = 0; i < settings.num_streams && !parts_with_ranges.empty(); ++i) + for (size_t i = 0; i < requested_num_streams && !parts_with_ranges.empty(); ++i) { size_t need_marks = min_marks_per_stream; RangesInDataParts new_parts; @@ -398,7 +437,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( while (need_marks > 0) { if (part.ranges.empty()) - throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); + throw Exception("Unexpected end of ranges while spreading marks among streams", + ErrorCodes::LOGICAL_ERROR); MarkRange & range = part.ranges.front(); @@ -423,7 +463,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( : ReadFromMergeTree::ReadType::InReverseOrder; pipes.emplace_back(read(std::move(new_parts), column_names, read_type, - settings.num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + requested_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); } if (need_preliminary_merge) @@ -451,7 +491,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( pipe.getHeader(), pipe.numOutputPorts(), sort_description, - settings.max_block_size); + max_block_size); pipe.addTransform(std::move(transform)); } @@ -573,7 +613,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( const Names & column_names, ActionsDAGPtr & out_projection) { - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); size_t sum_marks = 0; size_t adaptive_parts = 0; @@ -591,8 +631,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( index_granularity_bytes = data_settings->index_granularity_bytes; const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - q_settings.merge_tree_max_rows_to_use_cache, - q_settings.merge_tree_max_bytes_to_use_cache, + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, data_settings->index_granularity, index_granularity_bytes); @@ -600,9 +640,9 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - size_t used_num_streams = settings.num_streams; - if (used_num_streams > q_settings.max_final_threads) - used_num_streams = q_settings.max_final_threads; + size_t num_streams = requested_num_streams; + if (num_streams > settings.max_final_threads) + num_streams = settings.max_final_threads; /// If setting do_not_merge_across_partitions_select_final is true than we won't merge parts from different partitions. /// We have all parts in parts vector, where parts with same partition are nearby. @@ -613,7 +653,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( auto it = parts_with_range.begin(); parts_to_merge_ranges.push_back(it); - if (q_settings.do_not_merge_across_partitions_select_final) + if (settings.do_not_merge_across_partitions_select_final) { while (it != parts_with_range.end()) { @@ -623,7 +663,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( } /// We divide threads for each partition equally. But we will create at least the number of partitions threads. /// (So, the total number of threads could be more than initial num_streams. - used_num_streams /= (parts_to_merge_ranges.size() - 1); + num_streams /= (parts_to_merge_ranges.size() - 1); } else { @@ -650,7 +690,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// with level > 0 then we won't postprocess this part and if num_streams > 1 we /// can use parallel select on such parts. We save such parts in one vector and then use /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. - if (used_num_streams > 1 && q_settings.do_not_merge_across_partitions_select_final && + if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && 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) { @@ -670,19 +710,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (new_parts.empty()) continue; - // ReadFromMergeTree::Settings step_settings - // { - // .max_block_size = max_block_size, - // .preferred_block_size_bytes = settings.preferred_block_size_bytes, - // .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - // .min_marks_for_concurrent_read = 0, /// this setting is not used for reading in order - // .use_uncompressed_cache = use_uncompressed_cache, - // .reader_settings = reader_settings, - // .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - // }; - pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, - used_num_streams, 0, use_uncompressed_cache); + num_streams, 0, use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) @@ -699,7 +728,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// 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 - if (q_settings.do_not_merge_across_partitions_select_final && + if (settings.do_not_merge_across_partitions_select_final && 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) { @@ -720,8 +749,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( addMergingFinal( pipe, - std::min(used_num_streams, q_settings.max_final_threads), - sort_description, data.merging_params, partition_key_columns, settings.max_block_size); + std::min(num_streams, settings.max_final_threads), + sort_description, data.merging_params, partition_key_columns, max_block_size); partition_pipes.emplace_back(std::move(pipe)); } @@ -730,11 +759,11 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( { RangesInDataParts new_parts; - size_t num_streams_for_lonely_parts = used_num_streams * lonely_parts.size(); + size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - q_settings.merge_tree_min_rows_for_concurrent_read, - q_settings.merge_tree_min_bytes_for_concurrent_read, + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, index_granularity_bytes, sum_marks_in_lonely_parts); @@ -767,6 +796,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts) const { AnalysisResult result; + const auto & settings = context->getSettingsRef(); size_t total_parts = parts.size(); @@ -807,7 +837,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre result.sampling = MergeTreeDataSelectExecutor::getSampling( select, parts, metadata_snapshot, key_condition, - data, log, settings.sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); if (result.sampling.read_nothing) return result; @@ -824,9 +854,9 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre query_info, context, key_condition, - settings.reader_settings, + reader_settings, log, - settings.num_streams, + requested_num_streams, result.index_stats, true); @@ -865,8 +895,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre ? query_info.input_order_info : (query_info.projection ? query_info.projection->input_order_info : nullptr); - const auto & q_settings = context->getSettingsRef(); - if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info) result.read_type = (input_order_info->direction > 0) ? ReadType::InOrder : ReadType::InReverseOrder; @@ -906,7 +935,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build Pipe pipe; - const auto & q_settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); if (select.final()) { @@ -927,7 +956,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build column_names_to_read, result_projection); } - else if ((q_settings.optimize_read_in_order || q_settings.optimize_aggregation_in_order) && input_order_info) + else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info) { size_t prefix_size = input_order_info->order_key_prefix_descr.size(); auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone(); @@ -983,7 +1012,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build }; /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. - if (settings.sample_factor_column_queried) + if (sample_factor_column_queried) { ColumnWithTypeAndName column; column.name = "_sample_factor"; @@ -994,18 +1023,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build append_actions(std::move(adding_column)); } - // TODO There seems to be no place initializing remove_columns_actions - // if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) - // { - // auto remove_columns_action = std::make_shared( - // query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - - // pipe.addSimpleTransform([&](const Block & header) - // { - // return std::make_shared(header, remove_columns_action); - // }); - // } - /// Extra columns may be returned (for example, if sampling is used). /// Convert pipe to step header structure. if (!isCompatibleHeader(cur_header, getOutputStream().header)) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 0c3814c507f..07d45a71e0a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -40,22 +40,6 @@ public: using IndexStats = std::vector; - /// Part of settings which are needed for reading. - struct Settings - { - UInt64 max_block_size; - size_t num_streams; - size_t preferred_block_size_bytes; - size_t preferred_max_column_in_block_size_bytes; - //size_t min_marks_for_concurrent_read; - bool use_uncompressed_cache; - bool force_primary_key; - bool sample_factor_column_queried; - - MergeTreeReaderSettings reader_settings; - MergeTreeReadPool::BackoffSettings backoff_settings; - }; - enum class ReadType { /// By default, read will use MergeTreeReadPool and return pipe with num_streams outputs. @@ -72,17 +56,18 @@ public: }; ReadFromMergeTree( - const SelectQueryInfo & query_info_, - std::shared_ptr max_block_numbers_to_read_, - ContextPtr context_, + MergeTreeData::DataPartsVector parts_, + Names real_column_names_, + Names virt_column_names_, const MergeTreeData & data_, + const SelectQueryInfo & query_info_, StorageMetadataPtr metadata_snapshot_, StorageMetadataPtr metadata_snapshot_base_, - Names real_column_names_, - MergeTreeData::DataPartsVector parts_, - PrewhereInfoPtr prewhere_info_, - Names virt_column_names_, - Settings settings_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_, + bool sample_factor_column_queried_, + std::shared_ptr max_block_numbers_to_read_, Poco::Logger * log_ ); @@ -97,23 +82,33 @@ public: void describeIndexes(JSONBuilder::JSONMap & map) const override; private: - SelectQueryInfo query_info; - std::shared_ptr max_block_numbers_to_read; - ContextPtr context; + const MergeTreeReaderSettings reader_settings; + + MergeTreeData::DataPartsVector prepared_parts; + Names real_column_names; + Names virt_column_names; + const MergeTreeData & data; + SelectQueryInfo query_info; + PrewhereInfoPtr prewhere_info; + StorageMetadataPtr metadata_snapshot; StorageMetadataPtr metadata_snapshot_base; - Names real_column_names; - MergeTreeData::DataPartsVector prepared_parts; - PrewhereInfoPtr prewhere_info; - Names virt_column_names; - Settings settings; + ContextPtr context; + + const size_t max_block_size; + const size_t requested_num_streams; + const size_t preferred_block_size_bytes; + const size_t preferred_max_column_in_block_size_bytes; + const bool sample_factor_column_queried; + + std::shared_ptr max_block_numbers_to_read; Poco::Logger * log; - Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); - Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t used_max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache); template diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c719c7f996c..c12759bac97 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -381,7 +381,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const ASTSelectQuery & select, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, @@ -1189,43 +1189,19 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - const auto & settings = context->getSettingsRef(); - - MergeTreeReaderSettings reader_settings = - { - .min_bytes_to_use_direct_io = settings.min_bytes_to_use_direct_io, - .min_bytes_to_use_mmap_io = settings.min_bytes_to_use_mmap_io, - .mmap_cache = context->getMMappedFileCache(), - .max_read_buffer_size = settings.max_read_buffer_size, - .save_marks_in_cache = true, - .checksum_on_read = settings.checksum_on_read, - }; - - ReadFromMergeTree::Settings step_settings - { - .max_block_size = max_block_size, - .num_streams = num_streams, - .preferred_block_size_bytes = settings.preferred_block_size_bytes, - .preferred_max_column_in_block_size_bytes = settings.preferred_max_column_in_block_size_bytes, - .use_uncompressed_cache = settings.use_uncompressed_cache, - .force_primary_key = settings.force_primary_key, - .sample_factor_column_queried = sample_factor_column_queried, - .reader_settings = reader_settings, - .backoff_settings = MergeTreeReadPool::BackoffSettings(settings), - }; - auto read_from_merge_tree = std::make_unique( - query_info, - max_block_numbers_to_read, - context, + parts, + real_column_names, + virt_column_names, data, + query_info, metadata_snapshot, metadata_snapshot_base, - real_column_names, - parts, - query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info, - virt_column_names, - step_settings, + context, + max_block_size, + num_streams, + sample_factor_column_queried, + max_block_numbers_to_read, log ); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 00a347790a7..b43fb785573 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -176,7 +176,7 @@ public: static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, From cb2277381f1ee3b1ab091ac72d4eaed6bbd47c0e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 28 May 2021 18:45:56 +0300 Subject: [PATCH 253/652] try remove --- .../configs/enable_dictionaries.xml | 4 ---- .../configs/enable_dictionaries.xml | 5 ----- .../test_dictionaries_complex_key_cache_string/test.py | 6 ++---- .../configs/enable_dictionaries.xml | 4 ---- tests/integration/test_dictionaries_dependency_xml/test.py | 3 +-- .../configs/enable_dictionaries.xml | 4 ---- tests/integration/test_dictionaries_mysql/test.py | 7 +++---- .../configs/enable_dictionaries.xml | 4 ---- tests/integration/test_dictionaries_null_value/test.py | 3 +-- .../configs/enable_dictionaries.xml | 4 ---- tests/integration/test_dictionaries_select_all/test.py | 2 +- .../configs/enable_dictionaries.xml | 4 ---- .../test_dictionaries_update_and_reload/test.py | 3 +-- .../configs/enable_dictionaries.xml | 4 ---- .../test_default_reading.py | 3 +-- .../test_default_string.py | 3 +-- .../test_dict_get.py | 3 +-- .../test_dict_get_or_default.py | 3 +-- .../configs/enable_dictionaries.xml | 4 ---- tests/integration/test_dictionary_custom_settings/test.py | 3 +-- .../test_library_bridge/configs/enable_dict.xml | 4 ---- tests/integration/test_library_bridge/test.py | 3 +-- .../test_odbc_interaction/configs/enable_dictionaries.xml | 4 ---- tests/integration/test_odbc_interaction/test.py | 5 ++--- 24 files changed, 17 insertions(+), 75 deletions(-) delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml delete mode 100644 tests/integration/test_library_bridge/configs/enable_dict.xml delete mode 100644 tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/enable_dictionaries.xml deleted file mode 100644 index 8a3d6704670..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/*.xml - diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml deleted file mode 100644 index 46d148ad9b9..00000000000 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - /etc/clickhouse-server/config.d/complex_key_cache_string.xml - /etc/clickhouse-server/config.d/ssd_complex_key_cache_string.xml - diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index a01e60af47d..1046fff5e4d 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -10,11 +10,9 @@ def cluster(request): cluster = ClickHouseCluster(__file__) try: if request.param == "memory": - node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/complex_key_cache_string.xml']) + node = cluster.add_instance('node', dictionaries=['configs/dictionaries/complex_key_cache_string.xml']) if request.param == "ssd": - node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/ssd_complex_key_cache_string.xml']) + node = cluster.add_instance('node', dictionaries=['configs/dictionaries/ssd_complex_key_cache_string.xml']) cluster.start() node.query( "create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") diff --git a/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml deleted file mode 100644 index 89a4c99ef7a..00000000000 --- a/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/dep_*.xml - diff --git a/tests/integration/test_dictionaries_dependency_xml/test.py b/tests/integration/test_dictionaries_dependency_xml/test.py index b8ebcc6cc4b..cfd7d58d574 100644 --- a/tests/integration/test_dictionaries_dependency_xml/test.py +++ b/tests/integration/test_dictionaries_dependency_xml/test.py @@ -2,12 +2,11 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry -ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml'] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG + DICTIONARY_FILES, ) +instance = cluster.add_instance('instance', dictionaries=DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml deleted file mode 100644 index 76ed6af89ba..00000000000 --- a/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/mysql_dict*.xml - diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 16e432c6425..dc877933078 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -3,11 +3,10 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster -CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', - 'configs/remote_servers.xml'] -CONFIG_FILES += ['configs/enable_dictionaries.xml', 'configs/log_conf.xml'] +DICTS = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml'] +CONFIG_FILES = ['configs/log_conf.xml', 'configs/remote_servers.xml'] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql=True) +instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql=True, dictionaries=DICTS) create_table_mysql_template = """ CREATE TABLE IF NOT EXISTS `test`.`{}` ( diff --git a/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml deleted file mode 100644 index 8a3d6704670..00000000000 --- a/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/*.xml - diff --git a/tests/integration/test_dictionaries_null_value/test.py b/tests/integration/test_dictionaries_null_value/test.py index 3c3ef59902f..96ca76f594e 100644 --- a/tests/integration/test_dictionaries_null_value/test.py +++ b/tests/integration/test_dictionaries_null_value/test.py @@ -1,11 +1,10 @@ import pytest from helpers.cluster import ClickHouseCluster -ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache.xml'] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG + DICTIONARY_FILES) +instance = cluster.add_instance('instance', dictionaries=DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml deleted file mode 100644 index fa26ed7ec3d..00000000000 --- a/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/dictionary*.xml - diff --git a/tests/integration/test_dictionaries_select_all/test.py b/tests/integration/test_dictionaries_select_all/test.py index 5331f51f4c7..b1bf2e98b25 100644 --- a/tests/integration/test_dictionaries_select_all/test.py +++ b/tests/integration/test_dictionaries_select_all/test.py @@ -22,7 +22,7 @@ def setup_module(module): dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure) cluster = ClickHouseCluster(__file__) - instance = cluster.add_instance('instance', main_configs=dictionary_files + ['configs/enable_dictionaries.xml']) + instance = cluster.add_instance('instance', dictionaries=dictionary_files) test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv')) diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml deleted file mode 100644 index 8a3d6704670..00000000000 --- a/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/*.xml - diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 533a29dc245..e301b943b99 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -7,12 +7,11 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml'] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG + DICTIONARY_FILES) +instance = cluster.add_instance('instance', dictionaries=DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml b/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml deleted file mode 100644 index 8a3d6704670..00000000000 --- a/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/*.xml - diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 0c801ce3f12..29fc1178826 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -10,8 +10,7 @@ from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', dictionaries=['configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index caabdf12c66..699f5f81996 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -13,8 +13,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/cache_ints_dictionary.xml', +main_node = cluster.add_instance('main_node', dictionaries=['configs/dictionaries/cache_ints_dictionary.xml', 'configs/dictionaries/cache_strings_default_settings.xml']) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 488e01fe7f0..cf2234c0601 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -10,8 +10,7 @@ from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', dictionaries=['configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index e794ffa5a37..7da470e6535 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -10,8 +10,7 @@ from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', dictionaries=['configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml b/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml deleted file mode 100644 index 8a3d6704670..00000000000 --- a/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/*.xml - diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index aa6a16afb51..f6c487f073d 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -3,7 +3,6 @@ import os import pytest from helpers.cluster import ClickHouseCluster -ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = [ 'configs/dictionaries/FileSourceConfig.xml', 'configs/dictionaries/ExecutableSourceConfig.xml', @@ -13,7 +12,7 @@ DICTIONARY_FILES = [ ] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('node', main_configs=ENABLE_DICT_CONFIG + DICTIONARY_FILES) +instance = cluster.add_instance('node', dictionaties=DICTIONARY_FILES) def prepare(): diff --git a/tests/integration/test_library_bridge/configs/enable_dict.xml b/tests/integration/test_library_bridge/configs/enable_dict.xml deleted file mode 100644 index 264f1f667b1..00000000000 --- a/tests/integration/test_library_bridge/configs/enable_dict.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/dict*.xml - diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index 422a7ee954f..552e4033b8e 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -8,10 +8,9 @@ from helpers.cluster import ClickHouseCluster, run_and_check cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', + dictionaries=['configs/dictionaries/dict1.xml'], main_configs=[ - 'configs/enable_dict.xml', 'configs/config.d/config.xml', - 'configs/dictionaries/dict1.xml', 'configs/log_conf.xml']) @pytest.fixture(scope="module") diff --git a/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml b/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml deleted file mode 100644 index 93780125e8e..00000000000 --- a/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml +++ /dev/null @@ -1,4 +0,0 @@ - - - /etc/clickhouse-server/config.d/*dictionary.xml - diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 25668737885..967ca6fd4a9 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -10,9 +10,8 @@ from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, - main_configs=['configs/openssl.xml', 'configs/odbc_logging.xml', - 'configs/enable_dictionaries.xml', - 'configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', + main_configs=['configs/openssl.xml', 'configs/odbc_logging.xml'], + dictionaries=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) From 58fbc544cc5eee5c8d914d89895d483ca463ce15 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 May 2021 20:16:09 +0300 Subject: [PATCH 254/652] Add more comments. --- .../QueryPlan/ReadFromMergeTree.cpp | 18 +++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 5 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 33 ++++++----- .../MergeTree/MergeTreeDataSelectExecutor.h | 57 ++++++++++++------- 4 files changed, 66 insertions(+), 47 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4515db11067..0672d7ed040 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -43,7 +44,6 @@ struct ReadFromMergeTree::AnalysisResult { RangesInDataParts parts_with_ranges; MergeTreeDataSelectSamplingData sampling; - String query_id; IndexStats index_stats; Names column_names_to_read; ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default; @@ -830,14 +830,14 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); const auto & select = query_info.query->as(); - auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; MergeTreeDataSelectExecutor::filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, result.index_stats); + parts, part_values, metadata_snapshot_base, data, query_info, context, + max_block_numbers_to_read.get(), log, result.index_stats); result.sampling = MergeTreeDataSelectExecutor::getSampling( - select, parts, metadata_snapshot, key_condition, - data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + select, metadata_snapshot->getColumns().getAllPhysical(), parts, key_condition, + data, metadata_snapshot, context, sample_factor_column_queried, log); if (result.sampling.read_nothing) return result; @@ -885,8 +885,6 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre sum_marks, sum_ranges); - result.query_id = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); - ProfileEvents::increment(ProfileEvents::SelectedParts, result.parts_with_ranges.size()); ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); @@ -905,6 +903,8 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) { auto result = selectRangesToRead(prepared_parts); + auto query_id_holder = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); + if (result.parts_with_ranges.empty()) { pipeline.init(Pipe(std::make_shared(getOutputStream().header))); @@ -1048,8 +1048,8 @@ void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const Build processors.emplace_back(processor); // Attach QueryIdHolder if needed - if (!result.query_id.empty()) - pipe.addQueryIdHolder(std::make_shared(result.query_id, data)); + if (query_id_holder) + pipe.addQueryIdHolder(std::move(query_id_holder)); pipeline.init(std::move(pipe)); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 07d45a71e0a..6e1efffdb02 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -1,15 +1,14 @@ #pragma once #include -#include #include -#include -//#include namespace DB { using PartitionIdToMaxBlock = std::unordered_map; +class Pipe; + /// This step is created to read from MergeTree* table. /// For now, it takes a list of parts and creates source from it. class ReadFromMergeTree final : public ISourceStep diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c12759bac97..caf69b8ecfb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -381,14 +381,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const ASTSelectQuery & select, + NamesAndTypesList available_real_columns, const MergeTreeData::DataPartsVector & parts, - const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, - Poco::Logger * log, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, bool sample_factor_column_queried, - NamesAndTypesList available_real_columns, - ContextPtr context) + Poco::Logger * log) { const Settings & settings = context->getSettingsRef(); /// Sampling. @@ -643,7 +643,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( std::optional> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( const MergeTreeData & data, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const ASTPtr & query, ContextPtr context) { @@ -666,13 +666,12 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, const ContextPtr & context, - const ContextPtr & query_context, - MergeTreeData::DataPartsVector & parts, - const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats) @@ -709,6 +708,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( } } + auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; PartFilterCounters part_filter_counters; if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( @@ -766,7 +766,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, const ContextPtr & context, - KeyCondition & key_condition, + const KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, @@ -992,7 +992,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd return parts_with_ranges; } -String MergeTreeDataSelectExecutor::checkLimits( +std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context) @@ -1032,7 +1032,10 @@ String MergeTreeDataSelectExecutor::checkLimits( } } - return query_id; + if (!query_id.empty()) + return std::make_shared(query_id, data); + + return nullptr; } static void selectColumnNames( @@ -1135,15 +1138,15 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( } const auto & select = query_info.query->as(); - auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; ReadFromMergeTree::IndexStats index_stats; filterPartsByPartition( - metadata_snapshot_base, data, query_info, context, query_context, parts, part_values, max_block_numbers_to_read.get(), log, index_stats); + parts, part_values, metadata_snapshot_base, data, query_info, + context, max_block_numbers_to_read.get(), log, index_stats); auto sampling = MergeTreeDataSelectExecutor::getSampling( - select, parts, metadata_snapshot, key_condition, - data, log, sample_factor_column_queried, metadata_snapshot->getColumns().getAllPhysical(), context); + select, metadata_snapshot->getColumns().getAllPhysical(), parts, key_condition, + data, metadata_snapshot, context, sample_factor_column_queried, log); if (sampling.read_nothing) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index b43fb785573..bd2a79f0aee 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -45,16 +45,7 @@ public: QueryProcessingStage::Enum processed_stage, std::shared_ptr max_block_numbers_to_read = nullptr) const; - size_t estimateNumMarksToRead( - MergeTreeData::DataPartsVector parts, - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot_base, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - ContextPtr context, - unsigned num_streams, - std::shared_ptr max_block_numbers_to_read = nullptr) const; - + /// The same as read, but with specified set of parts. QueryPlanPtr readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, @@ -66,6 +57,19 @@ public: unsigned num_streams, std::shared_ptr max_block_numbers_to_read = nullptr) const; + /// Get an estimation for the number of marks we are going to read. + /// Reads nothing. Secondary indexes are not used. + /// This method is used to select best projection for table. + size_t estimateNumMarksToRead( + MergeTreeData::DataPartsVector parts, + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot_base, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + ContextPtr context, + unsigned num_streams, + std::shared_ptr max_block_numbers_to_read = nullptr) const; + private: const MergeTreeData & data; Poco::Logger * log; @@ -131,12 +135,15 @@ private: Poco::Logger * log); public: + /// For given number rows and bytes, get the number of marks to read. + /// It is a minimal number of marks which contain so many rows and bytes. static size_t roundRowsOrBytesToMarks( size_t rows_setting, size_t bytes_setting, size_t rows_granularity, size_t bytes_granularity); + /// The same as roundRowsOrBytesToMarks, but return no more than max_marks. static size_t minMarksForConcurrentRead( size_t rows_setting, size_t bytes_setting, @@ -144,48 +151,58 @@ public: size_t bytes_granularity, size_t max_marks); + /// If possible, filter using expression on virtual columns. + /// Example: SELECT count() FROM table WHERE _part = 'part_name' + /// If expression found, return a set with allowed part names (std::nullopt otherwise). static std::optional> filterPartsByVirtualColumns( const MergeTreeData & data, - MergeTreeData::DataPartsVector & parts, + const MergeTreeData::DataPartsVector & parts, const ASTPtr & query, ContextPtr context); + /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, + const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const SelectQueryInfo & query_info, const ContextPtr & context, - const ContextPtr & query_context, - MergeTreeData::DataPartsVector & parts, - const std::optional> & part_values, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats); + /// Filter parts using primary key and secondary indexes. + /// For every part, select mark ranges to read. static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, const ContextPtr & context, - KeyCondition & key_condition, + const KeyCondition & key_condition, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, ReadFromMergeTree::IndexStats & index_stats, bool use_skip_indexes); + /// Create expression for sampling. + /// Also, calculate _sample_factor if needed. + /// Also, update key condition with selected sampling range. static MergeTreeDataSelectSamplingData getSampling( const ASTSelectQuery & select, + NamesAndTypesList available_real_columns, const MergeTreeData::DataPartsVector & parts, - const StorageMetadataPtr & metadata_snapshot, KeyCondition & key_condition, const MergeTreeData & data, - Poco::Logger * log, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, bool sample_factor_column_queried, - NamesAndTypesList available_real_columns, - ContextPtr context); + Poco::Logger * log); - static String checkLimits( + /// Check query limits: max_partitions_to_read, max_concurrent_queries. + /// Also, return QueryIdHolder. If not null, we should keep it until query finishes. + static std::shared_ptr checkLimits( const MergeTreeData & data, const RangesInDataParts & parts_with_ranges, const ContextPtr & context); From 40b1dc181219b7a608d098ccee65dba3d142c4b2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 28 May 2021 20:49:47 +0300 Subject: [PATCH 255/652] support expressions in window frame -- draft --- src/Interpreters/ExpressionAnalyzer.cpp | 40 ++++++++++++++++++------ src/Parsers/ASTWindowDefinition.cpp | 18 ++++++++--- src/Parsers/ASTWindowDefinition.h | 6 +++- src/Parsers/ExpressionElementParsers.cpp | 16 ++++------ 4 files changed, 55 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b2589583d68..fa86a9b9377 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -14,17 +14,18 @@ #include #include -#include -#include #include +#include +#include +#include +#include +#include #include +#include +#include +#include #include #include -#include -#include -#include -#include -#include #include @@ -474,7 +475,8 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) return !aggregates().empty(); } -void makeWindowDescriptionFromAST(const WindowDescriptions & existing_descriptions, +void makeWindowDescriptionFromAST(const Context & context, + const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast) { const auto & definition = ast->as(); @@ -580,6 +582,22 @@ void makeWindowDescriptionFromAST(const WindowDescriptions & existing_descriptio } desc.frame = definition.frame; + + if (definition.frame.end_type == WindowFrame::BoundaryType::Offset) + { + auto [value, _] = evaluateConstantExpression( + definition.frame_end_offset, + const_pointer_cast(context.shared_from_this())); + desc.frame.end_offset = value; + } + + if (definition.frame.begin_type == WindowFrame::BoundaryType::Offset) + { + auto [value, _] = evaluateConstantExpression( + definition.frame_begin_offset, + const_pointer_cast(context.shared_from_this())); + desc.frame.begin_offset = value; + } } void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) @@ -605,7 +623,8 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) const auto & elem = ptr->as(); WindowDescription desc; desc.window_name = elem.name; - makeWindowDescriptionFromAST(window_descriptions, desc, elem.definition.get()); + makeWindowDescriptionFromAST(*getContext(), window_descriptions, + desc, elem.definition.get()); auto [it, inserted] = window_descriptions.insert( {desc.window_name, desc}); @@ -690,7 +709,8 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) const ASTWindowDefinition &>(); WindowDescription desc; desc.window_name = definition.getDefaultWindowName(); - makeWindowDescriptionFromAST(window_descriptions, desc, &definition); + makeWindowDescriptionFromAST(*getContext(), window_descriptions, + desc, &definition); auto [it, inserted] = window_descriptions.insert( {desc.window_name, desc}); diff --git a/src/Parsers/ASTWindowDefinition.cpp b/src/Parsers/ASTWindowDefinition.cpp index 376df5bc80c..636b826a1e1 100644 --- a/src/Parsers/ASTWindowDefinition.cpp +++ b/src/Parsers/ASTWindowDefinition.cpp @@ -28,6 +28,18 @@ ASTPtr ASTWindowDefinition::clone() const result->frame = frame; + if (frame_begin_offset) + { + result->frame_begin_offset = frame_begin_offset->clone(); + result->children.push_back(result->frame_begin_offset); + } + + if (frame_end_offset) + { + result->frame_end_offset = frame_end_offset->clone(); + result->children.push_back(result->frame_end_offset); + } + return result; } @@ -93,8 +105,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings, } else { - settings.ostr << applyVisitor(FieldVisitorToString(), - frame.begin_offset); + frame_begin_offset->formatImpl(settings, state, format_frame); settings.ostr << " " << (!frame.begin_preceding ? "FOLLOWING" : "PRECEDING"); } @@ -109,8 +120,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings, } else { - settings.ostr << applyVisitor(FieldVisitorToString(), - frame.end_offset); + frame_end_offset->formatImpl(settings, state, format_frame); settings.ostr << " " << (!frame.end_preceding ? "FOLLOWING" : "PRECEDING"); } diff --git a/src/Parsers/ASTWindowDefinition.h b/src/Parsers/ASTWindowDefinition.h index 7547f1527f2..4edf574cabf 100644 --- a/src/Parsers/ASTWindowDefinition.h +++ b/src/Parsers/ASTWindowDefinition.h @@ -16,8 +16,12 @@ struct ASTWindowDefinition : public IAST ASTPtr order_by; + // Be careful: offsets can contain constant expressions such as INTERVAL 1 DAY, + // that are evaluated later by ExpressionAnalyzer. The WindowFrame struct + // can be incomplete after parsing. WindowFrame frame; - + ASTPtr frame_begin_offset; + ASTPtr frame_end_offset; ASTPtr clone() const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 75a341a543d..a8a037f9d0e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -569,16 +569,14 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p } else { - ParserLiteral parser_literal; - ASTPtr ast_literal; + ParserExpression parser_expression; if (keyword_unbounded.ignore(pos, expected)) { node->frame.begin_type = WindowFrame::BoundaryType::Unbounded; } - else if (parser_literal.parse(pos, ast_literal, expected)) + else if (parser_expression.parse(pos, node->frame_begin_offset, expected)) { - const Field & value = ast_literal->as().value; - node->frame.begin_offset = value; + // We will evaluate the expression for offset expression later. node->frame.begin_type = WindowFrame::BoundaryType::Offset; } else @@ -618,16 +616,14 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p } else { - ParserLiteral parser_literal; - ASTPtr ast_literal; + ParserExpression parser_expression; if (keyword_unbounded.ignore(pos, expected)) { node->frame.end_type = WindowFrame::BoundaryType::Unbounded; } - else if (parser_literal.parse(pos, ast_literal, expected)) + else if (parser_expression.parse(pos, node->frame_end_offset, expected)) { - const Field & value = ast_literal->as().value; - node->frame.end_offset = value; + // We will evaluate the expression for offset expression later. node->frame.end_type = WindowFrame::BoundaryType::Offset; } else From 80587a5aa53e025b2ae71af09e88fca009d81e02 Mon Sep 17 00:00:00 2001 From: yang <642444906@qq.com> Date: Sat, 29 May 2021 09:14:20 +0800 Subject: [PATCH 256/652] Update "Tail_Number" column type in create query I found the CSV file "Tail_Number" value is like this: N517AE, so it should be the type String --- docs/en/getting-started/example-datasets/ontime.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md index f18acc6fd50..dfa1d7ff964 100644 --- a/docs/en/getting-started/example-datasets/ontime.md +++ b/docs/en/getting-started/example-datasets/ontime.md @@ -32,7 +32,7 @@ CREATE TABLE `ontime` `Reporting_Airline` String, `DOT_ID_Reporting_Airline` Int32, `IATA_CODE_Reporting_Airline` String, - `Tail_Number` Int32, + `Tail_Number` String, `Flight_Number_Reporting_Airline` String, `OriginAirportID` Int32, `OriginAirportSeqID` Int32, From 73c2ca7da5a23c5385698d5034c402884a403ea9 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Sat, 29 May 2021 16:12:18 +0300 Subject: [PATCH 257/652] fixes --- docker/test/fuzzer/run-fuzzer.sh | 7 ++- programs/client/QueryFuzzer.cpp | 72 ++++++++++++++++++------ programs/client/QueryFuzzer.h | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 23 ++++---- src/Parsers/ASTWindowDefinition.cpp | 23 +++++--- src/Parsers/ASTWindowDefinition.h | 10 ++-- src/Parsers/ExpressionElementParsers.cpp | 34 +++++------ 7 files changed, 111 insertions(+), 62 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 626bedb453c..300ac6ab7a0 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -119,7 +119,12 @@ continue # SC2012: Use find instead of ls to better handle non-alphanumeric filenames. They are all alphanumeric. # SC2046: Quote this to prevent word splitting. Actually I need word splitting. # shellcheck disable=SC2012,SC2046 - clickhouse-client --query-fuzzer-runs=1000 --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) $NEW_TESTS_OPT \ + clickhouse-client \ + --receive_timeout=10 \ + --receive_data_timeout_ms=10000 \ + --query-fuzzer-runs=1000 \ + --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \ + $NEW_TESTS_OPT \ > >(tail -n 100000 > fuzzer.log) \ 2>&1 \ || fuzzer_exit_code=$? diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index 721e5acb991..aa12468cde0 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -325,14 +325,14 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) // the generic recursion into IAST.children. } -void QueryFuzzer::fuzzWindowFrame(WindowFrame & frame) +void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def) { switch (fuzz_rand() % 40) { case 0: { const auto r = fuzz_rand() % 3; - frame.type = r == 0 ? WindowFrame::FrameType::Rows + def.frame_type = r == 0 ? WindowFrame::FrameType::Rows : r == 1 ? WindowFrame::FrameType::Range : WindowFrame::FrameType::Groups; break; @@ -340,44 +340,80 @@ void QueryFuzzer::fuzzWindowFrame(WindowFrame & frame) case 1: { const auto r = fuzz_rand() % 3; - frame.begin_type = r == 0 ? WindowFrame::BoundaryType::Unbounded + def.frame_begin_type = r == 0 ? WindowFrame::BoundaryType::Unbounded : r == 1 ? WindowFrame::BoundaryType::Current : WindowFrame::BoundaryType::Offset; + + if (def.frame_begin_type == WindowFrame::BoundaryType::Offset) + { + // The offsets are fuzzed normally through 'children'. + def.frame_begin_offset + = std::make_shared(getRandomField(0)); + def.children.push_back(def.frame_begin_offset); + } + else + { + // Don't keep the offset if it is not used, because it will + // acquire random mutations that will surely make it invalid. + const auto old_size = def.children.size(); + def.children.erase( + std::remove(def.children.begin(), def.children.end(), + def.frame_begin_offset), + def.children.end()); + assert(def.children.size() == old_size - 1 + || def.frame_begin_offset == nullptr); + def.frame_begin_offset = nullptr; + } break; } case 2: { const auto r = fuzz_rand() % 3; - frame.end_type = r == 0 ? WindowFrame::BoundaryType::Unbounded + def.frame_end_type = r == 0 ? WindowFrame::BoundaryType::Unbounded : r == 1 ? WindowFrame::BoundaryType::Current : WindowFrame::BoundaryType::Offset; - break; - } - case 3: - { - frame.begin_offset = getRandomField(0).get(); - break; - } - case 4: - { - frame.end_offset = getRandomField(0).get(); + + if (def.frame_end_type == WindowFrame::BoundaryType::Offset) + { + def.frame_end_offset + = std::make_shared(getRandomField(0)); + def.children.push_back(def.frame_end_offset); + } + else + { + def.children.erase( + std::remove(def.children.begin(), def.children.end(), + def.frame_end_offset), + def.children.end()); + def.frame_end_offset = nullptr; + } break; } case 5: { - frame.begin_preceding = fuzz_rand() % 2; + def.frame_begin_preceding = fuzz_rand() % 2; break; } case 6: { - frame.end_preceding = fuzz_rand() % 2; + def.frame_end_preceding = fuzz_rand() % 2; break; } default: break; } - frame.is_default = (frame == WindowFrame{}); + if (def.frame_type == WindowFrame::FrameType::Range + && def.frame_begin_type == WindowFrame::BoundaryType::Unbounded + && def.frame_begin_preceding == true + && def.frame_end_type == WindowFrame::BoundaryType::Current) + { + def.frame_is_default = true; + } + else + { + def.frame_is_default = false; + } } void QueryFuzzer::fuzz(ASTs & asts) @@ -464,7 +500,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast) auto & def = fn->window_definition->as(); fuzzColumnLikeExpressionList(def.partition_by.get()); fuzzOrderByList(def.order_by.get()); - fuzzWindowFrame(def.frame); + fuzzWindowFrame(def); } fuzz(fn->children); diff --git a/programs/client/QueryFuzzer.h b/programs/client/QueryFuzzer.h index 7c79e683eb4..19f089c6c4e 100644 --- a/programs/client/QueryFuzzer.h +++ b/programs/client/QueryFuzzer.h @@ -17,7 +17,7 @@ namespace DB class ASTExpressionList; class ASTOrderByElement; -struct WindowFrame; +struct ASTWindowDefinition; /* * This is an AST-based query fuzzer that makes random modifications to query @@ -69,7 +69,7 @@ struct QueryFuzzer void fuzzOrderByElement(ASTOrderByElement * elem); void fuzzOrderByList(IAST * ast); void fuzzColumnLikeExpressionList(IAST * ast); - void fuzzWindowFrame(WindowFrame & frame); + void fuzzWindowFrame(ASTWindowDefinition & def); void fuzz(ASTs & asts); void fuzz(ASTPtr & ast); void collectFuzzInfoMain(const ASTPtr ast); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index fa86a9b9377..b7029bc0a79 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -568,22 +568,23 @@ void makeWindowDescriptionFromAST(const Context & context, desc.full_sort_description.insert(desc.full_sort_description.end(), desc.order_by.begin(), desc.order_by.end()); - if (definition.frame.type != WindowFrame::FrameType::Rows - && definition.frame.type != WindowFrame::FrameType::Range) + if (definition.frame_type != WindowFrame::FrameType::Rows + && definition.frame_type != WindowFrame::FrameType::Range) { - std::string name = definition.frame.type == WindowFrame::FrameType::Rows - ? "ROWS" - : definition.frame.type == WindowFrame::FrameType::Groups - ? "GROUPS" : "RANGE"; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Window frame '{}' is not implemented (while processing '{}')", - name, ast->formatForErrorMessage()); + WindowFrame::toString(definition.frame_type), + ast->formatForErrorMessage()); } - desc.frame = definition.frame; + desc.frame.is_default = definition.frame_is_default; + desc.frame.type = definition.frame_type; + desc.frame.begin_type = definition.frame_begin_type; + desc.frame.begin_preceding = definition.frame_begin_preceding; + desc.frame.end_type = definition.frame_end_type; + desc.frame.end_preceding = definition.frame_end_preceding; - if (definition.frame.end_type == WindowFrame::BoundaryType::Offset) + if (definition.frame_end_type == WindowFrame::BoundaryType::Offset) { auto [value, _] = evaluateConstantExpression( definition.frame_end_offset, @@ -591,7 +592,7 @@ void makeWindowDescriptionFromAST(const Context & context, desc.frame.end_offset = value; } - if (definition.frame.begin_type == WindowFrame::BoundaryType::Offset) + if (definition.frame_begin_type == WindowFrame::BoundaryType::Offset) { auto [value, _] = evaluateConstantExpression( definition.frame_begin_offset, diff --git a/src/Parsers/ASTWindowDefinition.cpp b/src/Parsers/ASTWindowDefinition.cpp index 636b826a1e1..7a91097ce99 100644 --- a/src/Parsers/ASTWindowDefinition.cpp +++ b/src/Parsers/ASTWindowDefinition.cpp @@ -26,7 +26,12 @@ ASTPtr ASTWindowDefinition::clone() const result->children.push_back(result->order_by); } - result->frame = frame; + result->frame_is_default = frame_is_default; + result->frame_type = frame_type; + result->frame_begin_type = frame_begin_type; + result->frame_begin_preceding = frame_begin_preceding; + result->frame_end_type = frame_end_type; + result->frame_end_preceding = frame_end_preceding; if (frame_begin_offset) { @@ -87,19 +92,19 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings, need_space = true; } - if (!frame.is_default) + if (!frame_is_default) { if (need_space) { settings.ostr << " "; } - settings.ostr << WindowFrame::toString(frame.type) << " BETWEEN "; - if (frame.begin_type == WindowFrame::BoundaryType::Current) + settings.ostr << WindowFrame::toString(frame_type) << " BETWEEN "; + if (frame_begin_type == WindowFrame::BoundaryType::Current) { settings.ostr << "CURRENT ROW"; } - else if (frame.begin_type == WindowFrame::BoundaryType::Unbounded) + else if (frame_begin_type == WindowFrame::BoundaryType::Unbounded) { settings.ostr << "UNBOUNDED PRECEDING"; } @@ -107,14 +112,14 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings, { frame_begin_offset->formatImpl(settings, state, format_frame); settings.ostr << " " - << (!frame.begin_preceding ? "FOLLOWING" : "PRECEDING"); + << (!frame_begin_preceding ? "FOLLOWING" : "PRECEDING"); } settings.ostr << " AND "; - if (frame.end_type == WindowFrame::BoundaryType::Current) + if (frame_end_type == WindowFrame::BoundaryType::Current) { settings.ostr << "CURRENT ROW"; } - else if (frame.end_type == WindowFrame::BoundaryType::Unbounded) + else if (frame_end_type == WindowFrame::BoundaryType::Unbounded) { settings.ostr << "UNBOUNDED FOLLOWING"; } @@ -122,7 +127,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings, { frame_end_offset->formatImpl(settings, state, format_frame); settings.ostr << " " - << (!frame.end_preceding ? "FOLLOWING" : "PRECEDING"); + << (!frame_end_preceding ? "FOLLOWING" : "PRECEDING"); } } } diff --git a/src/Parsers/ASTWindowDefinition.h b/src/Parsers/ASTWindowDefinition.h index 4edf574cabf..c53f8ef856e 100644 --- a/src/Parsers/ASTWindowDefinition.h +++ b/src/Parsers/ASTWindowDefinition.h @@ -16,12 +16,14 @@ struct ASTWindowDefinition : public IAST ASTPtr order_by; - // Be careful: offsets can contain constant expressions such as INTERVAL 1 DAY, - // that are evaluated later by ExpressionAnalyzer. The WindowFrame struct - // can be incomplete after parsing. - WindowFrame frame; + bool frame_is_default = true; + WindowFrame::FrameType frame_type = WindowFrame::FrameType::Range; + WindowFrame::BoundaryType frame_begin_type = WindowFrame::BoundaryType::Unbounded; ASTPtr frame_begin_offset; + bool frame_begin_preceding = true; + WindowFrame::BoundaryType frame_end_type = WindowFrame::BoundaryType::Current; ASTPtr frame_end_offset; + bool frame_end_preceding = false; ASTPtr clone() const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index a8a037f9d0e..7367be9ac96 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -530,23 +530,23 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p ParserKeyword keyword_groups("GROUPS"); ParserKeyword keyword_range("RANGE"); - node->frame.is_default = false; + node->frame_is_default = false; if (keyword_rows.ignore(pos, expected)) { - node->frame.type = WindowFrame::FrameType::Rows; + node->frame_type = WindowFrame::FrameType::Rows; } else if (keyword_groups.ignore(pos, expected)) { - node->frame.type = WindowFrame::FrameType::Groups; + node->frame_type = WindowFrame::FrameType::Groups; } else if (keyword_range.ignore(pos, expected)) { - node->frame.type = WindowFrame::FrameType::Range; + node->frame_type = WindowFrame::FrameType::Range; } else { /* No frame clause. */ - node->frame.is_default = true; + node->frame_is_default = true; return true; } @@ -565,19 +565,19 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p if (keyword_current_row.ignore(pos, expected)) { - node->frame.begin_type = WindowFrame::BoundaryType::Current; + node->frame_begin_type = WindowFrame::BoundaryType::Current; } else { ParserExpression parser_expression; if (keyword_unbounded.ignore(pos, expected)) { - node->frame.begin_type = WindowFrame::BoundaryType::Unbounded; + node->frame_begin_type = WindowFrame::BoundaryType::Unbounded; } else if (parser_expression.parse(pos, node->frame_begin_offset, expected)) { // We will evaluate the expression for offset expression later. - node->frame.begin_type = WindowFrame::BoundaryType::Offset; + node->frame_begin_type = WindowFrame::BoundaryType::Offset; } else { @@ -586,12 +586,12 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p if (keyword_preceding.ignore(pos, expected)) { - node->frame.begin_preceding = true; + node->frame_begin_preceding = true; } else if (keyword_following.ignore(pos, expected)) { - node->frame.begin_preceding = false; - if (node->frame.begin_type == WindowFrame::BoundaryType::Unbounded) + node->frame_begin_preceding = false; + if (node->frame_begin_type == WindowFrame::BoundaryType::Unbounded) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame start cannot be UNBOUNDED FOLLOWING"); @@ -612,19 +612,19 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p if (keyword_current_row.ignore(pos, expected)) { - node->frame.end_type = WindowFrame::BoundaryType::Current; + node->frame_end_type = WindowFrame::BoundaryType::Current; } else { ParserExpression parser_expression; if (keyword_unbounded.ignore(pos, expected)) { - node->frame.end_type = WindowFrame::BoundaryType::Unbounded; + node->frame_end_type = WindowFrame::BoundaryType::Unbounded; } else if (parser_expression.parse(pos, node->frame_end_offset, expected)) { // We will evaluate the expression for offset expression later. - node->frame.end_type = WindowFrame::BoundaryType::Offset; + node->frame_end_type = WindowFrame::BoundaryType::Offset; } else { @@ -633,8 +633,8 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p if (keyword_preceding.ignore(pos, expected)) { - node->frame.end_preceding = true; - if (node->frame.end_type == WindowFrame::BoundaryType::Unbounded) + node->frame_end_preceding = true; + if (node->frame_end_type == WindowFrame::BoundaryType::Unbounded) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame end cannot be UNBOUNDED PRECEDING"); @@ -643,7 +643,7 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p else if (keyword_following.ignore(pos, expected)) { // Positive offset or UNBOUNDED FOLLOWING. - node->frame.end_preceding = false; + node->frame_end_preceding = false; } else { From 733610dece0b2f552c613f0e7aa10a46a62db3a5 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sun, 30 May 2021 14:09:50 +0300 Subject: [PATCH 258/652] Fixes after review --- src/Common/Config/YAMLParser.cpp | 39 ++++++++-------------------- src/Parsers/examples/YAML_fuzzer.cpp | 17 +++++------- 2 files changed, 18 insertions(+), 38 deletions(-) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index 704a2003e92..5595f61e1ee 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -57,15 +57,7 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen { case YAML::NodeType::Scalar: { - std::string value; - try - { - value = node.as(); - } - catch (const YAML::TypedBadConversion&) - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with value which cannot be represented as string and cannot continue parsing of the file"); - } + std::string value = node.as(); Poco::AutoPtr xml_value = xml_document->createTextNode(value); parent_xml_element.appendChild(xml_value); break; @@ -118,29 +110,13 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen { const auto & key_node = key_value_pair.first; const auto & value_node = key_value_pair.second; - std::string key; - try - { - key = key_node.as(); - } - catch (const YAML::TypedBadConversion&) - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with key which cannot be represented as string and cannot continue parsing of the file"); - } + std::string key = key_node.as(); bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar()); if (is_attribute) { /// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key auto attribute_name = key.substr(1); - std::string value; - try - { - value = value_node.as(); - } - catch (const YAML::TypedBadConversion&) - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with value which cannot be represented as string and cannot continue parsing of the file"); - } + std::string value = value_node.as(); parent_xml_element.setAttribute(attribute_name, value); } else @@ -182,7 +158,14 @@ Poco::AutoPtr YAMLParser::parse(const String& path) Poco::AutoPtr xml = new Document; Poco::AutoPtr root_node = xml->createElement("yandex"); xml->appendChild(root_node); - processNode(node_yml, *root_node); + try + { + processNode(node_yml, *root_node); + } + catch (const YAML::TypedBadConversion&) + { + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with key or value which cannot be represented as string and cannot continue parsing of the file"); + } return xml; } diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Parsers/examples/YAML_fuzzer.cpp index c18b9ffae70..223bd89c7b2 100644 --- a/src/Parsers/examples/YAML_fuzzer.cpp +++ b/src/Parsers/examples/YAML_fuzzer.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -10,15 +11,13 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) /// build ClickHouse with YAML_fuzzer.cpp /// ./YAML_fuzzer YAML_CORPUS /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer - - srand(time(NULL)); - std::string cur_file = std::to_string(rand()); - - while (std::filesystem::exists(cur_file)) - { - std::string cur_file = std::to_string(rand()); + char* file_name = std::tmpnam(nullptr); + if (file_name == nullptr) { + std::cout << "Cannot create temp file!\n"; + return 1; } - + std::string cur_file(file_name); + std::string input = std::string(reinterpret_cast(data), size); DB::YAMLParser parser; @@ -34,8 +33,6 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { std::cout << "YAMLParser exception from bad file, etc. OK\n"; } - - remove(cur_file.c_str()); return 0; } From 29e50feef6500a954718cd1b0656f8415788c456 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sun, 30 May 2021 14:09:50 +0300 Subject: [PATCH 259/652] Fixes after review --- src/Common/Config/YAMLParser.cpp | 39 ++++++++-------------------- src/Parsers/examples/YAML_fuzzer.cpp | 17 +++++------- 2 files changed, 18 insertions(+), 38 deletions(-) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index 704a2003e92..5595f61e1ee 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -57,15 +57,7 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen { case YAML::NodeType::Scalar: { - std::string value; - try - { - value = node.as(); - } - catch (const YAML::TypedBadConversion&) - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with value which cannot be represented as string and cannot continue parsing of the file"); - } + std::string value = node.as(); Poco::AutoPtr xml_value = xml_document->createTextNode(value); parent_xml_element.appendChild(xml_value); break; @@ -118,29 +110,13 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen { const auto & key_node = key_value_pair.first; const auto & value_node = key_value_pair.second; - std::string key; - try - { - key = key_node.as(); - } - catch (const YAML::TypedBadConversion&) - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with key which cannot be represented as string and cannot continue parsing of the file"); - } + std::string key = key_node.as(); bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar()); if (is_attribute) { /// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key auto attribute_name = key.substr(1); - std::string value; - try - { - value = value_node.as(); - } - catch (const YAML::TypedBadConversion&) - { - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with value which cannot be represented as string and cannot continue parsing of the file"); - } + std::string value = value_node.as(); parent_xml_element.setAttribute(attribute_name, value); } else @@ -182,7 +158,14 @@ Poco::AutoPtr YAMLParser::parse(const String& path) Poco::AutoPtr xml = new Document; Poco::AutoPtr root_node = xml->createElement("yandex"); xml->appendChild(root_node); - processNode(node_yml, *root_node); + try + { + processNode(node_yml, *root_node); + } + catch (const YAML::TypedBadConversion&) + { + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with key or value which cannot be represented as string and cannot continue parsing of the file"); + } return xml; } diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Parsers/examples/YAML_fuzzer.cpp index c18b9ffae70..223bd89c7b2 100644 --- a/src/Parsers/examples/YAML_fuzzer.cpp +++ b/src/Parsers/examples/YAML_fuzzer.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -10,15 +11,13 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) /// build ClickHouse with YAML_fuzzer.cpp /// ./YAML_fuzzer YAML_CORPUS /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer - - srand(time(NULL)); - std::string cur_file = std::to_string(rand()); - - while (std::filesystem::exists(cur_file)) - { - std::string cur_file = std::to_string(rand()); + char* file_name = std::tmpnam(nullptr); + if (file_name == nullptr) { + std::cout << "Cannot create temp file!\n"; + return 1; } - + std::string cur_file(file_name); + std::string input = std::string(reinterpret_cast(data), size); DB::YAMLParser parser; @@ -34,8 +33,6 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { std::cout << "YAMLParser exception from bad file, etc. OK\n"; } - - remove(cur_file.c_str()); return 0; } From 4a60e3545e9bcd9711c263c3e0bf42ea4a9a958c Mon Sep 17 00:00:00 2001 From: adevyatova Date: Sun, 30 May 2021 13:01:25 +0000 Subject: [PATCH 260/652] add en,ru docs --- docs/en/operations/settings/merge-tree-settings.md | 11 +++++++++++ docs/ru/operations/settings/merge-tree-settings.md | 11 +++++++++++ 2 files changed, 22 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 77b68715ba9..65283489226 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -115,6 +115,17 @@ Default value: 604800 (1 week). Similar to [replicated_deduplication_window](#replicated-deduplication-window), `replicated_deduplication_window_seconds` specifies how long to store hash sums of blocks for insert deduplication. Hash sums older than `replicated_deduplication_window_seconds` are removed from Zookeeper, even if they are less than ` replicated_deduplication_window`. +## non_replicated_deduplication_window {#non-replicated-deduplication-window } + +The number of most recently inserted blocks for which should be stores hash sums to check for duplicates for non-replicated MergeTree tables. + +Possible values: + +- Any positive integer. +- 0 (disable deduplication). + +Default value: 0. + ## old_parts_lifetime {#old-parts-lifetime} The time (in seconds) of storing inactive parts to protect against data loss during spontaneous server reboots. diff --git a/docs/ru/operations/settings/merge-tree-settings.md b/docs/ru/operations/settings/merge-tree-settings.md index bfc0b0a2644..75a82dfa72c 100644 --- a/docs/ru/operations/settings/merge-tree-settings.md +++ b/docs/ru/operations/settings/merge-tree-settings.md @@ -112,6 +112,17 @@ Eсли суммарное число активных кусков во все Аналогично [replicated_deduplication_window](#replicated-deduplication-window), задает, сколько времени хранить хеш-суммы блоков для дедупликции `Insert`-в. Хеш-суммы старше `replicated_deduplication_window_seconds` удаляются из Zookeeper, даже если их меньше чем `replicated_deduplication_window`. +## non_replicated_deduplication_window {#non-replicated-deduplication-window } + +Количество последних вставленных блоков, для которых должны храниться хэш-суммы для проверки дубликатов, для нереплицированных MergeTree таблиц. + +Возможные значения: + +- Положительное целое число. +- 0 (дедупликация отключена). + +Значение по умолчанию: 0. + ## old_parts_lifetime {#old-parts-lifetime} Время (в секундах) хранения неактивных кусков, для защиты от потери данных при спонтанной перезагрузке сервера или О.С. From 68b10843557bcca70009c86012e186926548b1ad Mon Sep 17 00:00:00 2001 From: adevyatova Date: Sun, 30 May 2021 13:28:21 +0000 Subject: [PATCH 261/652] add en docs --- docs/en/sql-reference/statements/alter/partition.md | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index f7183ba525c..30d79b47c0a 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -16,6 +16,7 @@ The following operations with [partitions](../../../engines/table-engines/merget - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition. - [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition. +- [UNFREEZE PARTITION](#alter_unfreeze-partition) — Removes a backup of a partition. - [FETCH PARTITION](#alter_fetch-partition) — Downloads a partition from another server. - [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume. @@ -172,7 +173,7 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a !!! note "Note" If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. -The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs ‘chmod’ for all files, forbidding writing into them. +The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs `chmod` for all files, forbidding writing into them. After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server. @@ -190,6 +191,14 @@ Restoring from a backup doesn’t require stopping the server. For more information about backups and restoring data, see the [Data Backup](../../../operations/backup.md) section. +## UNFREEZE PARTITION {#alter_unfreeze-partition} + +``` sql +ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' +``` + +Removes "freezed" partitions with the specified name from the disk. If the `PARTITION` clause is omitted, the query removes the backup of all partitions at once. + ## CLEAR INDEX IN PARTITION {#alter_clear-index-partition} ``` sql From 1cda2bb2f02cc38dec11b176ad70a29e72bdf087 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 30 May 2021 14:33:45 +0300 Subject: [PATCH 262/652] Process config w/o extensions as XML format I'm using such hack to hide the password: $ clickhouse-client --config <(echo '') But this had been broken in #21858 v2: add a test v3: fail for unsupported extensions --- src/Common/Config/ConfigProcessor.cpp | 11 ++- ..._clickhouse_client_config_format.reference | 14 ++++ .../01889_clickhouse_client_config_format.sh | 68 +++++++++++++++++++ 3 files changed, 90 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01889_clickhouse_client_config_format.reference create mode 100755 tests/queries/0_stateless/01889_clickhouse_client_config_format.sh diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index fa9e9b72087..f799a5cfa8e 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -36,6 +36,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_LOAD_CONFIG; } /// For cutting preprocessed path to this base @@ -464,13 +465,17 @@ XMLDocumentPtr ConfigProcessor::processConfig( if (fs::exists(path)) { fs::path p(path); - if (p.extension() == ".xml") + if (p.extension() == ".yaml" || p.extension() == ".yml") + { + config = YAMLParser::parse(path); + } + else if (p.extension() == ".xml" || p.extension() == ".conf" || p.extension().empty()) { config = dom_parser.parse(path); } - else if (p.extension() == ".yaml" || p.extension() == ".yml") + else { - config = YAMLParser::parse(path); + throw Exception(ErrorCodes::CANNOT_LOAD_CONFIG, "Unknown format of '{}' config", path); } } else diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference new file mode 100644 index 00000000000..30c8a44486b --- /dev/null +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -0,0 +1,14 @@ +default +2 +xml +2 +conf +2 +/dev/fd/PIPE +2 +yml +2 +yaml +2 +ini +Code: 347. Unknown format of '/config_default.ini' config diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh new file mode 100755 index 00000000000..d09efc9a15c --- /dev/null +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -0,0 +1,68 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# use $CLICKHOUSE_DATABASE so that clickhouse-test will replace it with default to match .reference +config=$CUR_DIR/config_$CLICKHOUSE_DATABASE +xml_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.xml +conf_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.conf +yml_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.yml +yaml_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.yaml +ini_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.ini + +function cleanup() +{ + rm "${config:?}" + rm "${xml_config:?}" + rm "${conf_config:?}" + rm "${yml_config:?}" + rm "${yaml_config:?}" + rm "${ini_config:?}" +} +trap cleanup EXIT + +cat > "$config" < + 2 + +EOL +cat > "$conf_config" < + 2 + +EOL +cat > "$xml_config" < + 2 + +EOL +cat > "$yml_config" < "$yaml_config" < "$ini_config" <2') -q "select getSetting('max_threads')" + +echo 'yml' +$CLICKHOUSE_CLIENT --config "$yml_config" -q "select getSetting('max_threads')" +echo 'yaml' +$CLICKHOUSE_CLIENT --config "$yaml_config" -q "select getSetting('max_threads')" +echo 'ini' +$CLICKHOUSE_CLIENT --config "$ini_config" -q "select getSetting('max_threads')" 2>&1 |& sed -e "s#$CUR_DIR##" -e "s#DB::Exception: ##" From eb08bb066947eb9fea0dde7c6e18c5d5e991d09a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 30 May 2021 15:28:41 +0300 Subject: [PATCH 263/652] Case-insensitive config extensions --- src/Common/Config/ConfigProcessor.cpp | 16 +++++++++++++--- ...889_clickhouse_client_config_format.reference | 2 ++ .../01889_clickhouse_client_config_format.sh | 9 +++++++++ 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index f799a5cfa8e..5d64e3ea4ea 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -440,6 +441,8 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & std::string extension = path.getExtension(); std::string base_name = path.getBaseName(); + boost::algorithm::to_lower(extension); + // Skip non-config and temporary files if (file.isFile() && (extension == "xml" || extension == "conf" || extension == "yaml" || extension == "yml") && !startsWith(base_name, ".")) { @@ -465,11 +468,15 @@ XMLDocumentPtr ConfigProcessor::processConfig( if (fs::exists(path)) { fs::path p(path); - if (p.extension() == ".yaml" || p.extension() == ".yml") + + std::string extension = p.extension(); + boost::algorithm::to_lower(extension); + + if (extension == ".yaml" || extension == ".yml") { config = YAMLParser::parse(path); } - else if (p.extension() == ".xml" || p.extension() == ".conf" || p.extension().empty()) + else if (extension == ".xml" || extension == ".conf" || extension.empty()) { config = dom_parser.parse(path); } @@ -514,7 +521,10 @@ XMLDocumentPtr ConfigProcessor::processConfig( XMLDocumentPtr with; fs::path p(merge_file); - if (p.extension() == ".yaml" || p.extension() == ".yml") + std::string extension = p.extension(); + boost::algorithm::to_lower(extension); + + if (extension == ".yaml" || extension == ".yml") { with = YAMLParser::parse(merge_file); } diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index 30c8a44486b..aa7748928f1 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -2,6 +2,8 @@ default 2 xml 2 +XML +2 conf 2 /dev/fd/PIPE diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index d09efc9a15c..588e582894a 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -7,6 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # use $CLICKHOUSE_DATABASE so that clickhouse-test will replace it with default to match .reference config=$CUR_DIR/config_$CLICKHOUSE_DATABASE xml_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.xml +XML_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.XML conf_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.conf yml_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.yml yaml_config=$CUR_DIR/config_$CLICKHOUSE_DATABASE.yaml @@ -16,6 +17,7 @@ function cleanup() { rm "${config:?}" rm "${xml_config:?}" + rm "${XML_config:?}" rm "${conf_config:?}" rm "${yml_config:?}" rm "${yaml_config:?}" @@ -38,6 +40,11 @@ cat > "$xml_config" <2 EOL +cat > "$XML_config" < + 2 + +EOL cat > "$yml_config" < Date: Sun, 30 May 2021 22:28:35 +0300 Subject: [PATCH 264/652] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 95d6bbba579..62fed1c8e6f 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 95d6bbba579b3a4e4c2dede954f541ff6f3dba51 +Subproject commit 62fed1c8e6fb863fcd49829c812a76be005ab756 From 16647fe8ce8a2f6b8b6f5b671aae2e60059d0070 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 May 2021 00:29:37 +0300 Subject: [PATCH 265/652] some unrelated fixes --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- src/Storages/MergeTree/MergeTreePartInfo.h | 7 +++++++ .../ReplicatedMergeTreePartCheckThread.cpp | 7 ++++--- .../ReplicatedMergeTreeRestartingThread.cpp | 8 +++++++- src/Storages/StorageReplicatedMergeTree.cpp | 15 ++++++++++++--- .../test.py | 18 ++++++++++++++---- .../01307_multiple_leaders_zookeeper.sh | 2 +- ...320_create_sync_race_condition_zookeeper.sh | 2 +- ...01417_freeze_partition_verbose_zookeeper.sh | 2 +- 9 files changed, 52 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ed3f89b2c3..188f6366f8b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4897,7 +4897,7 @@ void MergeTreeData::removeQueryId(const String & query_id) const { std::lock_guard lock(query_id_set_mutex); if (query_id_set.find(query_id) == query_id_set.end()) - LOG_WARNING(log, "We have query_id removed but it's not recorded. This is a bug"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "We have query_id removed but it's not recorded. This is a bug"); else query_id_set.erase(query_id); } @@ -5077,7 +5077,10 @@ CurrentlySubmergingEmergingTagger::~CurrentlySubmergingEmergingTagger() for (const auto & part : submerging_parts) { if (!storage.currently_submerging_big_parts.count(part)) - LOG_WARNING(log, "currently_submerging_big_parts doesn't contain part {} to erase. This is a bug", part->name); + { + LOG_ERROR(log, "currently_submerging_big_parts doesn't contain part {} to erase. This is a bug", part->name); + assert(false); + } else storage.currently_submerging_big_parts.erase(part); } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index e81143de8c2..66d5342b67f 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -71,6 +71,13 @@ struct MergeTreePartInfo || max_block < rhs.min_block; } + bool isFakeDropRangePart() const + { + /// Another max level was previously used for REPLACE/MOVE PARTITION + auto another_max_level = std::numeric_limits::max(); + return level == MergeTreePartInfo::MAX_LEVEL || level == another_max_level; + } + String getPartName() const; String getPartNameV0(DayNum left_date, DayNum right_date) const; UInt64 getBlocksCount() const diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 09b2a23767c..35c42a7f325 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -18,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int TABLE_DIFFERS_TOO_MUCH; + extern const int LOGICAL_ERROR; } static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000; @@ -190,7 +191,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPartAndFetchIfPossible( if (missing_part_search_result == MissingPartSearchResult::LostForever) { /// Is it in the replication queue? If there is - delete, because the task can not be processed. - if (!storage.queue.remove(zookeeper, part_name)) + if (!storage.queue.markPartAsLostForever(zookeeper, part_name)) { /// The part was not in our queue. LOG_WARNING(log, "Missing part {} is not in our queue, this can happen rarely.", part_name); @@ -367,8 +368,8 @@ void ReplicatedMergeTreePartCheckThread::run() { if (!parts_set.empty()) { - LOG_ERROR(log, "Non-empty parts_set with empty parts_queue. This is a bug."); parts_set.clear(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-empty parts_set with empty parts_queue. This is a bug."); } } else @@ -401,7 +402,7 @@ void ReplicatedMergeTreePartCheckThread::run() if (parts_queue.empty()) { - LOG_ERROR(log, "Someone erased checking part from parts_queue. This is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Someone erased checking part from parts_queue. This is a bug."); } else { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 6b7fb3bf17f..f3c5e88f54a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -174,6 +174,9 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() storage.partial_shutdown_called = false; storage.partial_shutdown_event.reset(); + /// Start queue processing + storage.background_executor.start(); + storage.queue_updating_task->activateAndSchedule(); storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); @@ -227,7 +230,7 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts() { LOG_DEBUG(log, "Found part {} with failed quorum. Moving to detached. This shouldn't happen often.", part_name); storage.forgetPartAndMoveToDetached(part, "noquorum"); - storage.queue.removeFromVirtualParts(part->info); + storage.queue.removeFailedQuorumPart(part->info); } } } @@ -352,6 +355,9 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown() storage.cleanup_thread.stop(); storage.part_check_thread.stop(); + /// Stop queue processing + storage.background_executor.finish(); + LOG_TRACE(log, "Threads finished"); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b3da3e2287b..23abb39eb3e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1465,6 +1465,12 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } + if (entry.type == LogEntry::PART_IS_LOST) + { + queue.executePartIsLost(getZooKeeper(), entry); + return true; + } + const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; if (is_get_or_attach || entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) @@ -2043,7 +2049,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) if (code == Coordination::Error::ZOK) { LOG_DEBUG(log, "Marked quorum for part {} as failed.", entry.new_part_name); - queue.removeFromVirtualParts(part_info); + queue.removeFailedQuorumPart(part_info); return true; } else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) @@ -2055,7 +2061,10 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) } else { - LOG_WARNING(log, "No active replica has part {}, but that part needs quorum and /quorum/status contains entry about another part {}. It means that part was successfully written to {} replicas, but then all of them goes offline. Or it is a bug.", entry.new_part_name, quorum_entry.part_name, entry.quorum); + LOG_WARNING(log, "No active replica has part {}, " + "but that part needs quorum and /quorum/status contains entry about another part {}. " + "It means that part was successfully written to {} replicas, but then all of them goes offline. " + "Or it is a bug.", entry.new_part_name, quorum_entry.part_name, entry.quorum); } } } @@ -2743,7 +2752,6 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo { if (active_parts_set.getContainingPart(part).empty()) { - queue.remove(zookeeper, part); parts_to_remove_from_zk.emplace_back(part); LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part); } @@ -2988,6 +2996,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke /// Clear obsolete queue that we no longer need. zookeeper->removeChildren(replica_path + "/queue"); + queue.clear(); /// Will do repair from the selected replica. cloneReplica(source_replica, source_is_lost_stat, zookeeper); diff --git a/tests/integration/test_consistent_parts_after_clone_replica/test.py b/tests/integration/test_consistent_parts_after_clone_replica/test.py index 784f94397af..459bba42028 100644 --- a/tests/integration/test_consistent_parts_after_clone_replica/test.py +++ b/tests/integration/test_consistent_parts_after_clone_replica/test.py @@ -37,12 +37,17 @@ def start_cluster(): def test_inconsistent_parts_if_drop_while_replica_not_active(start_cluster): with PartitionManager() as pm: # insert into all replicas - for i in range(50): + for i in range(10): node1.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(i)) assert_eq_with_retry(node2, "SELECT count(*) FROM test_table", node1.query("SELECT count(*) FROM test_table")) - # disable network on the first replica + # partition the first replica from the second one and (later) from zk pm.partition_instances(node1, node2) + + # insert some parts on the second replica only, we will drop these parts + for i in range(10): + node2.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(10 + i)) + pm.drop_instance_zk_connections(node1) # drop all parts on the second replica @@ -51,9 +56,14 @@ def test_inconsistent_parts_if_drop_while_replica_not_active(start_cluster): # insert into the second replica # DROP_RANGE will be removed from the replication log and the first replica will be lost - for i in range(50): - node2.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(50 + i)) + for i in range(20): + node2.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(20 + i)) # the first replica will be cloned from the second pm.heal_all() assert_eq_with_retry(node1, "SELECT count(*) FROM test_table", node2.query("SELECT count(*) FROM test_table")) + # ensure replica was cloned + assert node1.contains_in_log("Will mimic node2") + # queue must be empty (except some merges that are possibly executing right now) + assert node1.query("SELECT count() FROM system.replication_queue WHERE type != 'MERGE_PARTS'") == "0\n" + assert node2.query("SELECT count() FROM system.replication_queue WHERE type != 'MERGE_PARTS'") == "0\n" diff --git a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh index 21fc88d7c2d..37ee5bf7ad7 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh @@ -12,7 +12,7 @@ DATA_SIZE=200 SEQ=$(seq 0 $(($NUM_REPLICAS - 1))) for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE IF EXISTS r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done function thread() { diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index 97c200c651f..678a3aa12fe 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Diff function thread1() { - while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done + while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done } function thread2() diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index bb935a950ff..16b39db6e84 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -11,7 +11,7 @@ FREEZE_OUT_STRUCTURE='backup_name String, backup_path String , part_backup_path # setup ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" -${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" +${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" ${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ From 2571ad7d438297d9aa799783c2301f520718572c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 May 2021 00:30:50 +0300 Subject: [PATCH 266/652] try fix intersecting virtual parts --- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 40 ++- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 14 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 271 +++++++++++++++--- .../MergeTree/ReplicatedMergeTreeQueue.h | 16 +- 4 files changed, 297 insertions(+), 44 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index dbd55bc4ff3..ee802a2a5b1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -152,6 +152,10 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << "sync_pinned_part_uuids\n"; break; + case PART_IS_LOST: + out << "lost\n" << new_part_name; + break; + default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown log entry type: {}", static_cast(type)); } @@ -326,6 +330,11 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) in >> new_part_name; in >> "\nsource_shard: " >> source_shard; } + else if (type_str == "lost") + { + type = PART_IS_LOST; + in >> new_part_name; + } if (!trailing_newline_found) in >> "\n"; @@ -389,7 +398,6 @@ void ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry::readText(ReadBuffer & i bool ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry::isMovePartitionOrAttachFrom(const MergeTreePartInfo & drop_range_info) { - assert(drop_range_info.getBlocksCount() != 0); return drop_range_info.getBlocksCount() == 1; } @@ -413,6 +421,24 @@ ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String return res; } +std::optional ReplicatedMergeTreeLogEntryData::getDropRange(MergeTreeDataFormatVersion format_version) const +{ + if (type == DROP_RANGE) + return new_part_name; + + if (type == REPLACE_RANGE) + { + auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version); + if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info)) + { + /// It's REPLACE, not MOVE or ATTACH, so drop range is real + return replace_range_entry->drop_range_part_name; + } + } + + return {}; +} + Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormatVersion format_version) const { /// Doesn't produce any part @@ -430,12 +456,8 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat if (type == REPLACE_RANGE) { Strings res = replace_range_entry->new_part_names; - auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version); - if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info)) - { - /// It's REPLACE, not MOVE or ATTACH, so drop range is real - res.emplace_back(replace_range_entry->drop_range_part_name); - } + if (auto drop_range = getDropRange(format_version)) + res.emplace_back(*drop_range); return res; } @@ -447,6 +469,10 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat if (type == CLONE_PART_FROM_SHARD) return {}; + /// Doesn't produce any part. + if (type == PART_IS_LOST) + return {}; + return {new_part_name}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 12f1c78fb5d..5d62f219924 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -45,6 +45,7 @@ struct ReplicatedMergeTreeLogEntryData ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths SYNC_PINNED_PART_UUIDS, /// Synchronization point for ensuring that all replicas have up to date in-memory state. CLONE_PART_FROM_SHARD, /// Clone part from another shard. + PART_IS_LOST, /// Cancels previous operations with lost data part. Kinda "anti-merge". }; static String typeToString(Type type) @@ -62,6 +63,7 @@ struct ReplicatedMergeTreeLogEntryData case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; case ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS: return "SYNC_PINNED_PART_UUIDS"; case ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD: return "CLONE_PART_FROM_SHARD"; + case ReplicatedMergeTreeLogEntryData::PART_IS_LOST: return "PART_IS_LOST"; default: throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); } @@ -140,6 +142,18 @@ struct ReplicatedMergeTreeLogEntryData /// selection of merges. These parts are added to queue.virtual_parts. Strings getVirtualPartNames(MergeTreeDataFormatVersion format_version) const; + /// Returns name of part that will never appear and should be removed from virtual parts set. + /// It's required to correctly cancel merge which cannot be executed, because some source part is lost forever. + /// Do not use it for other purposes, it can be dangerous. + std::optional getAntiVirtualPartName() const + { + if (type == PART_IS_LOST) + return new_part_name; + return {}; + } + + std::optional getDropRange(MergeTreeDataFormatVersion format_version) const; + /// Returns set of parts that denote the block number ranges that should be blocked during the entry execution. /// These parts are added to future_parts. Strings getBlockingPartNames(MergeTreeDataFormatVersion format_version) const diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 30569e53f64..84050f21528 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -33,14 +33,20 @@ ReplicatedMergeTreeQueue::ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & log = &Poco::Logger::get(logger_name); } - -void ReplicatedMergeTreeQueue::initialize(const MergeTreeData::DataParts & parts) +void ReplicatedMergeTreeQueue::clear() { - addVirtualParts(parts); + auto locks = lockQueue(); + assert(future_parts.empty()); + current_parts.clear(); + virtual_parts.clear(); + queue.clear(); + inserts_by_time.clear(); + mutations_by_znode.clear(); + mutations_by_partition.clear(); + mutation_pointer.clear(); } - -void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts & parts) +void ReplicatedMergeTreeQueue::initialize(const MergeTreeData::DataParts & parts) { std::lock_guard lock(state_mutex); @@ -74,9 +80,6 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) /// Reset batch size on initialization to recover from possible errors of too large batch size. current_multi_batch_size = 1; - String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); - log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); - std::unordered_set already_loaded_paths; { std::lock_guard lock(state_mutex); @@ -132,6 +135,45 @@ void ReplicatedMergeTreeQueue::insertUnlocked( const LogEntryPtr & entry, std::optional & min_unprocessed_insert_time_changed, std::lock_guard & state_lock) { + if (auto lost_part = entry->getAntiVirtualPartName()) + { + LOG_TRACE(log, "Reading {}: PART_IS_LOST for {}", entry->znode_name, *lost_part); + QueueIters entries = findEntriesByNewPartName(*lost_part); + bool removed = virtual_parts.remove(*lost_part); + if (!removed && !entries.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Found {} entries for lost part {}, " + "but nothing is removed from virtual parts", entries.size(), *lost_part); + for (const auto & entry_it : entries) + { + for (const auto & source_part : (*entry_it)->source_parts) + { + String containing_current_part = current_parts.getContainingPart(source_part); + if (containing_current_part.empty()) + { + bool has_source_entry = std::any_of(queue.begin(), queue.end(), [&lost_part](auto e) + { + return e->new_part_name == *lost_part; + }); + if (!has_source_entry) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Source entry ({}) of lost virtual part {} does not exist", + source_part, *lost_part); + } + else + { + if (containing_current_part != source_part) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Source part {} of lost part {} is covered by current part {}. Entry: {}", + source_part, *lost_part, containing_current_part, (*entry_it)->toString()); + } + + virtual_parts.add(source_part); + } + } + + lost_forever_parts.insert(*lost_part); + } + for (const String & virtual_part_name : entry->getVirtualPartNames(format_version)) { virtual_parts.add(virtual_part_name); @@ -228,16 +270,10 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( removeCoveredPartsFromMutations(virtual_part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } - String drop_range_part_name; - if (entry->type == LogEntry::DROP_RANGE) - drop_range_part_name = entry->new_part_name; - else if (entry->type == LogEntry::REPLACE_RANGE) - drop_range_part_name = entry->replace_range_entry->drop_range_part_name; - - if (!drop_range_part_name.empty()) + if (auto drop_range_part_name = entry->getDropRange(format_version)) { - current_parts.remove(drop_range_part_name); - virtual_parts.remove(drop_range_part_name); + current_parts.remove(*drop_range_part_name); + virtual_parts.remove(*drop_range_part_name); } if (entry->type == LogEntry::ALTER_METADATA) @@ -301,10 +337,7 @@ void ReplicatedMergeTreeQueue::addPartToMutations(const String & part_name) { auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - /// Do not add special virtual parts to parts_to_do - auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION - auto another_max_level = std::numeric_limits::max(); /// REPLACE/MOVE PARTITION - if (part_info.level == max_level || part_info.level == another_max_level) + if (part_info.isFakeDropRangePart()) return; auto in_partition = mutations_by_partition.find(part_info.partition_id); @@ -392,7 +425,8 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep } if (!found && need_remove_from_zk) - throw Exception("Can't find " + entry->znode_name + " in the memory queue. It is a bug", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't find {} in the memory queue. It is a bug. Entry info: {}", + entry->znode_name, entry->toString()); notifySubscribers(queue_size); @@ -461,9 +495,162 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri return true; } - -bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & part_info) +bool ReplicatedMergeTreeQueue::markPartAsLostForever(zkutil::ZooKeeperPtr zookeeper, const String & part_name) { + LogEntry entry; + entry.type = LogEntry::PART_IS_LOST; + entry.source_replica = storage.replica_name; + entry.new_part_name = part_name; + + size_t max_iters = 100; + while (--max_iters) + { + ReplicatedMergeTreeMergePredicate merge_pred = getMergePredicate(zookeeper); + + if (!merge_pred.ensurePartIsLost(part_name, log)) + return false; + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + Coordination::Responses responses; + Coordination::Error rc = zookeeper->tryMulti(ops, responses); + + if (rc == Coordination::Error::ZBADVERSION) + { + LOG_TRACE(log, "A new log entry appeared while trying to commit PART_IS_LOST. Retry."); + continue; + } else + zkutil::KeeperMultiException::check(rc, ops, responses); + + String path_created = dynamic_cast(*responses.back()).path_created; + LOG_TRACE(log, "Created PART_IS_LOST entry ({}) for part {}", path_created, part_name); + return true; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create PART_IS_LOST entry for {}", part_name); +} + +ReplicatedMergeTreeQueue::QueueIters ReplicatedMergeTreeQueue::findEntriesByNewPartName(const String & part_name) const +{ + QueueIters entries; + size_t get_entries_num = 0; + for (auto it = queue.begin(); it != queue.end(); ++it) + { + if ((*it)->new_part_name == part_name && (*it)->type != LogEntry::PART_IS_LOST) + { + entries.push_back(it); + get_entries_num += (*it)->type == LogEntry::GET_PART; + } + } + + if (!entries.empty()) + { + if (entries.size() != 1) + { + /// Replication queue may contain multiple part producing entries with the same new_part_name + /// if replica was recently cloned. At first cloneReplica(...) copies source replica queue, + /// and after that it reads data parts set from source_path + "/parts" + /// to create GET_PART entries in own queue and download missing parts. + /// Therefore, some MERGE_PART (or MUTATE_PART, or even GET_PART) entry + /// may be duplicated with GET_PART with the same new_part_name. + if (1 < entries.size() - get_entries_num) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Found {} queue entries for {}, only {} are GET_PART", + entries.size(), part_name, get_entries_num); + + LOG_WARNING(log, "Found {} queue entries for {}, including {} GET_PART entries. " + "It may rarely happen after replica cloning", entries.size(), part_name, get_entries_num); + } + } + + return entries; +} + +void ReplicatedMergeTreeQueue::executePartIsLost(zkutil::ZooKeeperPtr zookeeper, LogEntry & entry_lost) +{ + LOG_TRACE(log, "Executing {}: PART_IS_LOST for part {} ({})", entry_lost.znode_name, entry_lost.new_part_name); + + /// There might be multiple cancelled parts if replica was cloned... + + size_t queue_size = 0; + std::optional min_unprocessed_insert_time_changed; + std::optional max_processed_insert_time_changed; + Coordination::Requests ops; + { + std::unique_lock lock(state_mutex); + QueueIters cancelled_entries = findEntriesByNewPartName(entry_lost.new_part_name); + + for (const auto entry_it : cancelled_entries) + { + LogEntryPtr canceled_entry; + canceled_entry = *entry_it; + LOG_TRACE(log, "Removing cancelled log entry {}: {}", canceled_entry->znode_name, canceled_entry->toString()); + ops.emplace_back(zkutil::makeRemoveRequest(replica_path + "/queue/" + canceled_entry->znode_name, -1)); + updateStateOnQueueEntryRemoval( + canceled_entry, /* is_successful = */ false, + min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); + queue.erase(entry_it); + queue_size = queue.size(); + + LOG_TRACE(log, "Waiting for {} to finish", canceled_entry->znode_name); + canceled_entry->execution_complete.wait(lock, [&canceled_entry] + { return !canceled_entry->currently_executing; }); + } + } + + notifySubscribers(queue_size); + + zookeeper->multi(ops); + + updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); + + { + std::unique_lock lock(state_mutex); + lost_forever_parts.erase(entry_lost.new_part_name); + } + + LOG_TRACE(log, "Executed {}: PART_IS_LOST for part {}", entry_lost.znode_name, entry_lost.new_part_name); +} + +bool ReplicatedMergeTreeMergePredicate::ensurePartIsLost(const String & part_name, Poco::Logger * log) const +{ + String containing_part; + { + std::lock_guard lock(queue.state_mutex); + containing_part = queue.virtual_parts.getContainingPart(part_name); + } + + if (containing_part.empty()) + { + LOG_WARNING(log, "Cannot find lost part {} in virtual parts set. " + "Probably it's already marked as lost by another replica.", part_name); + return false; + } + + if (containing_part != part_name) + { + auto info = MergeTreePartInfo::fromPartName(containing_part, queue.format_version); + if (info.isFakeDropRangePart()) + { + /// It does not matter anymore if part is lost, it should be removed anyway + LOG_INFO(log, "Lost part {} is covered by drop range {}, ignoring it", part_name, containing_part); + return false; + } + + /// Lost part is covered by real part. It means that merging/mutating operation + /// was assigned on lost part by some replica. It probably means that part is not lost. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Lost part {} is covered by real part {}", part_name, containing_part); + } + + /// We can mark part as lost + return true; +} + +bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & part_info) +{ + if (part_info.level != 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove part with non-zero level: {}", part_info.getPartName()); std::lock_guard lock(state_mutex); return virtual_parts.remove(part_info); } @@ -587,8 +774,6 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper { std::lock_guard state_lock(state_mutex); - log_pointer = last_entry_index + 1; - for (size_t copied_entry_idx = 0, num_copied_entries = copied_entries.size(); copied_entry_idx < num_copied_entries; ++copied_entry_idx) { String path_created = dynamic_cast(*responses[copied_entry_idx]).path_created; @@ -758,9 +943,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C /// Such parts do not exist and will never appear, so we should not add virtual parts to parts_to_do list. /// Fortunately, it's easy to distinguish virtual parts from normal parts by part level. /// See StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(...) - auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION - auto another_max_level = std::numeric_limits::max(); /// REPLACE/MOVE PARTITION - if (part_info.level == max_level || part_info.level == another_max_level) + if (part_info.isFakeDropRangePart()) continue; auto it = entry->block_numbers.find(part_info.partition_id); @@ -941,9 +1124,6 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( if ((*it)->currently_executing) to_wait.push_back(*it); auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); - /// FIXME it's probably unsafe to remove entries non-atomically - /// when this method called directly from alter query (not from replication queue task), - /// because entries will be lost if ALTER fails. if (code != Coordination::Error::ZOK) LOG_INFO(log, "Couldn't remove {}: {}", replica_path + "/queue/" + (*it)->znode_name, Coordination::errorMessage(code)); @@ -1038,11 +1218,26 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( || entry.type == LogEntry::ATTACH_PART || entry.type == LogEntry::MUTATE_PART) { + assert(entry.getBlockingPartNames(format_version).size() == 1); for (const String & new_part_name : entry.getBlockingPartNames(format_version)) { if (!isNotCoveredByFuturePartsImpl(entry.znode_name, new_part_name, out_postpone_reason, state_lock)) return false; } + + String covering_part = virtual_parts.getContainingPart(entry.new_part_name); + if (covering_part.empty()) + { + if (lost_forever_parts.count(entry.new_part_name)) + return false; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found in virtual parts, but it's not lost. " + "Entry {}: {}\nVirtual: {}\nLost:{}", + entry.new_part_name, entry.znode_name, entry.toString(), + fmt::join(virtual_parts.getParts(), ", "), fmt::join(lost_forever_parts, ", ")); + } + + /// NOTE: It's possible that (covering_part == entry.new_part_name), because isNotCoveredByFuturePartsImpl(...) + /// checks for future parts only, not for virtual parts. } /// Check that fetches pool is not overloaded @@ -1259,7 +1454,8 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(const Replicate for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version)) { if (!queue.future_parts.emplace(new_part_name, entry).second) - throw Exception("Tagging already tagged future part " + new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged future part {}. This is a bug. " + "It happened on attempt to execute {}: {}", new_part_name, entry->znode_name, entry->toString()); } } @@ -1277,7 +1473,8 @@ void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName(ReplicatedM return; if (!queue.future_parts.emplace(entry.actual_new_part_name, entry.shared_from_this()).second) - throw Exception("Attaching already existing future part " + entry.actual_new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attaching already existing future part {}. This is a bug. " + "It happened on attempt to execute {}: {}", entry.actual_new_part_name, entry.znode_name, entry.toString()); } @@ -1296,13 +1493,19 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version)) { if (!queue.future_parts.erase(new_part_name)) + { LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name); + assert(false); + } } if (!entry->actual_new_part_name.empty()) { if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name)) + { LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name); + assert(false); + } entry->actual_new_part_name.clear(); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 496f277d132..c1470e8ff01 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -92,8 +92,8 @@ private: using FuturePartsSet = std::map; FuturePartsSet future_parts; - /// Index of the first log entry that we didn't see yet. - Int64 log_pointer = 0; + using LostPartsSet = std::set; + LostPartsSet lost_forever_parts; /// Avoid parallel execution of queue enties, which may remove other entries from the queue. bool currently_executing_drop_or_replace_range = false; @@ -275,6 +275,7 @@ public: ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeMergeStrategyPicker & merge_strategy_picker_); ~ReplicatedMergeTreeQueue(); + void clear(); void initialize(const MergeTreeData::DataParts & parts); @@ -289,13 +290,20 @@ public: */ bool remove(zkutil::ZooKeeperPtr zookeeper, const String & part_name); + using QueueIters = std::vector; + QueueIters findEntriesByNewPartName(const String & part_name) const; + + bool markPartAsLostForever(zkutil::ZooKeeperPtr zookeeper, const String & part_name); + + void executePartIsLost(zkutil::ZooKeeperPtr zookeeper, LogEntry & entry_lost); + /** Load (initialize) a queue from ZooKeeper (/replicas/me/queue/). * If queue was not empty load() would not load duplicate records. * return true, if we update queue. */ bool load(zkutil::ZooKeeperPtr zookeeper); - bool removeFromVirtualParts(const MergeTreePartInfo & part_info); + bool removeFailedQuorumPart(const MergeTreePartInfo & part_info); /** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value. * If watch_callback is not empty, will call it when new entries appear in the log. @@ -478,6 +486,8 @@ public: /// The version of "log" node that is used to check that no new merges have appeared. int32_t getVersion() const { return merges_version; } + bool ensurePartIsLost(const String & part_name, Poco::Logger * log) const; + private: const ReplicatedMergeTreeQueue & queue; From 52e4c01aba814575d80472d18f00c1f91cf0dcc0 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Mon, 31 May 2021 00:56:27 +0300 Subject: [PATCH 267/652] Additional fixes after review --- src/Common/Config/YAMLParser.cpp | 2 +- src/Common/examples/CMakeLists.txt | 7 +++++++ .../examples/YAML_fuzzer.cpp | 17 ++++++++++------- src/Parsers/examples/CMakeLists.txt | 3 --- 4 files changed, 18 insertions(+), 11 deletions(-) rename src/{Parsers => Common}/examples/YAML_fuzzer.cpp (67%) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index 5595f61e1ee..8d758eefdc0 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -148,7 +148,7 @@ Poco::AutoPtr YAMLParser::parse(const String& path) catch (const YAML::ParserException& e) { /// yaml-cpp cannot parse the file because its contents are incorrect - throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "Unable to parse YAML configuration file {}", path, e.what()); + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "Unable to parse YAML configuration file {}, {}", path, e.what()); } catch (const YAML::BadFile&) { diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 64d28fec5c2..288fcf7f87e 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -80,3 +80,10 @@ target_link_libraries (average PRIVATE clickhouse_common_io) add_executable (shell_command_inout shell_command_inout.cpp) target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) + +set(SRCS) + +if (ENABLE_FUZZING) + add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) + target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) +endif () diff --git a/src/Parsers/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp similarity index 67% rename from src/Parsers/examples/YAML_fuzzer.cpp rename to src/Common/examples/YAML_fuzzer.cpp index 223bd89c7b2..4c8f0522d29 100644 --- a/src/Parsers/examples/YAML_fuzzer.cpp +++ b/src/Common/examples/YAML_fuzzer.cpp @@ -11,9 +11,10 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) /// build ClickHouse with YAML_fuzzer.cpp /// ./YAML_fuzzer YAML_CORPUS /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer - char* file_name = std::tmpnam(nullptr); + char buf[L_tmpnam]; + char* file_name = std::tmpnam(buf); if (file_name == nullptr) { - std::cout << "Cannot create temp file!\n"; + std::cerr << "Cannot create temp file!\n"; return 1; } std::string cur_file(file_name); @@ -21,17 +22,19 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::string input = std::string(reinterpret_cast(data), size); DB::YAMLParser parser; - std::ofstream temp_file(cur_file); - temp_file << input; - temp_file.close(); + { + std::ofstream temp_file(cur_file); + temp_file << input; + } try { DB::YAMLParser::parse(cur_file); } - catch (const DB::Exception&) + catch (...) { - std::cout << "YAMLParser exception from bad file, etc. OK\n"; + std::cerr << "YAML_fuzzer failed: " << getCurrentExceptionMessage() << std::endl; + return 1; } return 0; } diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 578e87761e1..3ff40b3cc3b 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -18,7 +18,4 @@ if (ENABLE_FUZZING) add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) - - add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) - target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) endif () From 80f07051f22827a0b0d0f01d06c60de879ea904c Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 May 2021 07:06:14 +0300 Subject: [PATCH 268/652] other path --- .../configs/dictionaries/ssd_complex_key_cache_string.xml | 4 +++- .../configs/enable_dictionaries.xml | 3 +-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml index def138a1c32..45d9d5f8e15 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml @@ -41,7 +41,9 @@ 16777216 131072 1048576 - /var/lib/clickhouse/dictionaries/radars + /tmp/dictionaries/radars + + 1 diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml index 46d148ad9b9..8a3d6704670 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml @@ -1,5 +1,4 @@ - /etc/clickhouse-server/config.d/complex_key_cache_string.xml - /etc/clickhouse-server/config.d/ssd_complex_key_cache_string.xml + /etc/clickhouse-server/config.d/*.xml From 0c2318c59bb41e58b13d3901370ea913fc10293f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 May 2021 07:07:32 +0300 Subject: [PATCH 269/652] tune runner for now --- tests/integration/ci-runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 1efc5c3e78a..20042ee596a 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -11,10 +11,10 @@ import json import csv -MAX_RETRY = 2 +MAX_RETRY = 3 NUM_WORKERS = 5 SLEEP_BETWEEN_RETRIES = 5 -PARALLEL_GROUP_SIZE = 100 +PARALLEL_GROUP_SIZE = 200 CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge" DOCKERD_LOGS_PATH = "/ClickHouse/tests/integration/dockerd.log" From 5f0373dace434404d1ed460aea78e1c5cf0cff84 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 May 2021 07:12:22 +0300 Subject: [PATCH 270/652] lost in merge --- tests/integration/test_zookeeper_config/test_password.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test_password.py b/tests/integration/test_zookeeper_config/test_password.py index b99007e67f2..c0ed4375978 100644 --- a/tests/integration/test_zookeeper_config/test_password.py +++ b/tests/integration/test_zookeeper_config/test_password.py @@ -7,10 +7,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, name="password") # TODO ACL not implemented in Keeper. -node1 = cluster.add_instance('node1', with_zookeeper=True, use_keeper=False, +node1 = cluster.add_instance('node1', with_zookeeper=True, main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_with_password.xml"]) -node2 = cluster.add_instance('node2', with_zookeeper=True, use_keeper=False, main_configs=["configs/remote_servers.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, main_configs=["configs/remote_servers.xml"]) @pytest.fixture(scope="module", autouse=True) def started_cluster(): From 79501da4207abb2ba333f0dd610aba81e2dfd4e7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 May 2021 07:17:08 +0300 Subject: [PATCH 271/652] typo --- tests/integration/test_dictionary_custom_settings/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index f6c487f073d..0d337e8c00e 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -12,7 +12,7 @@ DICTIONARY_FILES = [ ] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('node', dictionaties=DICTIONARY_FILES) +instance = cluster.add_instance('node', dictionaries=DICTIONARY_FILES) def prepare(): From 93754ca8b64a914b80990a0418080e66de681567 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Mon, 31 May 2021 09:34:10 +0300 Subject: [PATCH 272/652] Fix stylecheck --- src/Common/examples/YAML_fuzzer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp index 4c8f0522d29..b290bbc5fa8 100644 --- a/src/Common/examples/YAML_fuzzer.cpp +++ b/src/Common/examples/YAML_fuzzer.cpp @@ -13,12 +13,13 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer char buf[L_tmpnam]; char* file_name = std::tmpnam(buf); - if (file_name == nullptr) { + if (file_name == nullptr) + { std::cerr << "Cannot create temp file!\n"; return 1; } std::string cur_file(file_name); - + std::string input = std::string(reinterpret_cast(data), size); DB::YAMLParser parser; From 0035997e02ade74a11f6ee97ed2f011bae475283 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 May 2021 16:31:03 +0300 Subject: [PATCH 273/652] fix race on enqueue query --- src/Databases/DatabaseReplicated.cpp | 20 +++------- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/DatabaseReplicatedWorker.cpp | 45 ++++++++++++++++++++-- src/Databases/DatabaseReplicatedWorker.h | 3 ++ 4 files changed, 51 insertions(+), 19 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d36fe45f748..2630c896a7a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -273,19 +273,11 @@ bool DatabaseReplicated::createDatabaseNodesInZooKeeper(const zkutil::ZooKeeperP __builtin_unreachable(); } -void DatabaseReplicated::createEmptyLogEntry(Coordination::Requests & ops, const ZooKeeperPtr & current_zookeeper) +void DatabaseReplicated::createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper) { /// On replica creation add empty entry to log. Can be used to trigger some actions on other replicas (e.g. update cluster info). DDLLogEntry entry{}; - - String query_path_prefix = zookeeper_path + "/log/query-"; - String counter_prefix = zookeeper_path + "/counter/cnt-"; - String counter_path = current_zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential); - String query_path = query_path_prefix + counter_path.substr(counter_prefix.size()); - - ops.emplace_back(zkutil::makeCreateRequest(query_path, entry.toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(query_path + "/committed", getFullReplicaName(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1)); + DatabaseReplicatedDDLWorker::enqueueQueryImpl(current_zookeeper, entry, this, true); } void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper) @@ -296,8 +288,8 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); - createEmptyLogEntry(ops, current_zookeeper); current_zookeeper->multi(ops); + createEmptyLogEntry(current_zookeeper); } void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) @@ -659,10 +651,8 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node void DatabaseReplicated::drop(ContextPtr context_) { auto current_zookeeper = getZooKeeper(); - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(replica_path, DROPPED_MARK, -1)); - createEmptyLogEntry(ops, current_zookeeper); - current_zookeeper->multi(ops); + current_zookeeper->set(replica_path, DROPPED_MARK, -1); + createEmptyLogEntry(current_zookeeper); DatabaseAtomic::drop(context_); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index b930d27c19b..e6c3fb00eb6 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -78,7 +78,7 @@ private: ClusterPtr getClusterImpl() const; void setCluster(ClusterPtr && new_cluster); - void createEmptyLogEntry(Coordination::Requests & ops, const ZooKeeperPtr & current_zookeeper); + void createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper); String zookeeper_path; String shard_name; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 9ae4d026bf0..548f7f6f882 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -69,25 +70,63 @@ void DatabaseReplicatedDDLWorker::initializeReplication() String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) { auto zookeeper = getAndSetZooKeeper(); - const String query_path_prefix = queue_dir + "/query-"; + return enqueueQueryImpl(zookeeper, entry, database); +} + +String DatabaseReplicatedDDLWorker::enqueueQueryImpl(const ZooKeeperPtr & zookeeper, DDLLogEntry & entry, + DatabaseReplicated * const database, bool committed) +{ + const String query_path_prefix = database->zookeeper_path + "/log/query-"; /// We cannot create sequential node and it's ephemeral child in a single transaction, so allocate sequential number another way String counter_prefix = database->zookeeper_path + "/counter/cnt-"; - String counter_path = zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential); + String counter_lock_path = database->zookeeper_path + "/counter_lock"; + + String counter_path; + size_t iters = 1000; + while (--iters) + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(counter_lock_path, database->getFullReplicaName(), zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest(counter_prefix, "", zkutil::CreateMode::EphemeralSequential)); + Coordination::Responses res; + + Coordination::Error code = zookeeper->tryMulti(ops, res); + if (code == Coordination::Error::ZOK) + { + counter_path = dynamic_cast(*res.back()).path_created; + break; + } + else if (code != Coordination::Error::ZNODEEXISTS) + zkutil::KeeperMultiException::check(code, ops, res); + } + + if (iters == 0) + throw Exception(ErrorCodes::UNFINISHED, + "Cannot enqueue query, because some replica are trying to enqueue another query. " + "It may happen on high queries rate or, in rare cases, after connection loss. Client should retry."); + String node_path = query_path_prefix + counter_path.substr(counter_prefix.size()); + /// Now create task in queue Coordination::Requests ops; /// Query is not committed yet, but we have to write it into log to avoid reordering ops.emplace_back(zkutil::makeCreateRequest(node_path, entry.toString(), zkutil::CreateMode::Persistent)); /// '/try' will be replaced with '/committed' or will be removed due to expired session or other error - ops.emplace_back(zkutil::makeCreateRequest(node_path + "/try", database->getFullReplicaName(), zkutil::CreateMode::Ephemeral)); + if (committed) + ops.emplace_back(zkutil::makeCreateRequest(node_path + "/committed", database->getFullReplicaName(), zkutil::CreateMode::Persistent)); + else + ops.emplace_back(zkutil::makeCreateRequest(node_path + "/try", database->getFullReplicaName(), zkutil::CreateMode::Ephemeral)); /// We don't need it anymore ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1)); + /// Unlock counters + ops.emplace_back(zkutil::makeRemoveRequest(counter_lock_path, -1)); /// Create status dirs ops.emplace_back(zkutil::makeCreateRequest(node_path + "/active", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(node_path + "/finished", "", zkutil::CreateMode::Persistent)); zookeeper->multi(ops); + return node_path; } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 16ad100b81a..4020906f9b2 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -29,6 +29,9 @@ public: void shutdown() override; + static String enqueueQueryImpl(const ZooKeeperPtr & zookeeper, DDLLogEntry & entry, + DatabaseReplicated * const database, bool committed = false); + private: bool initializeMainThread() override; void initializeReplication(); From f60d971637882d799ee74cf0ccbac839c61221f2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 31 May 2021 16:45:47 +0300 Subject: [PATCH 274/652] cleanup --- programs/client/QueryFuzzer.cpp | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index aa12468cde0..5921f96e74e 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -349,19 +349,9 @@ void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def) // The offsets are fuzzed normally through 'children'. def.frame_begin_offset = std::make_shared(getRandomField(0)); - def.children.push_back(def.frame_begin_offset); } else { - // Don't keep the offset if it is not used, because it will - // acquire random mutations that will surely make it invalid. - const auto old_size = def.children.size(); - def.children.erase( - std::remove(def.children.begin(), def.children.end(), - def.frame_begin_offset), - def.children.end()); - assert(def.children.size() == old_size - 1 - || def.frame_begin_offset == nullptr); def.frame_begin_offset = nullptr; } break; @@ -377,14 +367,9 @@ void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def) { def.frame_end_offset = std::make_shared(getRandomField(0)); - def.children.push_back(def.frame_end_offset); } else { - def.children.erase( - std::remove(def.children.begin(), def.children.end(), - def.frame_end_offset), - def.children.end()); def.frame_end_offset = nullptr; } break; From 5bd6183e67377b3627e046f9754a102bb2c089d0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 20 Apr 2021 16:31:24 +0300 Subject: [PATCH 275/652] save --- tests/integration/helpers/cluster.py | 5 +- .../configs_taxi/conf.d/clusters.xml | 28 +++ .../configs_taxi/conf.d/ddl.xml | 6 + .../configs_taxi/config-copier.xml | 28 +++ .../configs_taxi/users.xml | 34 +++ .../test_cluster_copier/data/first.csv | 100 +++++++++ .../test_cluster_copier/prepare_taxi_data.py | 57 +++++ .../test_cluster_copier/task_taxi_data.xml | 43 ++++ .../test_cluster_copier/test_taxi_data.py | 196 ++++++++++++++++++ 9 files changed, 495 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_cluster_copier/configs_taxi/conf.d/clusters.xml create mode 100644 tests/integration/test_cluster_copier/configs_taxi/conf.d/ddl.xml create mode 100644 tests/integration/test_cluster_copier/configs_taxi/config-copier.xml create mode 100644 tests/integration/test_cluster_copier/configs_taxi/users.xml create mode 100644 tests/integration/test_cluster_copier/data/first.csv create mode 100644 tests/integration/test_cluster_copier/prepare_taxi_data.py create mode 100644 tests/integration/test_cluster_copier/task_taxi_data.xml create mode 100644 tests/integration/test_cluster_copier/test_taxi_data.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6287064b616..2fa4aa4c7a1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -126,7 +126,7 @@ class ClickHouseCluster: """ def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, - odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, + odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): for param in list(os.environ.keys()): print("ENV %40s %s" % (param, os.environ[param])) @@ -1524,7 +1524,7 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace("{main_config_file}", self.main_config_name) - + print("Entrypoint cmd: {}".format(entrypoint_cmd)) networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" @@ -1548,6 +1548,7 @@ class ClickHouseInstance: odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" library_bridge_volume = "- " + self.library_bridge_bin_path + ":/usr/share/clickhouse-library-bridge_fresh" + with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, diff --git a/tests/integration/test_cluster_copier/configs_taxi/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs_taxi/conf.d/clusters.xml new file mode 100644 index 00000000000..6993a7ad7fd --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_taxi/conf.d/clusters.xml @@ -0,0 +1,28 @@ + + + + + + false + + first + 9000 + + + + false + + second + 9000 + + + + false + + third + 9000 + + + + + diff --git a/tests/integration/test_cluster_copier/configs_taxi/conf.d/ddl.xml b/tests/integration/test_cluster_copier/configs_taxi/conf.d/ddl.xml new file mode 100644 index 00000000000..4bff11fb693 --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_taxi/conf.d/ddl.xml @@ -0,0 +1,6 @@ + + + + /clickhouse/task_queue/ddl + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml b/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml new file mode 100644 index 00000000000..f5784e732dc --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml @@ -0,0 +1,28 @@ + + + + trace + /var/log/clickhouse-server/copier/log.log + /var/log/clickhouse-server/copier/log.err.log + 1000M + 10 + /var/log/clickhouse-server/copier/stderr.log + /var/log/clickhouse-server/copier/stdout.log + + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 2000 + + diff --git a/tests/integration/test_cluster_copier/configs_taxi/users.xml b/tests/integration/test_cluster_copier/configs_taxi/users.xml new file mode 100644 index 00000000000..d27ca56eec7 --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_taxi/users.xml @@ -0,0 +1,34 @@ + + + + + 1 + + 5 + + + + + + + + ::/0 + + default + default + + + 12345678 + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_cluster_copier/data/first.csv b/tests/integration/test_cluster_copier/data/first.csv new file mode 100644 index 00000000000..5efc10c0ca2 --- /dev/null +++ b/tests/integration/test_cluster_copier/data/first.csv @@ -0,0 +1,100 @@ +1,2020-12-01 00:41:19,2020-12-01 00:49:45,1,1.60,1,N,140,263,1,8,3,0.5,2.95,0,0.3,14.75,2.5 +2,2020-12-01 00:33:40,2020-12-01 01:00:35,1,16.74,2,N,132,164,1,52,0,0.5,2.5,6.12,0.3,63.92,2.5 +2,2020-12-01 00:02:15,2020-12-01 00:13:09,1,4.16,1,N,238,48,1,14,0.5,0.5,1,0,0.3,18.8,2.5 +2,2020-12-01 00:37:42,2020-12-01 00:45:11,1,2.22,1,N,238,41,2,8.5,0.5,0.5,0,0,0.3,9.8,0 +1,2020-12-01 00:27:47,2020-12-01 00:45:40,0,8.40,1,N,138,137,1,25,3,0.5,6,6.12,0.3,40.92,2.5 +2,2020-12-01 00:40:47,2020-12-01 00:57:03,1,6.44,1,N,132,191,1,19.5,0.5,0.5,4.16,0,0.3,24.96,0 +2,2020-12-01 00:01:42,2020-12-01 00:06:06,1,.99,1,N,234,137,1,5.5,0.5,0.5,1.86,0,0.3,11.16,2.5 +2,2020-12-01 00:58:24,2020-12-01 01:36:14,2,11.81,1,N,261,7,1,36.5,0.5,0.5,1,0,0.3,41.3,2.5 +1,2020-12-01 00:08:15,2020-12-01 00:16:04,2,2.70,1,N,237,107,1,9.5,3,0.5,2.65,0,0.3,15.95,2.5 +2,2020-12-01 00:04:21,2020-12-01 00:29:00,1,6.28,1,N,41,68,2,23,0.5,0.5,0,0,0.3,26.8,2.5 +2,2020-12-01 00:22:03,2020-12-01 00:52:55,1,18.93,2,N,132,211,1,52,0,0.5,12.28,6.12,0.3,73.7,2.5 +2,2020-12-01 00:46:31,2020-12-01 00:53:49,5,2.75,1,N,249,230,2,9.5,0.5,0.5,0,0,0.3,13.3,2.5 +2,2020-12-01 00:51:30,2020-12-01 01:26:52,2,21.80,2,N,132,13,1,52,0,0.5,11.06,0,0.3,66.36,2.5 +1,2020-12-01 00:14:34,2020-12-01 00:31:04,1,7.60,1,N,140,65,1,23,3,0.5,5.35,0,0.3,32.15,2.5 +1,2020-12-01 00:11:02,2020-12-01 00:17:34,1,1.70,1,N,239,141,2,7.5,3,0.5,0,0,0.3,11.3,2.5 +1,2020-12-01 00:54:55,2020-12-01 00:57:09,1,.50,1,N,263,141,2,4,3,0.5,0,0,0.3,7.8,2.5 +1,2020-12-01 00:11:22,2020-12-01 00:40:36,1,21.00,2,N,132,231,1,52,2.5,0.5,16.55,0,0.3,71.85,2.5 +2,2020-11-30 23:59:22,2020-12-01 00:05:51,3,.81,1,N,50,48,1,5.5,0.5,0.5,1.88,0,0.3,11.18,2.5 +2,2020-12-01 00:24:34,2020-12-01 00:29:59,4,1.45,1,N,48,162,1,6.5,0.5,0.5,2.06,0,0.3,12.36,2.5 +1,2020-12-01 00:53:58,2020-12-01 00:54:06,1,.00,1,N,132,132,3,2.5,0.5,0.5,0,0,0.3,3.8,0 +2,2020-12-01 00:20:44,2020-12-01 00:32:48,5,3.78,1,N,140,42,2,13.5,0.5,0.5,0,0,0.3,17.3,2.5 +2,2020-12-01 00:42:13,2020-12-01 00:46:05,1,1.12,1,N,138,129,2,5.5,0.5,0.5,0,0,0.3,6.8,0 +2,2020-12-01 00:02:45,2020-12-01 00:11:35,1,1.63,1,N,137,48,1,8,0.5,0.5,2.36,0,0.3,14.16,2.5 +2,2020-12-01 00:14:38,2020-12-01 00:38:53,3,7.01,1,N,137,129,2,23.5,0.5,0.5,0,0,0.3,27.3,2.5 +2,2020-12-01 00:21:33,2020-12-01 00:33:44,1,5.31,1,N,141,69,1,16,0.5,0.5,0,0,0.3,19.8,2.5 +2,2020-12-01 00:34:26,2020-12-01 00:50:02,1,7.92,1,N,138,137,1,23,0.5,0.5,6.58,6.12,0.3,39.5,2.5 +1,2020-12-01 00:52:58,2020-12-01 00:54:28,1,.70,1,N,162,170,2,4,3,0.5,0,0,0.3,7.8,2.5 +2,2020-12-01 00:30:32,2020-12-01 00:39:11,2,2.33,1,N,161,140,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 +2,2020-11-30 23:56:50,2020-12-01 00:03:37,1,1.72,1,N,161,107,2,7.5,0.5,0.5,0,0,0.3,11.3,2.5 +2,2020-12-01 00:29:36,2020-12-01 00:31:19,1,.52,1,N,237,141,1,3.5,0.5,0.5,1.82,0,0.3,9.12,2.5 +1,2020-12-01 00:20:26,2020-12-01 00:23:06,0,1.10,1,N,90,230,2,5,3,0.5,0,0,0.3,8.8,2.5 +2,2020-12-01 00:19:55,2020-12-01 00:28:26,1,2.22,1,N,230,239,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 +2,2020-12-01 00:25:05,2020-12-01 00:47:44,1,18.75,2,N,132,262,1,52,0,0.5,20,6.12,0.3,81.42,2.5 +2,2020-12-01 00:12:55,2020-12-01 00:31:15,1,8.06,1,N,75,88,1,25,0.5,0.5,5.76,0,0.3,34.56,2.5 +1,2020-12-01 00:57:19,2020-12-01 01:10:52,1,3.70,1,N,148,49,2,13.5,3,0.5,0,0,0.3,17.3,2.5 +2,2020-12-01 00:00:41,2020-12-01 00:06:19,1,2.52,1,N,140,137,2,9,0.5,0.5,0,0,0.3,12.8,2.5 +2,2020-12-01 00:14:43,2020-12-01 00:38:13,2,10.05,1,N,132,225,1,30,0.5,0.5,6.26,0,0.3,37.56,0 +2,2020-12-01 00:14:08,2020-12-01 00:21:20,1,2.41,1,N,48,107,2,9,0.5,0.5,0,0,0.3,12.8,2.5 +2,2020-12-01 00:31:55,2020-12-01 00:36:09,1,1.37,1,N,68,230,2,6,0.5,0.5,0,0,0.3,9.8,2.5 +1,2020-12-01 00:08:49,2020-12-01 00:16:59,1,2.40,1,N,263,238,1,9,3.5,0.5,2.65,0,0.3,15.95,2.5 +1,2020-12-01 00:17:41,2020-12-01 00:24:29,1,2.40,1,N,233,236,1,9,3,0.5,2.55,0,0.3,15.35,2.5 +2,2020-12-01 00:19:22,2020-12-01 00:46:33,1,17.78,2,N,132,229,1,52,0,0.5,5,0,0.3,60.3,2.5 +1,2020-12-01 00:48:48,2020-12-01 01:05:24,1,4.90,1,N,170,151,2,16.5,3,0.5,0,0,0.3,20.3,2.5 +1,2020-12-01 00:06:54,2020-12-01 00:12:12,1,1.70,1,N,107,229,1,7,3,0.5,2.15,0,0.3,12.95,2.5 +1,2020-12-01 00:13:41,2020-12-01 00:19:20,1,2.00,1,N,229,263,1,7.5,3,0.5,2.25,0,0.3,13.55,2.5 +2,2020-12-01 00:01:54,2020-12-01 00:12:12,1,1.93,1,N,236,143,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 +1,2020-12-01 00:04:17,2020-12-01 00:04:32,1,.00,1,N,42,42,2,2.5,0.5,0.5,0,0,0.3,3.8,0 +1,2020-12-01 00:39:28,2020-12-01 00:47:45,1,2.80,1,N,10,197,1,10.5,0.5,0.5,7,0,0.3,18.8,0 +2,2020-12-01 00:02:55,2020-12-01 00:12:05,1,2.20,1,N,237,143,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 +2,2020-12-01 00:00:51,2020-12-01 00:13:34,2,5.10,1,N,137,80,1,16.5,0.5,0.5,5.08,0,0.3,25.38,2.5 +2,2020-12-01 00:24:06,2020-12-01 00:48:07,2,17.95,2,N,132,114,1,52,0,0.5,16.59,0,0.3,71.89,2.5 +2,2020-12-01 00:08:12,2020-12-01 00:30:28,1,6.77,1,N,237,102,1,22.5,0.5,0.5,5.26,0,0.3,31.56,2.5 +2,2020-12-01 00:08:51,2020-12-01 00:25:34,1,4.67,1,N,237,223,2,16.5,0.5,0.5,0,0,0.3,20.3,2.5 +2,2020-12-01 00:46:04,2020-12-01 01:03:51,1,8.76,1,N,132,95,2,26,0.5,0.5,0,0,0.3,27.3,0 +1,2020-12-01 00:38:59,2020-12-01 01:00:47,1,6.90,1,N,137,37,1,22.5,3,0.5,2,0,0.3,28.3,2.5 +2,2020-11-30 23:52:54,2020-11-30 23:54:54,1,.70,1,N,163,50,2,4,0.5,0.5,0,0,0.3,7.8,2.5 +2,2020-12-01 00:05:53,2020-12-01 00:08:38,1,.70,1,N,48,50,1,4.5,0.5,0.5,1,0,0.3,9.3,2.5 +2,2020-12-01 00:44:36,2020-12-01 00:46:47,1,1.00,1,N,249,90,1,5,0.5,0.5,1.76,0,0.3,10.56,2.5 +2,2020-12-01 00:41:19,2020-12-01 01:03:03,1,10.82,1,N,138,142,1,32,0.5,0.5,8.38,6.12,0.3,50.3,2.5 +2,2020-12-01 00:50:29,2020-12-01 01:03:28,1,3.75,1,N,237,211,2,13.5,0.5,0.5,0,0,0.3,17.3,2.5 +2,2020-12-01 00:48:34,2020-12-01 01:03:17,1,3.44,1,N,211,52,1,14,0.5,0.5,3.56,0,0.3,21.36,2.5 +2,2020-12-01 00:04:05,2020-12-01 00:10:59,1,2.38,1,N,142,68,1,8.5,0.5,0.5,2,0,0.3,14.3,2.5 +2,2020-12-01 00:35:23,2020-12-01 00:40:41,1,1.35,1,N,264,142,1,6,0.5,0.5,1.96,0,0.3,11.76,2.5 +2,2020-12-01 00:12:21,2020-12-01 00:20:05,6,.55,1,N,41,74,2,5,0.5,0.5,0,0,0.3,6.3,0 +2,2020-12-01 00:09:09,2020-12-01 00:25:10,4,4.09,1,N,137,239,1,15,0.5,0.5,3.76,0,0.3,22.56,2.5 +2,2020-12-01 00:40:06,2020-12-01 01:05:42,6,18.78,2,N,132,114,1,52,0,0.5,13.82,0,0.3,69.12,2.5 +2,2020-12-01 00:20:47,2020-12-01 00:33:57,1,4.96,1,N,75,7,1,15.5,0.5,0.5,0,0,0.3,19.3,2.5 +2,2020-12-01 00:20:21,2020-12-01 00:34:41,1,5.13,1,N,170,260,2,16.5,0.5,0.5,0,0,0.3,20.3,2.5 +2,2020-12-01 00:16:42,2020-12-01 00:48:28,1,10.50,1,N,138,114,1,31.5,0.5,0.5,10.59,0,0.3,45.89,2.5 +1,2020-12-01 00:43:56,2020-12-01 00:59:45,1,6.90,1,N,132,197,2,21.5,0.5,0.5,0,0,0.3,22.8,0 +1,2020-12-01 00:07:50,2020-12-01 00:13:30,1,2.50,1,N,233,263,2,8.5,3,0.5,0,0,0.3,12.3,2.5 +1,2020-12-01 00:56:41,2020-12-01 01:02:57,1,1.60,1,N,230,141,1,7.5,3,0.5,2.25,0,0.3,13.55,2.5 +2,2020-12-01 00:54:45,2020-12-01 01:01:28,4,3.20,1,N,132,10,1,10.5,0.5,0.5,2.95,0,0.3,14.75,0 +1,2020-12-01 00:17:14,2020-12-01 00:24:26,1,1.50,1,N,166,238,1,7.5,3,0.5,1,0,0.3,12.3,2.5 +2,2020-12-01 00:37:15,2020-12-01 00:42:39,1,1.39,1,N,229,262,2,6.5,0.5,0.5,0,0,0.3,10.3,2.5 +2,2020-12-01 00:47:28,2020-12-01 00:55:37,1,2.94,1,N,140,107,2,10.5,0.5,0.5,0,0,0.3,14.3,2.5 +1,2020-12-01 00:43:06,2020-12-01 00:52:45,2,3.50,1,N,132,10,1,12.5,0.5,0.5,0,0,0.3,13.8,0 +2,2020-12-01 00:33:32,2020-12-01 00:38:07,1,.77,1,N,68,234,1,5.5,0.5,0.5,1.5,0,0.3,10.8,2.5 +2,2020-11-30 23:59:22,2020-12-01 00:13:53,2,4.48,1,N,87,68,1,15,0.5,0.5,2,0,0.3,20.8,2.5 +2,2020-12-01 00:47:29,2020-12-01 00:51:53,2,1.43,1,N,162,107,1,6,0.5,0.5,1.96,0,0.3,11.76,2.5 +2,2020-12-01 00:31:04,2020-12-01 00:38:21,1,2.25,1,N,263,74,2,8.5,0.5,0.5,0,0,0.3,12.3,2.5 +2,2020-11-30 23:49:37,2020-11-30 23:59:04,1,2.56,1,N,262,151,1,10,0.5,0.5,2,0,0.3,15.8,2.5 +2,2020-12-01 00:04:47,2020-12-01 00:08:47,1,1.05,1,N,238,24,1,5.5,0.5,0.5,2.04,0,0.3,8.84,0 +2,2020-12-01 00:21:56,2020-12-01 00:39:04,1,8.46,1,N,231,193,2,24.5,0.5,0.5,0,0,0.3,28.3,2.5 +1,2020-12-01 00:31:03,2020-12-01 00:39:47,1,1.70,1,N,249,231,1,8.5,3,0.5,2.45,0,0.3,14.75,2.5 +1,2020-12-01 00:44:53,2020-12-01 00:56:32,1,4.20,1,N,125,142,1,13.5,3,0.5,3.45,0,0.3,20.75,2.5 +2,2020-12-01 00:23:04,2020-12-01 00:45:51,3,17.76,2,N,132,162,1,52,0,0.5,10.8,6.12,0.3,72.22,2.5 +1,2020-12-01 00:41:35,2020-12-01 00:52:03,1,2.40,1,N,161,143,1,10,3,0.5,2.75,0,0.3,16.55,2.5 +1,2020-12-01 00:57:28,2020-12-01 01:16:25,1,5.10,1,N,143,168,2,18,3,0.5,0,0,0.3,21.8,2.5 +2,2020-12-01 00:12:06,2020-12-01 00:37:49,1,11.21,1,N,48,89,4,-32,-0.5,-0.5,0,0,-0.3,-35.8,-2.5 +2,2020-12-01 00:12:06,2020-12-01 00:37:49,1,11.21,1,N,48,188,2,32,0.5,0.5,0,0,0.3,35.8,2.5 +2,2020-12-01 00:43:56,2020-12-01 01:05:51,2,16.77,2,N,132,170,1,52,0,0.5,10,6.12,0.3,71.42,2.5 +2,2020-12-01 00:24:15,2020-12-01 00:27:34,1,.80,1,N,234,90,1,5,0.5,0.5,1.76,0,0.3,10.56,2.5 +2,2020-12-01 00:53:14,2020-12-01 00:59:37,1,1.52,1,N,230,143,1,7.5,0.5,0.5,2.26,0,0.3,13.56,2.5 +2,2020-12-01 00:17:15,2020-12-01 00:38:57,1,3.56,1,N,68,232,2,16.5,0.5,0.5,0,0,0.3,20.3,2.5 +2,2020-12-01 00:45:58,2020-12-01 00:58:43,1,3.83,1,N,79,48,1,13.5,0.5,0.5,2.7,0,0.3,20,2.5 +1,2020-12-01 00:00:26,2020-12-01 00:07:25,1,2.90,1,N,137,262,1,9.5,3,0.5,2.66,0,0.3,15.96,2.5 +1,2020-12-01 00:03:45,2020-12-01 00:20:00,1,6.90,1,N,170,168,2,20.5,3,0.5,0,0,0.3,24.3,2.5 +1,2020-12-01 00:06:04,2020-12-01 00:23:52,1,10.30,1,N,138,162,2,30,3,0.5,0,6.12,0.3,39.92,2.5 +2,2020-11-30 23:36:21,2020-11-30 23:57:45,1,10.07,1,N,138,142,1,29.5,0.5,0.5,7.88,6.12,0.3,47.3,2.5 diff --git a/tests/integration/test_cluster_copier/prepare_taxi_data.py b/tests/integration/test_cluster_copier/prepare_taxi_data.py new file mode 100644 index 00000000000..7c55147273b --- /dev/null +++ b/tests/integration/test_cluster_copier/prepare_taxi_data.py @@ -0,0 +1,57 @@ +import os +import sys +import logging +import subprocess + +logging.basicConfig(stream=sys.stdout, level=logging.DEBUG) + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) + + +FIRST_DATA = [ +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-12.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-11.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-10.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-09.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-08.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-07.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-06.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-05.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-04.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-03.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-02.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-01.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-12.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-11.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-10.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-09.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-08.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-07.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-06.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-05.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-04.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-03.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-02.csv", +"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-01.csv", +] + +def prepare_data(): + first_path = os.path.join(CURRENT_TEST_DIR, "data/first.csv") + # if os.path.exists(first_path): + # return + + # Create file + with open(first_path, 'w') as _: + pass + + print("{} created".format(first_path)) + + for url in FIRST_DATA[:1]: + print(url) + subprocess.call("wget -O - {} 2> /dev/null | tail -n +3 | head -n 100 >> {}".format(url, first_path), shell = True) + logging.info("Successfully downloaded data from {}".format(url)) + + +if __name__ == "__main__": + prepare_data() \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/task_taxi_data.xml b/tests/integration/test_cluster_copier/task_taxi_data.xml new file mode 100644 index 00000000000..18a7d19f68d --- /dev/null +++ b/tests/integration/test_cluster_copier/task_taxi_data.xml @@ -0,0 +1,43 @@ + + + + + + false + + first + 9000 + + + + false + + second + 9000 + + + + false + + third + 9000 + + + + + + 2 + + + + events + dailyhistory + yellow_tripdata + events + monthlyhistory + yellow_tripdata + Engine=ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (pickup_location_id, toYYYYMM(tpep_pickup_datetime)) + rand() + + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test_taxi_data.py b/tests/integration/test_cluster_copier/test_taxi_data.py new file mode 100644 index 00000000000..e41a74c6afd --- /dev/null +++ b/tests/integration/test_cluster_copier/test_taxi_data.py @@ -0,0 +1,196 @@ +import os +import sys +import time +import logging +import subprocess +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +import docker + +logging.basicConfig( + stream=sys.stdout, + level=logging.DEBUG, + format='%(asctime)s.%(msecs)03d %(levelname)s %(module)s - %(funcName)s: %(message)s', + datefmt='%Y-%m-%d %H:%M:%S', +) + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) + + +COPYING_FAIL_PROBABILITY = 0.33 +MOVING_FAIL_PROBABILITY = 0.1 +cluster = None + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster = ClickHouseCluster(__file__) + + for name in ["first", "second", "third"]: + cluster.add_instance(name, + main_configs=["configs_taxi/conf.d/clusters.xml", "configs_taxi/conf.d/ddl.xml"], user_configs=["configs_taxi/users.xml"], + with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) + + cluster.start() + yield cluster + + finally: + pass + cluster.shutdown() + + +DATA_COMMANDS = [ +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-12.csv 2> /dev/null | tail -n +3 | time,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-11.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-10.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-09.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-08.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-07.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-06.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-05.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-04.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-03.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-02.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-01.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', + +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-12.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-11.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-10.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-09.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-08.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-07.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-06.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-05.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-04.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-03.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-02.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-01.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', + +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-12.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-11.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-10.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-09.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-08.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-07.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-06.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-05.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-04.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-03.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-02.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', +'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-01.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"' +] + + +class Task: + def __init__(self, cluster, use_sample_offset): + self.cluster = cluster + if use_sample_offset: + self.zk_task_path = "/clickhouse-copier/task" + else: + self.zk_task_path = "/clickhouse-copier/task" + self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_taxi_data.xml'), 'r').read() + + def start(self): + instance = cluster.instances['first'] + + # daily partition database + instance.query("CREATE DATABASE dailyhistory on cluster events;") + instance.query("""CREATE TABLE dailyhistory.yellow_tripdata_staging ON CLUSTER events + ( + id UUID DEFAULT generateUUIDv4(), vendor_id String, tpep_pickup_datetime DateTime('UTC'), tpep_dropoff_datetime DateTime('UTC'), + passenger_count Nullable(Float64), trip_distance String, pickup_longitude Float64, pickup_latitude Float64, + rate_code_id String, store_and_fwd_flag String, dropoff_longitude Float64, dropoff_latitude Float64, + payment_type String, fare_amount String, extra String, mta_tax String, tip_amount String, tolls_amount String, + improvement_surcharge String, total_amount String, pickup_location_id String, dropoff_location_id String, congestion_surcharge String, + junk1 String, junk2 String + ) + Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (toYYYYMMDD(tpep_pickup_datetime))""") + instance.query("CREATE TABLE dailyhistory.yellow_tripdata ON CLUSTER events AS dailyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'dailyhistory', yellow_tripdata_staging, rand());") + + # monthly partition database + instance.query("create database monthlyhistory on cluster events;") + instance.query("""CREATE TABLE monthlyhistory.yellow_tripdata_staging ON CLUSTER events + ( + id UUID DEFAULT generateUUIDv4(), vendor_id String, tpep_pickup_datetime DateTime('UTC'), tpep_dropoff_datetime DateTime('UTC'), + passenger_count Nullable(Float64), trip_distance String, pickup_longitude Float64, pickup_latitude Float64, rate_code_id String, + store_and_fwd_flag String, dropoff_longitude Float64, dropoff_latitude Float64, payment_type String, fare_amount String, + extra String, mta_tax String, tip_amount String, tolls_amount String, improvement_surcharge String, total_amount String, + pickup_location_id String, dropoff_location_id String, congestion_surcharge String, junk1 String, junk2 String + ) + Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (pickup_location_id, toYYYYMM(tpep_pickup_datetime))""") + instance.query("CREATE TABLE monthlyhistory.yellow_tripdata ON CLUSTER events AS monthlyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'monthlyhistory', yellow_tripdata_staging, rand());") + + + logging.info("Inserting in container") + first_query = """INSERT INTO dailyhistory.yellow_tripdata( + vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance, + rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type, + fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,congestion_surcharge) FORMAT CSV""" + instance.exec_in_container(['bash', '-c', 'cat /usr/share/clickhouse-external-data/first.csv | /usr/bin/clickhouse client --query="{}"'.format(first_query)], privileged=True) + logging.info("Insert completed") + + + def check(self): + pass + + +def execute_task(task, cmd_options): + task.start() + + zk = cluster.get_kazoo_client('zoo1') + print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) + + # Run cluster-copier processes on each node + docker_api = docker.from_env().api + copiers_exec_ids = [] + + cmd = ['/usr/bin/clickhouse', 'copier', + '--config', '/etc/clickhouse-server/config-copier.xml', + '--task-path', task.zk_task_path, + '--task-file', os.path.join(CURRENT_TEST_DIR, 'task_taxi_data.py'), + '--task-upload-force', ' ', + '--base-dir', '/var/log/clickhouse-server/copier'] + cmd += cmd_options + + print(cmd) + + for instance_name, instance in cluster.instances.items(): + instance = cluster.instances[instance_name] + container = instance.get_docker_handle() + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_taxi/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + logging.info("Copied copier config to {}".format(instance.name)) + exec_id = docker_api.exec_create(container.id, cmd, stderr=True) + output = docker_api.exec_start(exec_id).decode('utf8') + logging.info(output) + copiers_exec_ids.append(exec_id) + logging.info("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) + + # time.sleep(1000) + + # Wait for copiers stopping and check their return codes + for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + while True: + res = docker_api.exec_inspect(exec_id) + if not res['Running']: + break + time.sleep(1) + + assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) + + try: + task.check() + finally: + zk.delete(task.zk_task_path, recursive=True) + + +# Tests + +def test1(started_cluster): + execute_task(Task(started_cluster, False)) + From 2017d2f91814b204905c3929d47746f67bb5ec24 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 20 Apr 2021 21:52:24 +0300 Subject: [PATCH 276/652] save --- programs/copier/ClusterCopierApp.cpp | 3 ++ .../configs_taxi/users.xml | 2 -- .../test_cluster_copier/test_taxi_data.py | 30 ++++++++++++------- 3 files changed, 22 insertions(+), 13 deletions(-) diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index d3fff616b65..bb3e37bba45 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -14,6 +14,7 @@ namespace DB void ClusterCopierApp::initialize(Poco::Util::Application & self) { + std::cout << "CopierApp::initialize" << std::endl; is_help = config().has("help"); if (is_help) return; @@ -104,6 +105,7 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) void ClusterCopierApp::mainImpl() { + std::cout << "ClusterCopierApp::mainImpl()" << std::endl; StatusFile status_file(process_path + "/status", StatusFile::write_full_info); ThreadStatus thread_status; @@ -134,6 +136,7 @@ void ClusterCopierApp::mainImpl() /// Initialize query scope just in case. CurrentThread::QueryScope query_scope(context); + std::cout << "Will construct copier" << std::endl; auto copier = std::make_unique(task_path, host_id, default_database, context); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); diff --git a/tests/integration/test_cluster_copier/configs_taxi/users.xml b/tests/integration/test_cluster_copier/configs_taxi/users.xml index d27ca56eec7..023598304f2 100644 --- a/tests/integration/test_cluster_copier/configs_taxi/users.xml +++ b/tests/integration/test_cluster_copier/configs_taxi/users.xml @@ -3,8 +3,6 @@ 1 - - 5 diff --git a/tests/integration/test_cluster_copier/test_taxi_data.py b/tests/integration/test_cluster_copier/test_taxi_data.py index e41a74c6afd..99ac727fbe6 100644 --- a/tests/integration/test_cluster_copier/test_taxi_data.py +++ b/tests/integration/test_cluster_copier/test_taxi_data.py @@ -88,13 +88,16 @@ DATA_COMMANDS = [ class Task: - def __init__(self, cluster, use_sample_offset): + def __init__(self, cluster): self.cluster = cluster - if use_sample_offset: - self.zk_task_path = "/clickhouse-copier/task" - else: - self.zk_task_path = "/clickhouse-copier/task" - self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_taxi_data.xml'), 'r').read() + self.zk_task_path = '/clickhouse-copier/task' + self.container_task_file = "/task_taxi_data.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_taxi_data.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + def start(self): instance = cluster.instances['first'] @@ -137,7 +140,12 @@ class Task: def check(self): - pass + for instance_name, instance in cluster.instances.items(): + instance = cluster.instances[instance_name] + a = instance.query("SELECT count() from dailyhistory.yellow_tripdata_staging") + b = instance.query("SELECT count() from monthlyhistory.yellow_tripdata_staging") + print(a, b) + assert a == b def execute_task(task, cmd_options): @@ -153,8 +161,8 @@ def execute_task(task, cmd_options): cmd = ['/usr/bin/clickhouse', 'copier', '--config', '/etc/clickhouse-server/config-copier.xml', '--task-path', task.zk_task_path, - '--task-file', os.path.join(CURRENT_TEST_DIR, 'task_taxi_data.py'), - '--task-upload-force', ' ', + '--task-file', task.container_task_file, + '--task-upload-force', 'true', '--base-dir', '/var/log/clickhouse-server/copier'] cmd += cmd_options @@ -190,7 +198,7 @@ def execute_task(task, cmd_options): # Tests - +@pytest.mark.timeout(1200) def test1(started_cluster): - execute_task(Task(started_cluster, False)) + execute_task(Task(started_cluster), []) From 90ab39476976671a8b6ce75defc94425b5c835b6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 22 Apr 2021 21:04:32 +0300 Subject: [PATCH 277/652] better --- programs/copier/ClusterCopier.cpp | 420 +++++++++--------- programs/copier/ClusterCopier.h | 12 +- programs/copier/ClusterCopierApp.cpp | 6 +- programs/copier/TaskTableAndShard.h | 2 +- src/Storages/StorageDistributed.cpp | 24 +- .../configs_taxi/config-copier.xml | 2 +- .../configs_two_nodes/conf.d/clusters.xml | 23 + .../configs_two_nodes/conf.d/ddl.xml | 6 + .../configs_two_nodes/config-copier.xml | 20 + .../configs_two_nodes/users.xml | 32 ++ .../test_cluster_copier/task_taxi_data.xml | 6 +- .../task_with_different_schema.xml | 40 ++ .../test_tables_with_different_schema.py | 175 ++++++++ .../test_cluster_copier/test_taxi_data.py | 25 +- 14 files changed, 558 insertions(+), 235 deletions(-) create mode 100644 tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml create mode 100644 tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml create mode 100644 tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml create mode 100644 tests/integration/test_cluster_copier/configs_two_nodes/users.xml create mode 100644 tests/integration/test_cluster_copier/task_with_different_schema.xml create mode 100644 tests/integration/test_cluster_copier/test_tables_with_different_schema.py diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index a60896388a0..fa6b9fe5862 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -6,7 +6,9 @@ #include #include #include - +#include +#include +#include namespace DB { @@ -20,6 +22,12 @@ namespace ErrorCodes } +std::string wrapWithColor(const std::string & value) +{ + return "\u001b[36;1m" + value + "\u001b[0m"; +} + + void ClusterCopier::init() { auto zookeeper = getContext()->getZooKeeper(); @@ -29,7 +37,7 @@ void ClusterCopier::init() if (response.error != Coordination::Error::ZOK) return; UInt64 version = ++task_description_version; - LOG_DEBUG(log, "Task description should be updated, local version {}", version); + LOG_INFO(log, "Task description should be updated, local version {}", version); }; task_description_path = task_zookeeper_path + "/description"; @@ -50,7 +58,7 @@ void ClusterCopier::init() task_table.initShards(task_cluster->random_engine); } - LOG_DEBUG(log, "Will process {} table tasks", task_cluster->table_tasks.size()); + LOG_INFO(log, "Will process {} table tasks", task_cluster->table_tasks.size()); /// Do not initialize tables, will make deferred initialization in process() @@ -138,7 +146,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, { if (!task_table.enabled_partitions_set.count(partition_name)) { - LOG_DEBUG(log, "Partition {} will not be processed, since it is not in enabled_partitions of {}", partition_name, task_table.table_id); + LOG_INFO(log, "Partition {} will not be processed, since it is not in enabled_partitions of {}", partition_name, task_table.table_id); } } } @@ -173,7 +181,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, LOG_WARNING(log, "There are no {} partitions from enabled_partitions in shard {} :{}", missing_partitions.size(), task_shard->getDescription(), ss.str()); } - LOG_DEBUG(log, "Will copy {} partitions from shard {}", task_shard->partition_tasks.size(), task_shard->getDescription()); + LOG_INFO(log, "Will copy {} partitions from shard {}", task_shard->partition_tasks.size(), task_shard->getDescription()); } void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads) @@ -189,9 +197,10 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, discoverShardPartitions(timeouts, task_shard); }); - LOG_DEBUG(log, "Waiting for {} setup jobs", thread_pool.active()); + LOG_INFO(log, "Waiting for {} setup jobs", thread_pool.active()); thread_pool.wait(); } + std::cout << "discoverTablePartitions finished" << std::endl; } void ClusterCopier::uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) @@ -213,7 +222,7 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s if (code != Coordination::Error::ZOK && force) zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - LOG_DEBUG(log, "Task description {} uploaded to {} with result {} ({})", + LOG_INFO(log, "Task description {} uploaded to {} with result {} ({})", ((code != Coordination::Error::ZOK && !force) ? "not " : ""), local_task_description_path, code, Coordination::errorMessage(code)); } @@ -230,7 +239,7 @@ void ClusterCopier::reloadTaskDescription() if (code != Coordination::Error::ZOK) throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - LOG_DEBUG(log, "Loading description, zxid={}", task_description_current_stat.czxid); + LOG_INFO(log, "Loading description, zxid={}", task_description_current_stat.czxid); auto config = getConfigurationFromXMLString(task_config_str); /// Setup settings @@ -250,7 +259,7 @@ void ClusterCopier::updateConfigIfNeeded() if (!is_outdated_version && !is_expired_session) return; - LOG_DEBUG(log, "Updating task description"); + LOG_INFO(log, "Updating task description"); reloadTaskDescription(); task_description_current_version = version_to_update; @@ -361,7 +370,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee if (static_cast(stat.numChildren) >= task_cluster->max_workers) { - LOG_DEBUG(log, "Too many workers ({}, maximum {}). Postpone processing {}", stat.numChildren, task_cluster->max_workers, description); + LOG_INFO(log, "Too many workers ({}, maximum {}). Postpone processing {}", stat.numChildren, task_cluster->max_workers, description); if (unprioritized) current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); @@ -387,7 +396,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee /// Try to make fast retries if (num_bad_version_errors > 3) { - LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); + LOG_INFO(log, "A concurrent worker has just been added, will check free worker slots again"); std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); std::this_thread::sleep_for(random_sleep_time); num_bad_version_errors = 0; @@ -422,7 +431,7 @@ bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_tabl { bool piece_is_done = checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); if (!piece_is_done) - LOG_DEBUG(log, "Partition {} piece {} is not already done.", partition_name, piece_number); + LOG_INFO(log, "Partition {} piece {} is not already done.", partition_name, piece_number); answer &= piece_is_done; } @@ -438,7 +447,7 @@ bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_tabl bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, size_t piece_number, const TasksShard & shards_with_partition) { - LOG_DEBUG(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number); + LOG_INFO(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number); auto zookeeper = getContext()->getZooKeeper(); @@ -530,7 +539,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t inject_fault = value < move_fault_probability; } - LOG_DEBUG(log, "Try to move {} to destination table", partition_name); + LOG_INFO(log, "Try to move {} to destination table", partition_name); auto zookeeper = getContext()->getZooKeeper(); @@ -548,7 +557,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t { if (e.code == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG(log, "Someone is already moving pieces {}", current_partition_attach_is_active); + LOG_INFO(log, "Someone is already moving pieces {}", current_partition_attach_is_active); return TaskStatus::Active; } @@ -565,13 +574,13 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); if (status.state == TaskState::Finished) { - LOG_DEBUG(log, "All pieces for partition from this task {} has been successfully moved to destination table by {}", current_partition_attach_is_active, status.owner); + LOG_INFO(log, "All pieces for partition from this task {} has been successfully moved to destination table by {}", current_partition_attach_is_active, status.owner); return TaskStatus::Finished; } /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. /// Initialize DROP PARTITION - LOG_DEBUG(log, "Moving piece for partition {} has not been successfully finished by {}. Will try to move by myself.", current_partition_attach_is_active, status.owner); + LOG_INFO(log, "Moving piece for partition {} has not been successfully finished by {}. Will try to move by myself.", current_partition_attach_is_active, status.owner); /// Remove is_done marker. zookeeper->remove(current_partition_attach_is_done); @@ -588,7 +597,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { - LOG_DEBUG(log, "Trying to move partition {} piece {} to original table", partition_name, toString(current_piece_number)); + LOG_INFO(log, "Trying to move partition {} piece {} to original table", partition_name, toString(current_piece_number)); ASTPtr query_alter_ast; String query_alter_ast_string; @@ -611,7 +620,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ((partition_name == "'all'") ? " ATTACH PARTITION ID " : " ATTACH PARTITION ") + partition_name + " FROM " + getQuotedTable(helping_table); - LOG_DEBUG(log, "Executing ALTER query: {}", query_alter_ast_string); + LOG_INFO(log, "Executing ALTER query: {}", query_alter_ast_string); try { @@ -620,9 +629,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t task_table.cluster_push, query_alter_ast_string, task_cluster->settings_push, - PoolMode::GET_MANY, - execution_mode, - max_successful_executions_per_shard); + execution_mode); if (settings_push.replication_alter_partitions_sync == 1) { @@ -645,7 +652,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t } catch (...) { - LOG_DEBUG(log, "Error while moving partition {} piece {} to original table", partition_name, toString(current_piece_number)); + LOG_INFO(log, "Error while moving partition {} piece {} to original table", partition_name, toString(current_piece_number)); throw; } @@ -660,20 +667,20 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + ((partition_name == "'all'") ? " PARTITION ID " : " PARTITION ") + partition_name + " DEDUPLICATE;"; - LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_deduplicate_ast_string); + LOG_INFO(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_deduplicate_ast_string); UInt64 num_nodes = executeQueryOnCluster( task_table.cluster_push, query_deduplicate_ast_string, task_cluster->settings_push, - PoolMode::GET_MANY); + ClusterExecutionMode::ON_EACH_SHARD); LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes)); } } catch (...) { - LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name); + LOG_INFO(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name); throw; } } @@ -739,6 +746,8 @@ std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_ res->children.clear(); res->set(res->columns_list, create.columns_list->clone()); res->set(res->storage, new_storage_ast->clone()); + /// Just to make it better and don't store additional flag like `is_table_created` somewhere else + res->if_not_exists = true; return res; } @@ -771,7 +780,7 @@ bool ClusterCopier::tryDropPartitionPiece( { if (e.code == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG(log, "Partition {} piece {} is cleaning now by somebody, sleep", task_partition.name, toString(current_piece_number)); + LOG_INFO(log, "Partition {} piece {} is cleaning now by somebody, sleep", task_partition.name, toString(current_piece_number)); std::this_thread::sleep_for(default_sleep_time); return false; } @@ -784,7 +793,7 @@ bool ClusterCopier::tryDropPartitionPiece( { if (stat.numChildren != 0) { - LOG_DEBUG(log, "Partition {} contains {} active workers while trying to drop it. Going to sleep.", task_partition.name, stat.numChildren); + LOG_INFO(log, "Partition {} contains {} active workers while trying to drop it. Going to sleep.", task_partition.name, stat.numChildren); std::this_thread::sleep_for(default_sleep_time); return false; } @@ -804,7 +813,7 @@ bool ClusterCopier::tryDropPartitionPiece( { if (e.code == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG(log, "Partition {} is being filled now by somebody, sleep", task_partition.name); + LOG_INFO(log, "Partition {} is being filled now by somebody, sleep", task_partition.name); return false; } @@ -842,12 +851,11 @@ bool ClusterCopier::tryDropPartitionPiece( /// It is important, DROP PARTITION must be done synchronously settings_push.replication_alter_partitions_sync = 2; - LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query); + LOG_INFO(log, "Execute distributed DROP PARTITION: {}", query); /// We have to drop partition_piece on each replica size_t num_shards = executeQueryOnCluster( cluster_push, query, settings_push, - PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO(log, "DROP PARTITION was successfully executed on {} nodes of a cluster.", num_shards); @@ -863,7 +871,7 @@ bool ClusterCopier::tryDropPartitionPiece( } else { - LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); + LOG_INFO(log, "Clean state is altered when dropping the partition, cowardly bailing"); /// clean state is stale return false; } @@ -907,7 +915,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab ++cluster_partition.total_tries; - LOG_DEBUG(log, "Processing partition {} for the whole cluster", partition_name); + LOG_INFO(log, "Processing partition {} for the whole cluster", partition_name); /// Process each source shard having current partition and copy current partition /// NOTE: shards are sorted by "distance" to current host @@ -929,7 +937,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab { const size_t number_of_splits = task_table.number_of_splits; shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, number_of_splits)); - LOG_DEBUG(log, "Discovered partition {} in shard {}", partition_name, shard->getDescription()); + LOG_INFO(log, "Discovered partition {} in shard {}", partition_name, shard->getDescription()); /// To save references in the future. auto shard_partition_it = shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; @@ -942,7 +950,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab } else { - LOG_DEBUG(log, "Found that shard {} does not contain current partition {}", shard->getDescription(), partition_name); + LOG_INFO(log, "Found that shard {} does not contain current partition {}", shard->getDescription(), partition_name); continue; } } @@ -1100,18 +1108,18 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t InterpreterCreateQuery::prepareOnClusterQuery(create, getContext(), task_table.cluster_push_name); String query = queryToString(create_query_push_ast); - LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); + LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); - if (shards != task_table.cluster_push->getShardCount()) - { - return TaskStatus::Error; - } + // if (shards != task_table.cluster_push->getShardCount()) + // { + // return TaskStatus::Error; + // } } catch (...) { @@ -1226,17 +1234,17 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto create_is_dirty_node = [&] (const CleanStateClock & clock) { if (clock.is_stale()) - LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); + LOG_INFO(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); else if (!clock.is_clean()) - LOG_DEBUG(log, "Thank you, Captain Obvious"); + LOG_INFO(log, "Thank you, Captain Obvious"); else if (clock.discovery_version) { - LOG_DEBUG(log, "Updating clean state clock"); + LOG_INFO(log, "Updating clean state clock"); zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); } else { - LOG_DEBUG(log, "Creating clean state clock"); + LOG_INFO(log, "Creating clean state clock"); zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); } }; @@ -1271,7 +1279,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// Load balancing auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); - LOG_DEBUG(log, "Processing {}", current_task_piece_status_path); + LOG_INFO(log, "Processing {}", current_task_piece_status_path); const String piece_status_path = partition_piece.getPartitionPieceShardsPath(); @@ -1282,12 +1290,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// Do not start if partition piece is dirty, try to clean it if (is_clean) { - LOG_DEBUG(log, "Partition {} piece {} appears to be clean", task_partition.name, current_piece_number); + LOG_INFO(log, "Partition {} piece {} appears to be clean", task_partition.name, current_piece_number); zookeeper->createAncestors(current_task_piece_status_path); } else { - LOG_DEBUG(log, "Partition {} piece {} is dirty, try to drop it", task_partition.name, current_piece_number); + LOG_INFO(log, "Partition {} piece {} is dirty, try to drop it", task_partition.name, current_piece_number); try { @@ -1312,7 +1320,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( { if (e.code == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG(log, "Someone is already processing {}", current_task_piece_is_active_path); + LOG_INFO(log, "Someone is already processing {}", current_task_piece_is_active_path); return TaskStatus::Active; } @@ -1328,13 +1336,13 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); if (status.state == TaskState::Finished) { - LOG_DEBUG(log, "Task {} has been successfully executed by {}", current_task_piece_status_path, status.owner); + LOG_INFO(log, "Task {} has been successfully executed by {}", current_task_piece_status_path, status.owner); return TaskStatus::Finished; } /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. /// Initialize DROP PARTITION - LOG_DEBUG(log, "Task {} has not been successfully finished by {}. Partition will be dropped and refilled.", current_task_piece_status_path, status.owner); + LOG_INFO(log, "Task {} has not been successfully finished by {}. Partition will be dropped and refilled.", current_task_piece_status_path, status.owner); create_is_dirty_node(clean_state_clock); return TaskStatus::Error; @@ -1342,6 +1350,53 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( } + /// Try create table (if not exists) on each shard + /// We have to create this table even in case that partition piece is empty + /// This is significant, because we will have simplier code + { + /// 1) Get columns description from any replica of destination cluster + /// 2) Change ENGINE, database and table name + /// 3) Create helping table on the whole destination cluster + auto & settings_push = task_cluster->settings_push; + + /// Get a connection to any shard to fetch `CREATE` query + auto connection = task_table.cluster_push->getAnyShardInfo().pool->get(timeouts, &settings_push, true); + /// Execute a request and get `CREATE` query as a string. + String create_query = getRemoteCreateTable(task_shard.task_table.table_push, *connection, settings_push); + /// Parse it to ASTPtr + ParserCreateQuery parser_create_query; + auto create_query_ast = parseQuery(parser_create_query, create_query, settings_push.max_query_size, settings_push.max_parser_depth); + /// Define helping table database and name for current partition piece + DatabaseAndTableName database_and_table_for_current_piece{ + task_table.table_push.first, + task_table.table_push.second + "_piece_" + toString(current_piece_number)}; + + /// This is a bit of legacy, because we now could parse and engine AST from the whole create query. + /// But this is needed to make helping table non-replicated. We simply don't need this + auto new_engine_push_ast = task_table.engine_push_ast; + if (task_table.isReplicatedTable()) + new_engine_push_ast = task_table.rewriteReplicatedCreateQueryToPlain(); + + /// Take columns definition from destination table, new database and table name, and new engine (non replicated variant of MergeTree) + auto create_query_push_ast = rewriteCreateQueryStorage(create_query_ast, database_and_table_for_current_piece, new_engine_push_ast); + String query = queryToString(create_query_push_ast); + + LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); + LOG_INFO( + log, + "Destination tables {} have been created on {} shards of {}", + getQuotedTable(task_table.table_push), + shards, + task_table.cluster_push->getShardCount()); + + // if (shards != task_table.cluster_push->getShardCount()) + // { + // return TaskStatus::Error; + // } + } + + /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check /// whether each shard have processed each partitition (and its pieces). if (partition_piece.is_absent_piece) @@ -1349,9 +1404,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); if (res == Coordination::Error::ZNODEEXISTS) - LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. But other replicas have already marked it as done.", task_partition.name, current_piece_number); + LOG_INFO(log, "Partition {} piece {} is absent on current replica of a shard. But other replicas have already marked it as done.", task_partition.name, current_piece_number); if (res == Coordination::Error::ZOK) - LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.", task_partition.name, current_piece_number); + LOG_INFO(log, "Partition {} piece {} is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.", task_partition.name, current_piece_number); return TaskStatus::Finished; } @@ -1415,40 +1470,6 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); } - /// Try create table (if not exists) on each shard - { - /// Define push table for current partition piece - auto database_and_table_for_current_piece= std::pair( - task_table.table_push.first, - task_table.table_push.second + "_piece_" + toString(current_piece_number)); - - auto new_engine_push_ast = task_table.engine_push_ast; - if (task_table.isReplicatedTable()) - { - new_engine_push_ast = task_table.rewriteReplicatedCreateQueryToPlain(); - } - - auto create_query_push_ast = rewriteCreateQueryStorage( - task_shard.current_pull_table_create_query, - database_and_table_for_current_piece, new_engine_push_ast); - - create_query_push_ast->as().if_not_exists = true; - String query = queryToString(create_query_push_ast); - - LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); - LOG_INFO( - log, - "Destination tables {} have been created on {} shards of {}", - getQuotedTable(task_table.table_push), - shards, - task_table.cluster_push->getShardCount()); - - if (shards != task_table.cluster_push->getShardCount()) - { - return TaskStatus::Error; - } - } /// Do the copying { @@ -1462,7 +1483,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( // Select all fields ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ true, inject_fault ? "1" : ""); - LOG_DEBUG(log, "Executing SELECT query and pull from {} : {}", task_shard.getDescription(), queryToString(query_select_ast)); + LOG_INFO(log, "Executing SELECT query and pull from {} : {}", task_shard.getDescription(), queryToString(query_select_ast)); ASTPtr query_insert_ast; { @@ -1473,7 +1494,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( const auto & settings = getContext()->getSettingsRef(); query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth); - LOG_DEBUG(log, "Executing INSERT query: {}", query); + LOG_INFO(log, "Executing INSERT query: {}", query); } try @@ -1491,8 +1512,24 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - input = io_select.getInputStream(); + auto pure_input = io_select.getInputStream(); output = io_insert.out; + + /// Add converting actions to make it possible to copy blocks with slightly different schema + const auto & select_block = pure_input->getHeader(); + const auto & insert_block = output->getHeader(); + auto actions_dag = ActionsDAG::makeConvertingActions( + select_block.getColumnsWithTypeAndName(), + insert_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext())); + + input = std::make_shared(pure_input, actions); + + std::cout << "Input:" << std::endl; + std::cout << input->getHeader().dumpStructure() << std::endl; + std::cout << "Output:" << std::endl; + std::cout << output->getHeader().dumpStructure() << std::endl; } /// Fail-fast optimization to abort copying when the current clean state expires @@ -1600,7 +1637,7 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na void ClusterCopier::dropHelpingTablesByPieceNumber(const TaskTable & task_table, size_t current_piece_number) { - LOG_DEBUG(log, "Removing helping tables piece {}", current_piece_number); + LOG_INFO(log, "Removing helping tables piece {}", current_piece_number); DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table @@ -1611,17 +1648,17 @@ void ClusterCopier::dropHelpingTablesByPieceNumber(const TaskTable & task_table, const ClusterPtr & cluster_push = task_table.cluster_push; Settings settings_push = task_cluster->settings_push; - LOG_DEBUG(log, "Execute distributed DROP TABLE: {}", query); + LOG_INFO(log, "Execute distributed DROP TABLE: {}", query); /// We have to drop partition_piece on each replica - UInt64 num_nodes = executeQueryOnCluster(cluster_push, query, settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); + UInt64 num_nodes = executeQueryOnCluster(cluster_push, query, settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO(log, "DROP TABLE query was successfully executed on {} nodes.", toString(num_nodes)); } void ClusterCopier::dropHelpingTables(const TaskTable & task_table) { - LOG_DEBUG(log, "Removing helping tables"); + LOG_INFO(log, "Removing helping tables"); for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { dropHelpingTablesByPieceNumber(task_table, current_piece_number); @@ -1630,7 +1667,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table) void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name) { - LOG_DEBUG(log, "Try drop partition partition from all helping tables."); + LOG_INFO(log, "Try drop partition partition from all helping tables."); for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { DatabaseAndTableName original_table = task_table.table_push; @@ -1641,17 +1678,16 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT const ClusterPtr & cluster_push = task_table.cluster_push; Settings settings_push = task_cluster->settings_push; - LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query); + LOG_INFO(log, "Execute distributed DROP PARTITION: {}", query); /// We have to drop partition_piece on each replica UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, settings_push, - PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO(log, "DROP PARTITION query was successfully executed on {} nodes.", toString(num_nodes)); } - LOG_DEBUG(log, "All helping tables dropped partition {}", partition_name); + LOG_INFO(log, "All helping tables dropped partition {}", partition_name); } String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings & settings) @@ -1666,6 +1702,8 @@ String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, C return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); } + + ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) { /// Fetch and parse (possibly) new definition @@ -1680,6 +1718,21 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth); } + +ASTPtr ClusterCopier::getCreateTableForPushShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) +{ + /// Fetch and parse (possibly) new definition + auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_push, true); + String create_query_pull_str = getRemoteCreateTable( + task_shard.task_table.table_push, + *connection_entry, + task_cluster->settings_push); + + ParserCreateQuery parser_create_query; + const auto & settings = getContext()->getSettingsRef(); + return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth); +} + /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split) @@ -1688,6 +1741,8 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout /// We need to update table definitions for each part, it could be changed after ALTER task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + // task_shard.current_push_table_create_query = getCreateTableForPushShard(timeouts, task_shard); + /// Create local Distributed tables: /// a table fetching data from current shard and a table inserting data to the whole destination cluster @@ -1709,7 +1764,8 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + // auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + auto create_query_ast = task_shard.current_pull_table_create_query; auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); dropAndCreateLocalTable(create_table_pull_ast); @@ -1768,7 +1824,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); - LOG_DEBUG(log, "Computing destination partition set, executing query: {}", query); + LOG_INFO(log, "Computing destination partition set, executing query: \n {}", wrapWithColor(query)); auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); @@ -1787,7 +1843,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti } } - LOG_DEBUG(log, "There are {} destination partitions in shard {}", res.size(), task_shard.getDescription()); + LOG_INFO(log, "There are {} destination partitions in shard {}", res.size(), task_shard.getDescription()); return res; } @@ -1799,21 +1855,22 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskTable & task_table = task_shard.task_table; - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + - " = (" + partition_quoted_name + " AS partition_key))"; - + WriteBufferFromOwnString ss; + ss << "SELECT 1 FROM " << getQuotedTable(task_shard.table_read_shard); + ss << " WHERE (" << queryToString(task_table.engine_push_partition_key_ast); + ss << " = (" + partition_quoted_name << " AS partition_key))"; if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard {} for partition {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, query); + ss << " AND (" << task_table.where_condition_str << ")"; + ss << " LIMIT 1"; + auto query = ss.str(); ParserQuery parser_query(query.data() + query.size()); const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); + LOG_INFO(log, "Checking shard {} for partition {} existence, executing query: \n {}", + task_shard.getDescription(), partition_quoted_name, query_ast->formatForErrorMessage()); + auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); return InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows() != 0; @@ -1847,7 +1904,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi query += " LIMIT 1"; - LOG_DEBUG(log, "Checking shard {} for partition {} piece {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query); + LOG_INFO(log, "Checking shard {} for partition {} piece {} existence, executing query: \n \u001b[36m {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query); ParserQuery parser_query(query.data() + query.size()); const auto & settings = getContext()->getSettingsRef(); @@ -1857,12 +1914,13 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi local_context->setSettings(task_cluster->settings_pull); auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows(); if (result != 0) - LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); + LOG_INFO(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); else - LOG_DEBUG(log, "Partition {} piece number {} is ABSENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); + LOG_INFO(log, "Partition {} piece number {} is ABSENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); return result != 0; } + /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster * Returns number of shards for which at least one replica executed query successfully */ @@ -1870,112 +1928,64 @@ UInt64 ClusterCopier::executeQueryOnCluster( const ClusterPtr & cluster, const String & query, const Settings & current_settings, - PoolMode pool_mode, - ClusterExecutionMode execution_mode, - UInt64 max_successful_executions_per_shard) const + ClusterExecutionMode execution_mode) const { - auto num_shards = cluster->getShardsInfo().size(); - std::vector per_shard_num_successful_replicas(num_shards, 0); - - ParserQuery p_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(p_query, query, current_settings.max_query_size, current_settings.max_parser_depth); - - /// We will have to execute query on each replica of a shard. + ClusterPtr cluster_for_query = cluster; if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) - max_successful_executions_per_shard = 0; + cluster_for_query = cluster->getClusterWithReplicasAsShards(current_settings); - std::atomic origin_replicas_number = 0; + std::vector> connections; + connections.reserve(cluster->getShardCount()); - /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index, Settings shard_settings) + std::atomic successfully_executed = 0; + + for (const auto & replicas : cluster_for_query->getShardsAddresses()) { - setThreadName("QueryForShard"); - - const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); - UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); - num_successful_executions = 0; - - auto increment_and_check_exit = [&] () -> bool + const auto & node = replicas[0]; + try { - ++num_successful_executions; - return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; - }; + connections.emplace_back(std::make_shared( + node.host_name, node.port, node.default_database, + node.user, node.password, node.cluster, node.cluster_secret, + "ClusterCopier", node.compression, node.secure + )); - UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + /// We execute only Alter, Create and Drop queries. + const auto header = Block{}; - origin_replicas_number += num_replicas; - UInt64 num_local_replicas = shard.getLocalNodeCount(); - UInt64 num_remote_replicas = num_replicas - num_local_replicas; + /// For unknown reason global context is passed to IStorage::read() method + /// So, task_identifier is passed as constructor argument. It is more obvious. + auto remote_query_executor = std::make_shared( + *connections.back(), query, header, getContext(), + /*throttler=*/nullptr, Scalars(), Tables(), QueryProcessingStage::Complete); - /// In that case we don't have local replicas, but do it just in case - for (UInt64 i = 0; i < num_local_replicas; ++i) - { - auto interpreter = InterpreterFactory::get(query_ast, getContext()); - interpreter->execute(); - - if (increment_and_check_exit()) - return; - } - - /// Will try to make as many as possible queries - if (shard.hasRemoteConnections()) - { - shard_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(shard_settings).getSaturated(shard_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, &shard_settings, pool_mode); - - auto shard_context = Context::createCopy(context); - shard_context->setSettings(shard_settings); - - for (auto & connection : connections) + try { - if (connection.isNull()) - continue; - - try - { - /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, shard_context}; - NullBlockOutputStream output{Block{}}; - copyData(stream, output); - - if (increment_and_check_exit()) - return; - } - catch (const Exception &) - { - LOG_INFO(log, getCurrentExceptionMessage(false, true)); - } + remote_query_executor->sendQuery(); } + catch (...) + { + LOG_WARNING(log, "Seemns like node with address {} is unreachable {}", node.host_name); + } + + while (true) + { + auto block = remote_query_executor->read(); + if (!block) + break; + } + + remote_query_executor->finish(); + ++successfully_executed; + } + catch (...) + { + LOG_WARNING(log, "An error occured while processing query : \n {}", wrapWithColor(query)); + tryLogCurrentException(log); } - }; - - { - ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); - - for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=, shard_settings = current_settings] { do_for_shard(shard_index, std::move(shard_settings)); }); - - thread_pool.wait(); } - UInt64 successful_nodes = 0; - for (UInt64 num_replicas : per_shard_num_successful_replicas) - { - if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) - successful_nodes += num_replicas; - else - /// Count only successful shards - successful_nodes += (num_replicas > 0); - } - - if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number) - { - LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on {} nodes. But had to be executed on {}", toString(successful_nodes), toString(origin_replicas_number.load())); - } - - return successful_nodes; + return successfully_executed.load(); } } diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index e875ca7df2e..7a547a8c552 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -18,12 +18,15 @@ public: ClusterCopier(const String & task_path_, const String & host_id_, const String & proxy_database_name_, - ContextPtr context_) + ContextPtr context_, + Poco::Logger * log_) : WithContext(context_), task_zookeeper_path(task_path_), host_id(host_id_), working_database_name(proxy_database_name_), - log(&Poco::Logger::get("ClusterCopier")) {} + log(log_) { + std::cout << "Level from constructor" << log->getLevel() << std::endl; + } void init(); @@ -159,6 +162,7 @@ protected: String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings & settings); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); + ASTPtr getCreateTableForPushShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true); @@ -189,9 +193,7 @@ protected: const ClusterPtr & cluster, const String & query, const Settings & current_settings, - PoolMode pool_mode = PoolMode::GET_ALL, - ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD, - UInt64 max_successful_executions_per_shard = 0) const; + ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD) const; private: String task_zookeeper_path; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index bb3e37bba45..27e212d29cc 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -21,7 +21,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) config_xml_path = config().getString("config-file"); task_path = config().getString("task-path"); - log_level = config().getString("log-level", "trace"); + log_level = config().getString("log-level", "info"); is_safe_mode = config().has("safe-mode"); if (config().has("copy-fault-probability")) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); @@ -110,6 +110,8 @@ void ClusterCopierApp::mainImpl() ThreadStatus thread_status; auto * log = &logger(); + log->setLevel(6); /// Information + std::cout << log->getLevel() << std::endl; LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::getVersionRevision()); SharedContextHolder shared_context = Context::createShared(); @@ -137,7 +139,7 @@ void ClusterCopierApp::mainImpl() CurrentThread::QueryScope query_scope(context); std::cout << "Will construct copier" << std::endl; - auto copier = std::make_unique(task_path, host_id, default_database, context); + auto copier = std::make_unique(task_path, host_id, default_database, context, log); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); copier->setMoveFaultProbability(move_fault_probability); diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 4f5bfb443e6..b3a56460f3d 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -50,7 +50,6 @@ struct TaskTable String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - bool isReplicatedTable() const { return is_replicated_table; } /// Partitions will be split into number-of-splits pieces. @@ -181,6 +180,7 @@ struct TaskShard /// Last CREATE TABLE query of the table of the shard ASTPtr current_pull_table_create_query; + ASTPtr current_push_table_create_query; /// Internal distributed tables DatabaseAndTableName table_read_shard; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3a3291c6c48..192765e33ea 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -497,14 +497,14 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query); if (optimized_cluster) { - LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", + LOG_TRACE(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); cluster = optimized_cluster; query_info.optimized_cluster = cluster; } else { - LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", + LOG_TRACE(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", has_sharding_key ? "" : " (no sharding key)"); } } @@ -536,7 +536,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( auto stage = getOptimizedQueryProcessingStage(query_info, settings.extremes, sharding_key_block); if (stage) { - LOG_DEBUG(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); + LOG_TRACE(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); return *stage; } } @@ -789,7 +789,7 @@ void StorageDistributed::startup() if (inc > file_names_increment.value) file_names_increment.value.store(inc); } - LOG_DEBUG(log, "Auto-increment is {}", file_names_increment.value); + LOG_TRACE(log, "Auto-increment is {}", file_names_increment.value); } @@ -799,9 +799,9 @@ void StorageDistributed::shutdown() std::lock_guard lock(cluster_nodes_mutex); - LOG_DEBUG(log, "Joining background threads for async INSERT"); + LOG_TRACE(log, "Joining background threads for async INSERT"); cluster_nodes_data.clear(); - LOG_DEBUG(log, "Background threads for async INSERT joined"); + LOG_TRACE(log, "Background threads for async INSERT joined"); } void StorageDistributed::drop() { @@ -819,13 +819,13 @@ void StorageDistributed::drop() if (relative_data_path.empty()) return; - LOG_DEBUG(log, "Removing pending blocks for async INSERT from filesystem on DROP TABLE"); + LOG_TRACE(log, "Removing pending blocks for async INSERT from filesystem on DROP TABLE"); auto disks = data_volume->getDisks(); for (const auto & disk : disks) disk->removeRecursive(relative_data_path); - LOG_DEBUG(log, "Removed"); + LOG_TRACE(log, "Removed"); } Strings StorageDistributed::getDataPaths() const @@ -845,7 +845,7 @@ void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, Co { std::lock_guard lock(cluster_nodes_mutex); - LOG_DEBUG(log, "Removing pending blocks for async INSERT from filesystem on TRUNCATE TABLE"); + LOG_TRACE(log, "Removing pending blocks for async INSERT from filesystem on TRUNCATE TABLE"); for (auto it = cluster_nodes_data.begin(); it != cluster_nodes_data.end();) { @@ -853,7 +853,7 @@ void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, Co it = cluster_nodes_data.erase(it); } - LOG_DEBUG(log, "Removed"); + LOG_TRACE(log, "Removed"); } StoragePolicyPtr StorageDistributed::getStoragePolicy() const @@ -881,7 +881,7 @@ void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) if (std::filesystem::is_empty(dir_path)) { - LOG_DEBUG(log, "Removing {} (used for async INSERT into Distributed)", dir_path.string()); + LOG_TRACE(log, "Removing {} (used for async INSERT into Distributed)", dir_path.string()); /// Will be created by DistributedBlockOutputStream on demand. std::filesystem::remove(dir_path); } @@ -1138,7 +1138,7 @@ void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) disk->moveDirectory(relative_data_path, new_path_to_table_data); auto new_path = disk->getPath() + new_path_to_table_data; - LOG_DEBUG(log, "Updating path to {}", new_path); + LOG_TRACE(log, "Updating path to {}", new_path); std::lock_guard lock(cluster_nodes_mutex); for (auto & node : cluster_nodes_data) diff --git a/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml b/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml index f5784e732dc..09dd7a00762 100644 --- a/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml +++ b/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml @@ -1,7 +1,7 @@ - trace + fatal /var/log/clickhouse-server/copier/log.log /var/log/clickhouse-server/copier/log.err.log 1000M diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml new file mode 100644 index 00000000000..21c92e676fd --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/clusters.xml @@ -0,0 +1,23 @@ + + + + + + false + + first + 9000 + + + + + + false + + second + 9000 + + + + + diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml new file mode 100644 index 00000000000..4bff11fb693 --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/ddl.xml @@ -0,0 +1,6 @@ + + + + /clickhouse/task_queue/ddl + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml b/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml new file mode 100644 index 00000000000..ebea79e92e6 --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml @@ -0,0 +1,20 @@ + + + + fatal + /var/log/clickhouse-server/copier/log.log + /var/log/clickhouse-server/copier/log.err.log + 1000M + 10 + /var/log/clickhouse-server/copier/stderr.log + /var/log/clickhouse-server/copier/stdout.log + + + + + zoo1 + 2181 + + 2000 + + diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/users.xml b/tests/integration/test_cluster_copier/configs_two_nodes/users.xml new file mode 100644 index 00000000000..023598304f2 --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_two_nodes/users.xml @@ -0,0 +1,32 @@ + + + + + 1 + + + + + + + + ::/0 + + default + default + + + 12345678 + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_cluster_copier/task_taxi_data.xml b/tests/integration/test_cluster_copier/task_taxi_data.xml index 18a7d19f68d..fafffe3ebc9 100644 --- a/tests/integration/test_cluster_copier/task_taxi_data.xml +++ b/tests/integration/test_cluster_copier/task_taxi_data.xml @@ -32,12 +32,12 @@ events dailyhistory - yellow_tripdata + yellow_tripdata_staging events monthlyhistory - yellow_tripdata + yellow_tripdata_staging Engine=ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (pickup_location_id, toYYYYMM(tpep_pickup_datetime)) - rand() + sipHash64(id) % 3 \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/task_with_different_schema.xml b/tests/integration/test_cluster_copier/task_with_different_schema.xml new file mode 100644 index 00000000000..4482bece2db --- /dev/null +++ b/tests/integration/test_cluster_copier/task_with_different_schema.xml @@ -0,0 +1,40 @@ + + + + + + false + + first + 9000 + + + + + + false + + second + 9000 + + + + + + 2 + + + + source + db_different_schema + source + + destination + db_different_schema + destination + + ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column9, Column1, Column2, Column3, Column4) + rand() + + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test_tables_with_different_schema.py b/tests/integration/test_cluster_copier/test_tables_with_different_schema.py new file mode 100644 index 00000000000..d8c9068d744 --- /dev/null +++ b/tests/integration/test_cluster_copier/test_tables_with_different_schema.py @@ -0,0 +1,175 @@ +import os +import sys +import time +import logging +import subprocess +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +import docker + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) + + +COPYING_FAIL_PROBABILITY = 0.33 +MOVING_FAIL_PROBABILITY = 0.1 +cluster = None + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster = ClickHouseCluster(__file__) + + for name in ["first", "second", "third"]: + cluster.add_instance(name, + main_configs=["configs_two_nodes/conf.d/clusters.xml", "configs_two_nodes/conf.d/ddl.xml"], user_configs=["configs_two_nodes/users.xml"], + with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) + + cluster.start() + yield cluster + + finally: + pass + cluster.shutdown() + +# Will copy table from `first` node to `second` +class TaskWithDifferentSchema: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path = '/clickhouse-copier/task_with_different_schema' + self.container_task_file = "/task_with_different_schema.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_with_different_schema.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + + def start(self): + first = cluster.instances["first"] + first.query("CREATE DATABASE db_different_schema;") + first.query("""CREATE TABLE db_different_schema.source + ( + Column1 String, + Column2 UInt32, + Column3 Date, + Column4 DateTime, + Column5 UInt16, + Column6 String, + Column7 String, + Column8 String, + Column9 String, + Column10 String, + Column11 String, + Column12 Decimal(3, 1), + Column13 DateTime, + Column14 UInt16 + ) + ENGINE = MergeTree() + PARTITION BY (toYYYYMMDD(Column3), Column3) + PRIMARY KEY (Column1, Column2, Column3, Column4, Column6, Column7, Column8, Column9) + ORDER BY (Column1, Column2, Column3, Column4, Column6, Column7, Column8, Column9) + SETTINGS index_granularity = 8192""") + + first.query("""INSERT INTO db_different_schema.source SELECT * FROM generateRandom( + 'Column1 String, Column2 UInt32, Column3 Date, Column4 DateTime, Column5 UInt16, + Column6 String, Column7 String, Column8 String, Column9 String, Column10 String, + Column11 String, Column12 Decimal(3, 1), Column13 DateTime, Column14 UInt16', 1, 10, 2) LIMIT 100;""") + + second = cluster.instances["second"] + second.query("CREATE DATABASE db_different_schema;") + second.query("""CREATE TABLE db_different_schema.destination + ( + Column1 LowCardinality(String) CODEC(LZ4), + Column2 UInt32 CODEC(LZ4), + Column3 Date CODEC(DoubleDelta, LZ4), + Column4 DateTime CODEC(DoubleDelta, LZ4), + Column5 UInt16 CODEC(LZ4), + Column6 LowCardinality(String) CODEC(ZSTD), + Column7 LowCardinality(String) CODEC(ZSTD), + Column8 LowCardinality(String) CODEC(ZSTD), + Column9 LowCardinality(String) CODEC(ZSTD), + Column10 String CODEC(ZSTD(6)), + Column11 LowCardinality(String) CODEC(LZ4), + Column12 Decimal(3,1) CODEC(LZ4), + Column13 DateTime CODEC(DoubleDelta, LZ4), + Column14 UInt16 CODEC(LZ4) + ) ENGINE = MergeTree() + PARTITION BY toYYYYMMDD(Column3) + ORDER BY (Column9, Column1, Column2, Column3, Column4);""") + + print("Preparation completed") + + def check(self): + first = cluster.instances["first"] + second = cluster.instances["second"] + + a = first.query("SELECT count() from db_different_schema.source") + b = second.query("SELECT count() from db_different_schema.destination") + assert a == b, "Count" + + a = TSV(first.query("""SELECT sipHash64(*) from db_different_schema.source + ORDER BY (Column1, Column2, Column3, Column4, Column5, Column6, Column7, Column8, Column9, Column10, Column11, Column12, Column13, Column14)""")) + b = TSV(second.query("""SELECT sipHash64(*) from db_different_schema.destination + ORDER BY (Column1, Column2, Column3, Column4, Column5, Column6, Column7, Column8, Column9, Column10, Column11, Column12, Column13, Column14)""")) + assert a == b, "Data" + + +def execute_task(task, cmd_options): + task.start() + + zk = cluster.get_kazoo_client('zoo1') + print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) + + # Run cluster-copier processes on each node + docker_api = docker.from_env().api + copiers_exec_ids = [] + + cmd = ['/usr/bin/clickhouse', 'copier', + '--config', '/etc/clickhouse-server/config-copier.xml', + '--task-path', task.zk_task_path, + '--task-file', task.container_task_file, + '--task-upload-force', 'true', + '--base-dir', '/var/log/clickhouse-server/copier'] + cmd += cmd_options + + print(cmd) + + for instance_name, instance in cluster.instances.items(): + instance = cluster.instances[instance_name] + container = instance.get_docker_handle() + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_taxi/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + logging.info("Copied copier config to {}".format(instance.name)) + exec_id = docker_api.exec_create(container.id, cmd, stderr=True) + output = docker_api.exec_start(exec_id).decode('utf8') + logging.info(output) + copiers_exec_ids.append(exec_id) + logging.info("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) + + # time.sleep(1000) + + # Wait for copiers stopping and check their return codes + for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + while True: + res = docker_api.exec_inspect(exec_id) + if not res['Running']: + break + time.sleep(1) + + assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) + + try: + task.check() + finally: + zk.delete(task.zk_task_path, recursive=True) + + +# Tests +@pytest.mark.timeout(1200) +def test1(started_cluster): + execute_task(TaskWithDifferentSchema(started_cluster), []) + diff --git a/tests/integration/test_cluster_copier/test_taxi_data.py b/tests/integration/test_cluster_copier/test_taxi_data.py index 99ac727fbe6..81c1fe9fb25 100644 --- a/tests/integration/test_cluster_copier/test_taxi_data.py +++ b/tests/integration/test_cluster_copier/test_taxi_data.py @@ -114,7 +114,7 @@ class Task: junk1 String, junk2 String ) Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (toYYYYMMDD(tpep_pickup_datetime))""") - instance.query("CREATE TABLE dailyhistory.yellow_tripdata ON CLUSTER events AS dailyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'dailyhistory', yellow_tripdata_staging, rand());") + instance.query("CREATE TABLE dailyhistory.yellow_tripdata ON CLUSTER events AS dailyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'dailyhistory', yellow_tripdata_staging, sipHash64(id) % 3);") # monthly partition database instance.query("create database monthlyhistory on cluster events;") @@ -127,25 +127,38 @@ class Task: pickup_location_id String, dropoff_location_id String, congestion_surcharge String, junk1 String, junk2 String ) Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (pickup_location_id, toYYYYMM(tpep_pickup_datetime))""") - instance.query("CREATE TABLE monthlyhistory.yellow_tripdata ON CLUSTER events AS monthlyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'monthlyhistory', yellow_tripdata_staging, rand());") + instance.query("CREATE TABLE monthlyhistory.yellow_tripdata ON CLUSTER events AS monthlyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'monthlyhistory', yellow_tripdata_staging, sipHash64(id) % 3);") - logging.info("Inserting in container") + print("Inserting in container") + first_query = """INSERT INTO dailyhistory.yellow_tripdata( vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance, rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type, fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,congestion_surcharge) FORMAT CSV""" instance.exec_in_container(['bash', '-c', 'cat /usr/share/clickhouse-external-data/first.csv | /usr/bin/clickhouse client --query="{}"'.format(first_query)], privileged=True) - logging.info("Insert completed") + + print("Insert completed") def check(self): + instance = cluster.instances["first"] + a = TSV(instance.query("SELECT count() from dailyhistory.yellow_tripdata")) + b = TSV(instance.query("SELECT count() from monthlyhistory.yellow_tripdata")) + assert a == b, "Distributed tables" + for instance_name, instance in cluster.instances.items(): instance = cluster.instances[instance_name] a = instance.query("SELECT count() from dailyhistory.yellow_tripdata_staging") b = instance.query("SELECT count() from monthlyhistory.yellow_tripdata_staging") - print(a, b) - assert a == b + assert a == b, "MergeTree tables on each shard" + + a = TSV(instance.query("SELECT sipHash64(*) from dailyhistory.yellow_tripdata_staging ORDER BY id")) + b = TSV(instance.query("SELECT sipHash64(*) from monthlyhistory.yellow_tripdata_staging ORDER BY id")) + + assert a == b, "Data on each shard" + + def execute_task(task, cmd_options): From c41cc36046bed2c873ea0ed468eeb2ca09eb6650 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 22 Apr 2021 23:37:22 +0300 Subject: [PATCH 278/652] support TTL on columns --- programs/copier/ClusterCopier.cpp | 18 ++++- .../test_cluster_copier/task_ttl_columns.xml | 40 ++++++++++ .../test_tables_with_different_schema.py | 78 ++++++++++++++++++- 3 files changed, 131 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_cluster_copier/task_ttl_columns.xml diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index fa6b9fe5862..c00165adf89 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -694,7 +694,10 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t return TaskStatus::Finished; } -/// Removes MATERIALIZED and ALIAS columns from create table query +/// Remove column's TTL expression from `CREATE` query +/// This is needed to create internal Distributed table +/// Also it removes MATEREALIZED or ALIAS columns not to copy additional and useless data over the network. +/// TODO: Make removing MATERIALIZED and ALIAS columns optional. ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { const ASTs & column_asts = query_ast->as().columns_list->columns->children; @@ -704,6 +707,7 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast { const auto & column = column_ast->as(); + /// Skip this columns if (!column.default_specifier.empty()) { ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); @@ -711,7 +715,13 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast continue; } - new_columns->children.emplace_back(column_ast->clone()); + /// Remove TTL on columns definition. + auto new_column = column_ast->clone(); + auto * new_column_ptr = new_column->as(); + if (new_column_ptr->ttl) + new_column_ptr->ttl.reset(); + + new_columns->children.emplace_back(new_column); } ASTPtr new_query_ast = query_ast->clone(); @@ -1764,8 +1774,8 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - // auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_query_ast = task_shard.current_pull_table_create_query; + auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + // auto create_query_ast = task_shard.current_pull_table_create_query; auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); dropAndCreateLocalTable(create_table_pull_ast); diff --git a/tests/integration/test_cluster_copier/task_ttl_columns.xml b/tests/integration/test_cluster_copier/task_ttl_columns.xml new file mode 100644 index 00000000000..2b8174a7c4b --- /dev/null +++ b/tests/integration/test_cluster_copier/task_ttl_columns.xml @@ -0,0 +1,40 @@ + + + + + + false + + first + 9000 + + + + + + false + + second + 9000 + + + + + + 2 + + + + source + db_ttl_columns + source + + destination + db_ttl_columns + destination + + ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1) + rand() + + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test_tables_with_different_schema.py b/tests/integration/test_cluster_copier/test_tables_with_different_schema.py index d8c9068d744..8e98387b5e9 100644 --- a/tests/integration/test_cluster_copier/test_tables_with_different_schema.py +++ b/tests/integration/test_cluster_copier/test_tables_with_different_schema.py @@ -119,6 +119,76 @@ class TaskWithDifferentSchema: assert a == b, "Data" +# Just simple copying, but table schema has TTL on columns +# Also table will have slightly different schema +class TaskTTL: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path = '/clickhouse-copier/task_ttl_columns' + self.container_task_file = "/task_ttl_columns.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_ttl_columns.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + + def start(self): + first = cluster.instances["first"] + first.query("CREATE DATABASE db_ttl_columns;") + first.query("""CREATE TABLE db_ttl_columns.source + ( + Column1 String, + Column2 UInt32, + Column3 Date, + Column4 DateTime, + Column5 UInt16, + Column6 String TTL now() + INTERVAL 1 MONTH, + Column7 Decimal(3, 1) TTL now() + INTERVAL 1 MONTH, + Column8 Tuple(Float64, Float64) TTL now() + INTERVAL 1 MONTH + ) + ENGINE = MergeTree() + PARTITION BY (toYYYYMMDD(Column3), Column3) + PRIMARY KEY (Column1, Column2, Column3) + ORDER BY (Column1, Column2, Column3) + SETTINGS index_granularity = 8192""") + + first.query("""INSERT INTO db_ttl_columns.source SELECT * FROM generateRandom( + 'Column1 String, Column2 UInt32, Column3 Date, Column4 DateTime, Column5 UInt16, + Column6 String, Column7 Decimal(3, 1), Column8 Tuple(Float64, Float64)', 1, 10, 2) LIMIT 100;""") + + second = cluster.instances["second"] + second.query("CREATE DATABASE db_ttl_columns;") + second.query("""CREATE TABLE db_ttl_columns.destination + ( + Column1 String, + Column2 UInt32, + Column3 Date, + Column4 DateTime TTL now() + INTERVAL 1 MONTH, + Column5 UInt16 TTL now() + INTERVAL 1 MONTH, + Column6 String TTL now() + INTERVAL 1 MONTH, + Column7 Decimal(3, 1) TTL now() + INTERVAL 1 MONTH, + Column8 Tuple(Float64, Float64) + ) ENGINE = MergeTree() + PARTITION BY toYYYYMMDD(Column3) + ORDER BY (Column3, Column2, Column1);""") + + print("Preparation completed") + + def check(self): + first = cluster.instances["first"] + second = cluster.instances["second"] + + a = first.query("SELECT count() from db_ttl_columns.source") + b = second.query("SELECT count() from db_ttl_columns.destination") + assert a == b, "Count" + + a = TSV(first.query("""SELECT sipHash64(*) from db_ttl_columns.source + ORDER BY (Column1, Column2, Column3, Column4, Column5, Column6, Column7, Column8)""")) + b = TSV(second.query("""SELECT sipHash64(*) from db_ttl_columns.destination + ORDER BY (Column1, Column2, Column3, Column4, Column5, Column6, Column7, Column8)""")) + assert a == b, "Data" + + def execute_task(task, cmd_options): task.start() @@ -168,8 +238,14 @@ def execute_task(task, cmd_options): zk.delete(task.zk_task_path, recursive=True) + + # Tests @pytest.mark.timeout(1200) -def test1(started_cluster): +def test_different_schema(started_cluster): execute_task(TaskWithDifferentSchema(started_cluster), []) + + +def test_ttl_columns(started_cluster): + execute_task(TaskTTL(started_cluster), []) \ No newline at end of file From 24af47063e22e603f2a2b7c7f8be859ed326880f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 01:32:16 +0300 Subject: [PATCH 279/652] allow to copy materialized and alias --- programs/copier/ClusterCopier.cpp | 22 ++++++++++------------ programs/copier/ClusterCopier.h | 2 +- programs/copier/TaskCluster.h | 2 +- programs/copier/TaskTableAndShard.h | 6 +++++- 4 files changed, 17 insertions(+), 15 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index c00165adf89..97f5fd0e38c 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -697,8 +697,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t /// Remove column's TTL expression from `CREATE` query /// This is needed to create internal Distributed table /// Also it removes MATEREALIZED or ALIAS columns not to copy additional and useless data over the network. -/// TODO: Make removing MATERIALIZED and ALIAS columns optional. -ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) +ASTPtr ClusterCopier::removeAliasMaterializedAndTTLColumnsFromCreateQuery(const ASTPtr & query_ast, bool allow_to_copy_alias_and_materialized_columns) { const ASTs & column_asts = query_ast->as().columns_list->columns->children; auto new_columns = std::make_shared(); @@ -708,7 +707,7 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast const auto & column = column_ast->as(); /// Skip this columns - if (!column.default_specifier.empty()) + if (!column.default_specifier.empty() && !allow_to_copy_alias_and_materialized_columns) { ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) @@ -716,12 +715,12 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast } /// Remove TTL on columns definition. - auto new_column = column_ast->clone(); - auto * new_column_ptr = new_column->as(); - if (new_column_ptr->ttl) - new_column_ptr->ttl.reset(); + auto new_column_ast = column_ast->clone(); + auto & new_column = new_column_ast->as(); + if (new_column.ttl) + new_column.ttl.reset(); - new_columns->children.emplace_back(new_column); + new_columns->children.emplace_back(new_column_ast); } ASTPtr new_query_ast = query_ast->clone(); @@ -1751,8 +1750,6 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout /// We need to update table definitions for each part, it could be changed after ALTER task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); - // task_shard.current_push_table_create_query = getCreateTableForPushShard(timeouts, task_shard); - /// Create local Distributed tables: /// a table fetching data from current shard and a table inserting data to the whole destination cluster @@ -1774,8 +1771,9 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - // auto create_query_ast = task_shard.current_pull_table_create_query; + auto create_query_ast = removeAliasMaterializedAndTTLColumnsFromCreateQuery( + task_shard.current_pull_table_create_query, + task_table.allow_to_copy_alias_and_materialized_columns); auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); dropAndCreateLocalTable(create_table_pull_ast); diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 7a547a8c552..ad44e3b0ec4 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -120,7 +120,7 @@ protected: TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); + static ASTPtr removeAliasMaterializedAndTTLColumnsFromCreateQuery(const ASTPtr & query_ast, bool allow_to_copy_alias_and_materialized_columns); bool tryDropPartitionPiece(ShardPartition & task_partition, size_t current_piece_number, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); diff --git a/programs/copier/TaskCluster.h b/programs/copier/TaskCluster.h index 1a50597d07f..5cf247abaf9 100644 --- a/programs/copier/TaskCluster.h +++ b/programs/copier/TaskCluster.h @@ -92,7 +92,7 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat /// Override important settings settings_pull.readonly = 1; - settings_push.insert_distributed_sync = 1; + settings_push.insert_distributed_sync = true; set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); set_default_value(settings_pull.max_threads, 1); set_default_value(settings_pull.max_block_size, 8192UL); diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index b3a56460f3d..569aebae429 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -56,6 +56,8 @@ struct TaskTable /// Each piece will be copied independently. (10 by default) size_t number_of_splits; + bool allow_to_copy_alias_and_materialized_columns{false}; + String name_in_config; /// Used as task ID @@ -250,7 +252,9 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf name_in_config = table_key; - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 3); + + allow_to_copy_alias_and_materialized_columns = config.getBool(table_prefix + "allow_to_copy_alias_and_materialized_columns", false); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); From 86c9373fb768afe6d6a859981b3a80892317a7e4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 02:13:05 +0300 Subject: [PATCH 280/652] prettify tests --- .../test_cluster_copier/data/.gitkeep | 0 .../test_cluster_copier/data/first.csv | 100 -------- .../test_cluster_copier/prepare_taxi_data.py | 57 ----- tests/integration/test_cluster_copier/test.py | 6 +- .../test_cluster_copier/test_taxi_data.py | 217 ----------------- .../test_cluster_copier/test_three_nodes.py | 229 ++++++++++++++++++ ..._different_schema.py => test_two_nodes.py} | 17 +- 7 files changed, 237 insertions(+), 389 deletions(-) create mode 100644 tests/integration/test_cluster_copier/data/.gitkeep delete mode 100644 tests/integration/test_cluster_copier/data/first.csv delete mode 100644 tests/integration/test_cluster_copier/prepare_taxi_data.py delete mode 100644 tests/integration/test_cluster_copier/test_taxi_data.py create mode 100644 tests/integration/test_cluster_copier/test_three_nodes.py rename tests/integration/test_cluster_copier/{test_tables_with_different_schema.py => test_two_nodes.py} (97%) diff --git a/tests/integration/test_cluster_copier/data/.gitkeep b/tests/integration/test_cluster_copier/data/.gitkeep new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_cluster_copier/data/first.csv b/tests/integration/test_cluster_copier/data/first.csv deleted file mode 100644 index 5efc10c0ca2..00000000000 --- a/tests/integration/test_cluster_copier/data/first.csv +++ /dev/null @@ -1,100 +0,0 @@ -1,2020-12-01 00:41:19,2020-12-01 00:49:45,1,1.60,1,N,140,263,1,8,3,0.5,2.95,0,0.3,14.75,2.5 -2,2020-12-01 00:33:40,2020-12-01 01:00:35,1,16.74,2,N,132,164,1,52,0,0.5,2.5,6.12,0.3,63.92,2.5 -2,2020-12-01 00:02:15,2020-12-01 00:13:09,1,4.16,1,N,238,48,1,14,0.5,0.5,1,0,0.3,18.8,2.5 -2,2020-12-01 00:37:42,2020-12-01 00:45:11,1,2.22,1,N,238,41,2,8.5,0.5,0.5,0,0,0.3,9.8,0 -1,2020-12-01 00:27:47,2020-12-01 00:45:40,0,8.40,1,N,138,137,1,25,3,0.5,6,6.12,0.3,40.92,2.5 -2,2020-12-01 00:40:47,2020-12-01 00:57:03,1,6.44,1,N,132,191,1,19.5,0.5,0.5,4.16,0,0.3,24.96,0 -2,2020-12-01 00:01:42,2020-12-01 00:06:06,1,.99,1,N,234,137,1,5.5,0.5,0.5,1.86,0,0.3,11.16,2.5 -2,2020-12-01 00:58:24,2020-12-01 01:36:14,2,11.81,1,N,261,7,1,36.5,0.5,0.5,1,0,0.3,41.3,2.5 -1,2020-12-01 00:08:15,2020-12-01 00:16:04,2,2.70,1,N,237,107,1,9.5,3,0.5,2.65,0,0.3,15.95,2.5 -2,2020-12-01 00:04:21,2020-12-01 00:29:00,1,6.28,1,N,41,68,2,23,0.5,0.5,0,0,0.3,26.8,2.5 -2,2020-12-01 00:22:03,2020-12-01 00:52:55,1,18.93,2,N,132,211,1,52,0,0.5,12.28,6.12,0.3,73.7,2.5 -2,2020-12-01 00:46:31,2020-12-01 00:53:49,5,2.75,1,N,249,230,2,9.5,0.5,0.5,0,0,0.3,13.3,2.5 -2,2020-12-01 00:51:30,2020-12-01 01:26:52,2,21.80,2,N,132,13,1,52,0,0.5,11.06,0,0.3,66.36,2.5 -1,2020-12-01 00:14:34,2020-12-01 00:31:04,1,7.60,1,N,140,65,1,23,3,0.5,5.35,0,0.3,32.15,2.5 -1,2020-12-01 00:11:02,2020-12-01 00:17:34,1,1.70,1,N,239,141,2,7.5,3,0.5,0,0,0.3,11.3,2.5 -1,2020-12-01 00:54:55,2020-12-01 00:57:09,1,.50,1,N,263,141,2,4,3,0.5,0,0,0.3,7.8,2.5 -1,2020-12-01 00:11:22,2020-12-01 00:40:36,1,21.00,2,N,132,231,1,52,2.5,0.5,16.55,0,0.3,71.85,2.5 -2,2020-11-30 23:59:22,2020-12-01 00:05:51,3,.81,1,N,50,48,1,5.5,0.5,0.5,1.88,0,0.3,11.18,2.5 -2,2020-12-01 00:24:34,2020-12-01 00:29:59,4,1.45,1,N,48,162,1,6.5,0.5,0.5,2.06,0,0.3,12.36,2.5 -1,2020-12-01 00:53:58,2020-12-01 00:54:06,1,.00,1,N,132,132,3,2.5,0.5,0.5,0,0,0.3,3.8,0 -2,2020-12-01 00:20:44,2020-12-01 00:32:48,5,3.78,1,N,140,42,2,13.5,0.5,0.5,0,0,0.3,17.3,2.5 -2,2020-12-01 00:42:13,2020-12-01 00:46:05,1,1.12,1,N,138,129,2,5.5,0.5,0.5,0,0,0.3,6.8,0 -2,2020-12-01 00:02:45,2020-12-01 00:11:35,1,1.63,1,N,137,48,1,8,0.5,0.5,2.36,0,0.3,14.16,2.5 -2,2020-12-01 00:14:38,2020-12-01 00:38:53,3,7.01,1,N,137,129,2,23.5,0.5,0.5,0,0,0.3,27.3,2.5 -2,2020-12-01 00:21:33,2020-12-01 00:33:44,1,5.31,1,N,141,69,1,16,0.5,0.5,0,0,0.3,19.8,2.5 -2,2020-12-01 00:34:26,2020-12-01 00:50:02,1,7.92,1,N,138,137,1,23,0.5,0.5,6.58,6.12,0.3,39.5,2.5 -1,2020-12-01 00:52:58,2020-12-01 00:54:28,1,.70,1,N,162,170,2,4,3,0.5,0,0,0.3,7.8,2.5 -2,2020-12-01 00:30:32,2020-12-01 00:39:11,2,2.33,1,N,161,140,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 -2,2020-11-30 23:56:50,2020-12-01 00:03:37,1,1.72,1,N,161,107,2,7.5,0.5,0.5,0,0,0.3,11.3,2.5 -2,2020-12-01 00:29:36,2020-12-01 00:31:19,1,.52,1,N,237,141,1,3.5,0.5,0.5,1.82,0,0.3,9.12,2.5 -1,2020-12-01 00:20:26,2020-12-01 00:23:06,0,1.10,1,N,90,230,2,5,3,0.5,0,0,0.3,8.8,2.5 -2,2020-12-01 00:19:55,2020-12-01 00:28:26,1,2.22,1,N,230,239,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 -2,2020-12-01 00:25:05,2020-12-01 00:47:44,1,18.75,2,N,132,262,1,52,0,0.5,20,6.12,0.3,81.42,2.5 -2,2020-12-01 00:12:55,2020-12-01 00:31:15,1,8.06,1,N,75,88,1,25,0.5,0.5,5.76,0,0.3,34.56,2.5 -1,2020-12-01 00:57:19,2020-12-01 01:10:52,1,3.70,1,N,148,49,2,13.5,3,0.5,0,0,0.3,17.3,2.5 -2,2020-12-01 00:00:41,2020-12-01 00:06:19,1,2.52,1,N,140,137,2,9,0.5,0.5,0,0,0.3,12.8,2.5 -2,2020-12-01 00:14:43,2020-12-01 00:38:13,2,10.05,1,N,132,225,1,30,0.5,0.5,6.26,0,0.3,37.56,0 -2,2020-12-01 00:14:08,2020-12-01 00:21:20,1,2.41,1,N,48,107,2,9,0.5,0.5,0,0,0.3,12.8,2.5 -2,2020-12-01 00:31:55,2020-12-01 00:36:09,1,1.37,1,N,68,230,2,6,0.5,0.5,0,0,0.3,9.8,2.5 -1,2020-12-01 00:08:49,2020-12-01 00:16:59,1,2.40,1,N,263,238,1,9,3.5,0.5,2.65,0,0.3,15.95,2.5 -1,2020-12-01 00:17:41,2020-12-01 00:24:29,1,2.40,1,N,233,236,1,9,3,0.5,2.55,0,0.3,15.35,2.5 -2,2020-12-01 00:19:22,2020-12-01 00:46:33,1,17.78,2,N,132,229,1,52,0,0.5,5,0,0.3,60.3,2.5 -1,2020-12-01 00:48:48,2020-12-01 01:05:24,1,4.90,1,N,170,151,2,16.5,3,0.5,0,0,0.3,20.3,2.5 -1,2020-12-01 00:06:54,2020-12-01 00:12:12,1,1.70,1,N,107,229,1,7,3,0.5,2.15,0,0.3,12.95,2.5 -1,2020-12-01 00:13:41,2020-12-01 00:19:20,1,2.00,1,N,229,263,1,7.5,3,0.5,2.25,0,0.3,13.55,2.5 -2,2020-12-01 00:01:54,2020-12-01 00:12:12,1,1.93,1,N,236,143,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 -1,2020-12-01 00:04:17,2020-12-01 00:04:32,1,.00,1,N,42,42,2,2.5,0.5,0.5,0,0,0.3,3.8,0 -1,2020-12-01 00:39:28,2020-12-01 00:47:45,1,2.80,1,N,10,197,1,10.5,0.5,0.5,7,0,0.3,18.8,0 -2,2020-12-01 00:02:55,2020-12-01 00:12:05,1,2.20,1,N,237,143,1,9.5,0.5,0.5,2.66,0,0.3,15.96,2.5 -2,2020-12-01 00:00:51,2020-12-01 00:13:34,2,5.10,1,N,137,80,1,16.5,0.5,0.5,5.08,0,0.3,25.38,2.5 -2,2020-12-01 00:24:06,2020-12-01 00:48:07,2,17.95,2,N,132,114,1,52,0,0.5,16.59,0,0.3,71.89,2.5 -2,2020-12-01 00:08:12,2020-12-01 00:30:28,1,6.77,1,N,237,102,1,22.5,0.5,0.5,5.26,0,0.3,31.56,2.5 -2,2020-12-01 00:08:51,2020-12-01 00:25:34,1,4.67,1,N,237,223,2,16.5,0.5,0.5,0,0,0.3,20.3,2.5 -2,2020-12-01 00:46:04,2020-12-01 01:03:51,1,8.76,1,N,132,95,2,26,0.5,0.5,0,0,0.3,27.3,0 -1,2020-12-01 00:38:59,2020-12-01 01:00:47,1,6.90,1,N,137,37,1,22.5,3,0.5,2,0,0.3,28.3,2.5 -2,2020-11-30 23:52:54,2020-11-30 23:54:54,1,.70,1,N,163,50,2,4,0.5,0.5,0,0,0.3,7.8,2.5 -2,2020-12-01 00:05:53,2020-12-01 00:08:38,1,.70,1,N,48,50,1,4.5,0.5,0.5,1,0,0.3,9.3,2.5 -2,2020-12-01 00:44:36,2020-12-01 00:46:47,1,1.00,1,N,249,90,1,5,0.5,0.5,1.76,0,0.3,10.56,2.5 -2,2020-12-01 00:41:19,2020-12-01 01:03:03,1,10.82,1,N,138,142,1,32,0.5,0.5,8.38,6.12,0.3,50.3,2.5 -2,2020-12-01 00:50:29,2020-12-01 01:03:28,1,3.75,1,N,237,211,2,13.5,0.5,0.5,0,0,0.3,17.3,2.5 -2,2020-12-01 00:48:34,2020-12-01 01:03:17,1,3.44,1,N,211,52,1,14,0.5,0.5,3.56,0,0.3,21.36,2.5 -2,2020-12-01 00:04:05,2020-12-01 00:10:59,1,2.38,1,N,142,68,1,8.5,0.5,0.5,2,0,0.3,14.3,2.5 -2,2020-12-01 00:35:23,2020-12-01 00:40:41,1,1.35,1,N,264,142,1,6,0.5,0.5,1.96,0,0.3,11.76,2.5 -2,2020-12-01 00:12:21,2020-12-01 00:20:05,6,.55,1,N,41,74,2,5,0.5,0.5,0,0,0.3,6.3,0 -2,2020-12-01 00:09:09,2020-12-01 00:25:10,4,4.09,1,N,137,239,1,15,0.5,0.5,3.76,0,0.3,22.56,2.5 -2,2020-12-01 00:40:06,2020-12-01 01:05:42,6,18.78,2,N,132,114,1,52,0,0.5,13.82,0,0.3,69.12,2.5 -2,2020-12-01 00:20:47,2020-12-01 00:33:57,1,4.96,1,N,75,7,1,15.5,0.5,0.5,0,0,0.3,19.3,2.5 -2,2020-12-01 00:20:21,2020-12-01 00:34:41,1,5.13,1,N,170,260,2,16.5,0.5,0.5,0,0,0.3,20.3,2.5 -2,2020-12-01 00:16:42,2020-12-01 00:48:28,1,10.50,1,N,138,114,1,31.5,0.5,0.5,10.59,0,0.3,45.89,2.5 -1,2020-12-01 00:43:56,2020-12-01 00:59:45,1,6.90,1,N,132,197,2,21.5,0.5,0.5,0,0,0.3,22.8,0 -1,2020-12-01 00:07:50,2020-12-01 00:13:30,1,2.50,1,N,233,263,2,8.5,3,0.5,0,0,0.3,12.3,2.5 -1,2020-12-01 00:56:41,2020-12-01 01:02:57,1,1.60,1,N,230,141,1,7.5,3,0.5,2.25,0,0.3,13.55,2.5 -2,2020-12-01 00:54:45,2020-12-01 01:01:28,4,3.20,1,N,132,10,1,10.5,0.5,0.5,2.95,0,0.3,14.75,0 -1,2020-12-01 00:17:14,2020-12-01 00:24:26,1,1.50,1,N,166,238,1,7.5,3,0.5,1,0,0.3,12.3,2.5 -2,2020-12-01 00:37:15,2020-12-01 00:42:39,1,1.39,1,N,229,262,2,6.5,0.5,0.5,0,0,0.3,10.3,2.5 -2,2020-12-01 00:47:28,2020-12-01 00:55:37,1,2.94,1,N,140,107,2,10.5,0.5,0.5,0,0,0.3,14.3,2.5 -1,2020-12-01 00:43:06,2020-12-01 00:52:45,2,3.50,1,N,132,10,1,12.5,0.5,0.5,0,0,0.3,13.8,0 -2,2020-12-01 00:33:32,2020-12-01 00:38:07,1,.77,1,N,68,234,1,5.5,0.5,0.5,1.5,0,0.3,10.8,2.5 -2,2020-11-30 23:59:22,2020-12-01 00:13:53,2,4.48,1,N,87,68,1,15,0.5,0.5,2,0,0.3,20.8,2.5 -2,2020-12-01 00:47:29,2020-12-01 00:51:53,2,1.43,1,N,162,107,1,6,0.5,0.5,1.96,0,0.3,11.76,2.5 -2,2020-12-01 00:31:04,2020-12-01 00:38:21,1,2.25,1,N,263,74,2,8.5,0.5,0.5,0,0,0.3,12.3,2.5 -2,2020-11-30 23:49:37,2020-11-30 23:59:04,1,2.56,1,N,262,151,1,10,0.5,0.5,2,0,0.3,15.8,2.5 -2,2020-12-01 00:04:47,2020-12-01 00:08:47,1,1.05,1,N,238,24,1,5.5,0.5,0.5,2.04,0,0.3,8.84,0 -2,2020-12-01 00:21:56,2020-12-01 00:39:04,1,8.46,1,N,231,193,2,24.5,0.5,0.5,0,0,0.3,28.3,2.5 -1,2020-12-01 00:31:03,2020-12-01 00:39:47,1,1.70,1,N,249,231,1,8.5,3,0.5,2.45,0,0.3,14.75,2.5 -1,2020-12-01 00:44:53,2020-12-01 00:56:32,1,4.20,1,N,125,142,1,13.5,3,0.5,3.45,0,0.3,20.75,2.5 -2,2020-12-01 00:23:04,2020-12-01 00:45:51,3,17.76,2,N,132,162,1,52,0,0.5,10.8,6.12,0.3,72.22,2.5 -1,2020-12-01 00:41:35,2020-12-01 00:52:03,1,2.40,1,N,161,143,1,10,3,0.5,2.75,0,0.3,16.55,2.5 -1,2020-12-01 00:57:28,2020-12-01 01:16:25,1,5.10,1,N,143,168,2,18,3,0.5,0,0,0.3,21.8,2.5 -2,2020-12-01 00:12:06,2020-12-01 00:37:49,1,11.21,1,N,48,89,4,-32,-0.5,-0.5,0,0,-0.3,-35.8,-2.5 -2,2020-12-01 00:12:06,2020-12-01 00:37:49,1,11.21,1,N,48,188,2,32,0.5,0.5,0,0,0.3,35.8,2.5 -2,2020-12-01 00:43:56,2020-12-01 01:05:51,2,16.77,2,N,132,170,1,52,0,0.5,10,6.12,0.3,71.42,2.5 -2,2020-12-01 00:24:15,2020-12-01 00:27:34,1,.80,1,N,234,90,1,5,0.5,0.5,1.76,0,0.3,10.56,2.5 -2,2020-12-01 00:53:14,2020-12-01 00:59:37,1,1.52,1,N,230,143,1,7.5,0.5,0.5,2.26,0,0.3,13.56,2.5 -2,2020-12-01 00:17:15,2020-12-01 00:38:57,1,3.56,1,N,68,232,2,16.5,0.5,0.5,0,0,0.3,20.3,2.5 -2,2020-12-01 00:45:58,2020-12-01 00:58:43,1,3.83,1,N,79,48,1,13.5,0.5,0.5,2.7,0,0.3,20,2.5 -1,2020-12-01 00:00:26,2020-12-01 00:07:25,1,2.90,1,N,137,262,1,9.5,3,0.5,2.66,0,0.3,15.96,2.5 -1,2020-12-01 00:03:45,2020-12-01 00:20:00,1,6.90,1,N,170,168,2,20.5,3,0.5,0,0,0.3,24.3,2.5 -1,2020-12-01 00:06:04,2020-12-01 00:23:52,1,10.30,1,N,138,162,2,30,3,0.5,0,6.12,0.3,39.92,2.5 -2,2020-11-30 23:36:21,2020-11-30 23:57:45,1,10.07,1,N,138,142,1,29.5,0.5,0.5,7.88,6.12,0.3,47.3,2.5 diff --git a/tests/integration/test_cluster_copier/prepare_taxi_data.py b/tests/integration/test_cluster_copier/prepare_taxi_data.py deleted file mode 100644 index 7c55147273b..00000000000 --- a/tests/integration/test_cluster_copier/prepare_taxi_data.py +++ /dev/null @@ -1,57 +0,0 @@ -import os -import sys -import logging -import subprocess - -logging.basicConfig(stream=sys.stdout, level=logging.DEBUG) - -CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) -sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) - - -FIRST_DATA = [ -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-12.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-11.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-10.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-09.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-08.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-07.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-06.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-05.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-04.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-03.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-02.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-01.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-12.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-11.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-10.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-09.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-08.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-07.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-06.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-05.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-04.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-03.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-02.csv", -"https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2019-01.csv", -] - -def prepare_data(): - first_path = os.path.join(CURRENT_TEST_DIR, "data/first.csv") - # if os.path.exists(first_path): - # return - - # Create file - with open(first_path, 'w') as _: - pass - - print("{} created".format(first_path)) - - for url in FIRST_DATA[:1]: - print(url) - subprocess.call("wget -O - {} 2> /dev/null | tail -n +3 | head -n 100 >> {}".format(url, first_path), shell = True) - logging.info("Successfully downloaded data from {}".format(url)) - - -if __name__ == "__main__": - prepare_data() \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 57f9d150c8d..6ec875c9eee 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -2,14 +2,14 @@ import os import random import sys import time -from contextlib import contextmanager - -import docker import kazoo import pytest +from contextlib import contextmanager from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +import docker + CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) diff --git a/tests/integration/test_cluster_copier/test_taxi_data.py b/tests/integration/test_cluster_copier/test_taxi_data.py deleted file mode 100644 index 81c1fe9fb25..00000000000 --- a/tests/integration/test_cluster_copier/test_taxi_data.py +++ /dev/null @@ -1,217 +0,0 @@ -import os -import sys -import time -import logging -import subprocess -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -import docker - -logging.basicConfig( - stream=sys.stdout, - level=logging.DEBUG, - format='%(asctime)s.%(msecs)03d %(levelname)s %(module)s - %(funcName)s: %(message)s', - datefmt='%Y-%m-%d %H:%M:%S', -) - -CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) -sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) - - -COPYING_FAIL_PROBABILITY = 0.33 -MOVING_FAIL_PROBABILITY = 0.1 -cluster = None - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster = ClickHouseCluster(__file__) - - for name in ["first", "second", "third"]: - cluster.add_instance(name, - main_configs=["configs_taxi/conf.d/clusters.xml", "configs_taxi/conf.d/ddl.xml"], user_configs=["configs_taxi/users.xml"], - with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) - - cluster.start() - yield cluster - - finally: - pass - cluster.shutdown() - - -DATA_COMMANDS = [ -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-12.csv 2> /dev/null | tail -n +3 | time,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-11.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-10.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-09.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-08.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-07.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-06.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-05.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-04.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-03.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-02.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-01.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', - -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-12.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-11.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-10.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-09.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-08.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-07.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-06.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-05.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-04.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-03.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-02.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2017-01.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount) FORMAT CSV"', - -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-12.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-11.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-10.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-09.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-08.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-07.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-06.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-05.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-04.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-03.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-02.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"', -'wget -O - https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2016-01.csv 2> /dev/null | tail -n +3 | clickhouse-client --query="INSERT INTO dailyhistory.yellow_tripdata(vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,junk1,junk2) FORMAT CSV"' -] - - -class Task: - def __init__(self, cluster): - self.cluster = cluster - self.zk_task_path = '/clickhouse-copier/task' - self.container_task_file = "/task_taxi_data.xml" - - for instance_name, _ in cluster.instances.items(): - instance = cluster.instances[instance_name] - instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_taxi_data.xml'), self.container_task_file) - print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) - - - def start(self): - instance = cluster.instances['first'] - - # daily partition database - instance.query("CREATE DATABASE dailyhistory on cluster events;") - instance.query("""CREATE TABLE dailyhistory.yellow_tripdata_staging ON CLUSTER events - ( - id UUID DEFAULT generateUUIDv4(), vendor_id String, tpep_pickup_datetime DateTime('UTC'), tpep_dropoff_datetime DateTime('UTC'), - passenger_count Nullable(Float64), trip_distance String, pickup_longitude Float64, pickup_latitude Float64, - rate_code_id String, store_and_fwd_flag String, dropoff_longitude Float64, dropoff_latitude Float64, - payment_type String, fare_amount String, extra String, mta_tax String, tip_amount String, tolls_amount String, - improvement_surcharge String, total_amount String, pickup_location_id String, dropoff_location_id String, congestion_surcharge String, - junk1 String, junk2 String - ) - Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (toYYYYMMDD(tpep_pickup_datetime))""") - instance.query("CREATE TABLE dailyhistory.yellow_tripdata ON CLUSTER events AS dailyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'dailyhistory', yellow_tripdata_staging, sipHash64(id) % 3);") - - # monthly partition database - instance.query("create database monthlyhistory on cluster events;") - instance.query("""CREATE TABLE monthlyhistory.yellow_tripdata_staging ON CLUSTER events - ( - id UUID DEFAULT generateUUIDv4(), vendor_id String, tpep_pickup_datetime DateTime('UTC'), tpep_dropoff_datetime DateTime('UTC'), - passenger_count Nullable(Float64), trip_distance String, pickup_longitude Float64, pickup_latitude Float64, rate_code_id String, - store_and_fwd_flag String, dropoff_longitude Float64, dropoff_latitude Float64, payment_type String, fare_amount String, - extra String, mta_tax String, tip_amount String, tolls_amount String, improvement_surcharge String, total_amount String, - pickup_location_id String, dropoff_location_id String, congestion_surcharge String, junk1 String, junk2 String - ) - Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) PARTITION BY (pickup_location_id, toYYYYMM(tpep_pickup_datetime))""") - instance.query("CREATE TABLE monthlyhistory.yellow_tripdata ON CLUSTER events AS monthlyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'monthlyhistory', yellow_tripdata_staging, sipHash64(id) % 3);") - - - print("Inserting in container") - - first_query = """INSERT INTO dailyhistory.yellow_tripdata( - vendor_id,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance, - rate_code_id,store_and_fwd_flag,pickup_location_id,dropoff_location_id,payment_type, - fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount,congestion_surcharge) FORMAT CSV""" - instance.exec_in_container(['bash', '-c', 'cat /usr/share/clickhouse-external-data/first.csv | /usr/bin/clickhouse client --query="{}"'.format(first_query)], privileged=True) - - print("Insert completed") - - - def check(self): - instance = cluster.instances["first"] - a = TSV(instance.query("SELECT count() from dailyhistory.yellow_tripdata")) - b = TSV(instance.query("SELECT count() from monthlyhistory.yellow_tripdata")) - assert a == b, "Distributed tables" - - for instance_name, instance in cluster.instances.items(): - instance = cluster.instances[instance_name] - a = instance.query("SELECT count() from dailyhistory.yellow_tripdata_staging") - b = instance.query("SELECT count() from monthlyhistory.yellow_tripdata_staging") - assert a == b, "MergeTree tables on each shard" - - a = TSV(instance.query("SELECT sipHash64(*) from dailyhistory.yellow_tripdata_staging ORDER BY id")) - b = TSV(instance.query("SELECT sipHash64(*) from monthlyhistory.yellow_tripdata_staging ORDER BY id")) - - assert a == b, "Data on each shard" - - - - -def execute_task(task, cmd_options): - task.start() - - zk = cluster.get_kazoo_client('zoo1') - print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) - - # Run cluster-copier processes on each node - docker_api = docker.from_env().api - copiers_exec_ids = [] - - cmd = ['/usr/bin/clickhouse', 'copier', - '--config', '/etc/clickhouse-server/config-copier.xml', - '--task-path', task.zk_task_path, - '--task-file', task.container_task_file, - '--task-upload-force', 'true', - '--base-dir', '/var/log/clickhouse-server/copier'] - cmd += cmd_options - - print(cmd) - - for instance_name, instance in cluster.instances.items(): - instance = cluster.instances[instance_name] - container = instance.get_docker_handle() - instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_taxi/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") - logging.info("Copied copier config to {}".format(instance.name)) - exec_id = docker_api.exec_create(container.id, cmd, stderr=True) - output = docker_api.exec_start(exec_id).decode('utf8') - logging.info(output) - copiers_exec_ids.append(exec_id) - logging.info("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) - - # time.sleep(1000) - - # Wait for copiers stopping and check their return codes - for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): - while True: - res = docker_api.exec_inspect(exec_id) - if not res['Running']: - break - time.sleep(1) - - assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) - - try: - task.check() - finally: - zk.delete(task.zk_task_path, recursive=True) - - -# Tests -@pytest.mark.timeout(1200) -def test1(started_cluster): - execute_task(Task(started_cluster), []) - diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py new file mode 100644 index 00000000000..b30e63d9fb8 --- /dev/null +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -0,0 +1,229 @@ +import os +import sys +import time +import logging +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +import docker + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) + +cluster = None + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster = ClickHouseCluster(__file__) + + for name in ["first", "second", "third"]: + cluster.add_instance(name, + main_configs=["configs_taxi/conf.d/clusters.xml", "configs_taxi/conf.d/ddl.xml"], user_configs=["configs_taxi/users.xml"], + with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) + + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +class Task: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path = '/clickhouse-copier/task' + self.container_task_file = "/task_taxi_data.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_taxi_data.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + + + def start(self): + instance = cluster.instances['first'] + + # daily partition database + instance.query("CREATE DATABASE dailyhistory on cluster events;") + instance.query("""CREATE TABLE dailyhistory.yellow_tripdata_staging ON CLUSTER events + ( + id UUID DEFAULT generateUUIDv4(), + vendor_id String, + tpep_pickup_datetime DateTime('UTC'), + tpep_dropoff_datetime DateTime('UTC'), + passenger_count Nullable(Float64), + trip_distance String, + pickup_longitude Float64, + pickup_latitude Float64, + rate_code_id String, + store_and_fwd_flag String, + dropoff_longitude Float64, + dropoff_latitude Float64, + payment_type String, + fare_amount String, + extra String, + mta_tax String, + tip_amount String, + tolls_amount String, + improvement_surcharge String, + total_amount String, + pickup_location_id String, + dropoff_location_id String, + congestion_surcharge String, + junk1 String, junk2 String + ) + Engine = ReplacingMergeTree() + PRIMARY KEY (tpep_pickup_datetime, id) + ORDER BY (tpep_pickup_datetime, id) + PARTITION BY (toYYYYMMDD(tpep_pickup_datetime))""") + + instance.query("""CREATE TABLE dailyhistory.yellow_tripdata + ON CLUSTER events + AS dailyhistory.yellow_tripdata_staging + ENGINE = Distributed('events', 'dailyhistory', yellow_tripdata_staging, sipHash64(id) % 3);""") + + instance.query("""INSERT INTO dailyhistory.yellow_tripdata + SELECT * FROM generateRandom( + 'id UUID DEFAULT generateUUIDv4(), + vendor_id String, + tpep_pickup_datetime DateTime(\\'UTC\\'), + tpep_dropoff_datetime DateTime(\\'UTC\\'), + passenger_count Nullable(Float64), + trip_distance String, + pickup_longitude Float64, + pickup_latitude Float64, + rate_code_id String, + store_and_fwd_flag String, + dropoff_longitude Float64, + dropoff_latitude Float64, + payment_type String, + fare_amount String, + extra String, + mta_tax String, + tip_amount String, + tolls_amount String, + improvement_surcharge String, + total_amount String, + pickup_location_id String, + dropoff_location_id String, + congestion_surcharge String, + junk1 String, + junk2 String', + 1, 10, 2) LIMIT 50;""") + + # monthly partition database + instance.query("create database monthlyhistory on cluster events;") + instance.query("""CREATE TABLE monthlyhistory.yellow_tripdata_staging ON CLUSTER events + ( + id UUID DEFAULT generateUUIDv4(), + vendor_id String, + tpep_pickup_datetime DateTime('UTC'), + tpep_dropoff_datetime DateTime('UTC'), + passenger_count Nullable(Float64), + trip_distance String, + pickup_longitude Float64, + pickup_latitude Float64, + rate_code_id String, + store_and_fwd_flag String, + dropoff_longitude Float64, + dropoff_latitude Float64, + payment_type String, + fare_amount String, + extra String, + mta_tax String, + tip_amount String, + tolls_amount String, + improvement_surcharge String, + total_amount String, + pickup_location_id String, + dropoff_location_id String, + congestion_surcharge String, + junk1 String, + junk2 String + ) + Engine = ReplacingMergeTree() + PRIMARY KEY (tpep_pickup_datetime, id) + ORDER BY (tpep_pickup_datetime, id) + PARTITION BY (pickup_location_id, toYYYYMM(tpep_pickup_datetime))""") + + instance.query("""CREATE TABLE monthlyhistory.yellow_tripdata + ON CLUSTER events + AS monthlyhistory.yellow_tripdata_staging + ENGINE = Distributed('events', 'monthlyhistory', yellow_tripdata_staging, sipHash64(id) % 3);""") + + + def check(self): + instance = cluster.instances["first"] + a = TSV(instance.query("SELECT count() from dailyhistory.yellow_tripdata")) + b = TSV(instance.query("SELECT count() from monthlyhistory.yellow_tripdata")) + assert a == b, "Distributed tables" + + for instance_name, instance in cluster.instances.items(): + instance = cluster.instances[instance_name] + a = instance.query("SELECT count() from dailyhistory.yellow_tripdata_staging") + b = instance.query("SELECT count() from monthlyhistory.yellow_tripdata_staging") + assert a == b, "MergeTree tables on each shard" + + a = TSV(instance.query("SELECT sipHash64(*) from dailyhistory.yellow_tripdata_staging ORDER BY id")) + b = TSV(instance.query("SELECT sipHash64(*) from monthlyhistory.yellow_tripdata_staging ORDER BY id")) + + assert a == b, "Data on each shard" + + +def execute_task(task, cmd_options): + task.start() + + zk = cluster.get_kazoo_client('zoo1') + print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) + + # Run cluster-copier processes on each node + docker_api = docker.from_env().api + copiers_exec_ids = [] + + cmd = ['/usr/bin/clickhouse', 'copier', + '--config', '/etc/clickhouse-server/config-copier.xml', + '--task-path', task.zk_task_path, + '--task-file', task.container_task_file, + '--task-upload-force', 'true', + '--base-dir', '/var/log/clickhouse-server/copier'] + cmd += cmd_options + + print(cmd) + + for instance_name, instance in cluster.instances.items(): + instance = cluster.instances[instance_name] + container = instance.get_docker_handle() + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_taxi/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + logging.info("Copied copier config to {}".format(instance.name)) + exec_id = docker_api.exec_create(container.id, cmd, stderr=True) + output = docker_api.exec_start(exec_id).decode('utf8') + logging.info(output) + copiers_exec_ids.append(exec_id) + logging.info("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) + + # time.sleep(1000) + + # Wait for copiers stopping and check their return codes + for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + while True: + res = docker_api.exec_inspect(exec_id) + if not res['Running']: + break + time.sleep(1) + + assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) + + try: + task.check() + finally: + zk.delete(task.zk_task_path, recursive=True) + + +# Tests +@pytest.mark.timeout(600) +def test(started_cluster): + execute_task(Task(started_cluster), []) + diff --git a/tests/integration/test_cluster_copier/test_tables_with_different_schema.py b/tests/integration/test_cluster_copier/test_two_nodes.py similarity index 97% rename from tests/integration/test_cluster_copier/test_tables_with_different_schema.py rename to tests/integration/test_cluster_copier/test_two_nodes.py index 8e98387b5e9..0aa09ca8c12 100644 --- a/tests/integration/test_cluster_copier/test_tables_with_different_schema.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -2,7 +2,6 @@ import os import sys import time import logging -import subprocess import pytest from helpers.cluster import ClickHouseCluster @@ -13,9 +12,6 @@ import docker CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) - -COPYING_FAIL_PROBABILITY = 0.33 -MOVING_FAIL_PROBABILITY = 0.1 cluster = None @@ -25,7 +21,7 @@ def started_cluster(): try: cluster = ClickHouseCluster(__file__) - for name in ["first", "second", "third"]: + for name in ["first", "second"]: cluster.add_instance(name, main_configs=["configs_two_nodes/conf.d/clusters.xml", "configs_two_nodes/conf.d/ddl.xml"], user_configs=["configs_two_nodes/users.xml"], with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) @@ -34,7 +30,6 @@ def started_cluster(): yield cluster finally: - pass cluster.shutdown() # Will copy table from `first` node to `second` @@ -78,7 +73,7 @@ class TaskWithDifferentSchema: first.query("""INSERT INTO db_different_schema.source SELECT * FROM generateRandom( 'Column1 String, Column2 UInt32, Column3 Date, Column4 DateTime, Column5 UInt16, Column6 String, Column7 String, Column8 String, Column9 String, Column10 String, - Column11 String, Column12 Decimal(3, 1), Column13 DateTime, Column14 UInt16', 1, 10, 2) LIMIT 100;""") + Column11 String, Column12 Decimal(3, 1), Column13 DateTime, Column14 UInt16', 1, 10, 2) LIMIT 50;""") second = cluster.instances["second"] second.query("CREATE DATABASE db_different_schema;") @@ -154,7 +149,7 @@ class TaskTTL: first.query("""INSERT INTO db_ttl_columns.source SELECT * FROM generateRandom( 'Column1 String, Column2 UInt32, Column3 Date, Column4 DateTime, Column5 UInt16, - Column6 String, Column7 Decimal(3, 1), Column8 Tuple(Float64, Float64)', 1, 10, 2) LIMIT 100;""") + Column6 String, Column7 Decimal(3, 1), Column8 Tuple(Float64, Float64)', 1, 10, 2) LIMIT 50;""") second = cluster.instances["second"] second.query("CREATE DATABASE db_ttl_columns;") @@ -238,14 +233,12 @@ def execute_task(task, cmd_options): zk.delete(task.zk_task_path, recursive=True) - - # Tests -@pytest.mark.timeout(1200) +@pytest.mark.timeout(600) def test_different_schema(started_cluster): execute_task(TaskWithDifferentSchema(started_cluster), []) - +@pytest.mark.timeout(600) def test_ttl_columns(started_cluster): execute_task(TaskTTL(started_cluster), []) \ No newline at end of file From 90226466331038209411be1565caa2910dfde3da Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 02:54:57 +0300 Subject: [PATCH 281/652] support copying of tables with data skipping indices --- programs/copier/ClusterCopier.cpp | 15 ++-- .../test_cluster_copier/task_skip_index.xml | 40 +++++++++++ .../test_cluster_copier/test_two_nodes.py | 72 ++++++++++++++++++- 3 files changed, 120 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_cluster_copier/task_skip_index.xml diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 97f5fd0e38c..08a8749213a 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -694,9 +694,10 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t return TaskStatus::Finished; } -/// Remove column's TTL expression from `CREATE` query -/// This is needed to create internal Distributed table -/// Also it removes MATEREALIZED or ALIAS columns not to copy additional and useless data over the network. +/// This is needed to create internal Distributed table +/// Removes column's TTL expression from `CREATE` query +/// Removes MATEREALIZED or ALIAS columns not to copy additional and useless data over the network. +/// Removes data skipping indices. ASTPtr ClusterCopier::removeAliasMaterializedAndTTLColumnsFromCreateQuery(const ASTPtr & query_ast, bool allow_to_copy_alias_and_materialized_columns) { const ASTs & column_asts = query_ast->as().columns_list->columns->children; @@ -1279,6 +1280,8 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (!limit.empty()) query += " LIMIT " + limit; + query += "FORMAT Native"; + ParserQuery p_query(query.data() + query.size()); const auto & settings = getContext()->getSettingsRef(); @@ -1497,7 +1500,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( ASTPtr query_insert_ast; { String query; - query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES "; + query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " FORMAT Native "; ParserQuery p_query(query.data() + query.size()); const auto & settings = getContext()->getSettingsRef(); @@ -1772,7 +1775,7 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); auto create_query_ast = removeAliasMaterializedAndTTLColumnsFromCreateQuery( - task_shard.current_pull_table_create_query, + task_shard.current_pull_table_create_query, task_table.allow_to_copy_alias_and_materialized_columns); auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); @@ -1876,7 +1879,7 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); - LOG_INFO(log, "Checking shard {} for partition {} existence, executing query: \n {}", + LOG_INFO(log, "Checking shard {} for partition {} existence, executing query: \n {}", task_shard.getDescription(), partition_quoted_name, query_ast->formatForErrorMessage()); auto local_context = Context::createCopy(context); diff --git a/tests/integration/test_cluster_copier/task_skip_index.xml b/tests/integration/test_cluster_copier/task_skip_index.xml new file mode 100644 index 00000000000..24c40c622e3 --- /dev/null +++ b/tests/integration/test_cluster_copier/task_skip_index.xml @@ -0,0 +1,40 @@ + + + + + + false + + first + 9000 + + + + + + false + + second + 9000 + + + + + + 2 + + + + source + db_skip_index + source + + destination + db_skip_index + destination + + ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1) + rand() + + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 0aa09ca8c12..6c29d2e9f2d 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -184,6 +184,71 @@ class TaskTTL: assert a == b, "Data" +class TaskSkipIndex: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path = '/clickhouse-copier/task_skip_index' + self.container_task_file = "/task_skip_index.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_skip_index.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + + def start(self): + first = cluster.instances["first"] + first.query("CREATE DATABASE db_skip_index;") + first.query("""CREATE TABLE db_skip_index.source + ( + Column1 UInt64, + Column2 Int32, + Column3 Date, + Column4 DateTime, + Column5 String, + INDEX a (Column1 * Column2, Column5) TYPE minmax GRANULARITY 3, + INDEX b (Column1 * length(Column5)) TYPE set(1000) GRANULARITY 4 + ) + ENGINE = MergeTree() + PARTITION BY (toYYYYMMDD(Column3), Column3) + PRIMARY KEY (Column1, Column2, Column3) + ORDER BY (Column1, Column2, Column3) + SETTINGS index_granularity = 8192""") + + first.query("""INSERT INTO db_skip_index.source SELECT * FROM generateRandom( + 'Column1 UInt64, Column2 Int32, Column3 Date, Column4 DateTime, Column5 String', 1, 10, 2) LIMIT 100;""") + + second = cluster.instances["second"] + second.query("CREATE DATABASE db_skip_index;") + second.query("""CREATE TABLE db_skip_index.destination + ( + Column1 UInt64, + Column2 Int32, + Column3 Date, + Column4 DateTime, + Column5 String, + INDEX a (Column1 * Column2, Column5) TYPE minmax GRANULARITY 3, + INDEX b (Column1 * length(Column5)) TYPE set(1000) GRANULARITY 4 + ) ENGINE = MergeTree() + PARTITION BY toYYYYMMDD(Column3) + ORDER BY (Column3, Column2, Column1);""") + + print("Preparation completed") + + def check(self): + first = cluster.instances["first"] + second = cluster.instances["second"] + + a = first.query("SELECT count() from db_skip_index.source") + b = second.query("SELECT count() from db_skip_index.destination") + assert a == b, "Count" + + a = TSV(first.query("""SELECT sipHash64(*) from db_skip_index.source + ORDER BY (Column1, Column2, Column3, Column4, Column5)""")) + b = TSV(second.query("""SELECT sipHash64(*) from db_skip_index.destination + ORDER BY (Column1, Column2, Column3, Column4, Column5)""")) + assert a == b, "Data" + + def execute_task(task, cmd_options): task.start() @@ -241,4 +306,9 @@ def test_different_schema(started_cluster): @pytest.mark.timeout(600) def test_ttl_columns(started_cluster): - execute_task(TaskTTL(started_cluster), []) \ No newline at end of file + execute_task(TaskTTL(started_cluster), []) + + +@pytest.mark.timeout(600) +def test_skip_index(started_cluster): + execute_task(TaskSkipIndex(started_cluster), []) \ No newline at end of file From 267319ffebe489ff871e9933c9eef4b93fb0835e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 17:39:03 +0300 Subject: [PATCH 282/652] added test with ttl move to volume --- .../conf.d/storage_configuration.xml | 34 +++++++ .../task_ttl_move_to_volume.xml | 40 +++++++++ .../test_cluster_copier/test_two_nodes.py | 88 ++++++++++++++++++- 3 files changed, 159 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_cluster_copier/configs_two_nodes/conf.d/storage_configuration.xml create mode 100644 tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/storage_configuration.xml b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/storage_configuration.xml new file mode 100644 index 00000000000..07b5c577267 --- /dev/null +++ b/tests/integration/test_cluster_copier/configs_two_nodes/conf.d/storage_configuration.xml @@ -0,0 +1,34 @@ + + + + + + + + /jbod1/ + + + /jbod2/ + + + /external/ + + + + + + + + external + +

+ jbod1 + jbod2 +
+ + + + + + +
diff --git a/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml b/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml new file mode 100644 index 00000000000..68fb65db6d9 --- /dev/null +++ b/tests/integration/test_cluster_copier/task_ttl_move_to_volume.xml @@ -0,0 +1,40 @@ + + + + + + false + + first + 9000 + + + + + + false + + second + 9000 + + + + + + 2 + + + + source + db_move_to_volume + source + + destination + db_move_to_volume + destination + + ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1) TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' SETTINGS storage_policy = 'external_with_jbods' + rand() + + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 6c29d2e9f2d..2da2e1d6190 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -22,11 +22,22 @@ def started_cluster(): cluster = ClickHouseCluster(__file__) for name in ["first", "second"]: - cluster.add_instance(name, - main_configs=["configs_two_nodes/conf.d/clusters.xml", "configs_two_nodes/conf.d/ddl.xml"], user_configs=["configs_two_nodes/users.xml"], + instance = cluster.add_instance(name, + main_configs=[ + "configs_two_nodes/conf.d/clusters.xml", + "configs_two_nodes/conf.d/ddl.xml", + "configs_two_nodes/conf.d/storage_configuration.xml"], + user_configs=["configs_two_nodes/users.xml"], with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) cluster.start() + + for name in ["first", "second"]: + instance = cluster.instances[name] + instance.exec_in_container(['bash', '-c', 'mkdir /jbod1']) + instance.exec_in_container(['bash', '-c', 'mkdir /jbod2']) + instance.exec_in_container(['bash', '-c', 'mkdir /external']) + yield cluster finally: @@ -249,6 +260,72 @@ class TaskSkipIndex: assert a == b, "Data" +class TaskTTLMoveToVolume: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path = '/clickhouse-copier/task_ttl_move_to_volume' + self.container_task_file = "/task_ttl_move_to_volume.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_ttl_move_to_volume.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + + def start(self): + first = cluster.instances["first"] + first.query("CREATE DATABASE db_move_to_volume;") + first.query("""CREATE TABLE db_move_to_volume.source + ( + Column1 UInt64, + Column2 Int32, + Column3 Date, + Column4 DateTime, + Column5 String + ) + ENGINE = MergeTree() + PARTITION BY (toYYYYMMDD(Column3), Column3) + PRIMARY KEY (Column1, Column2, Column3) + ORDER BY (Column1, Column2, Column3) + TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' + SETTINGS storage_policy = 'external_with_jbods';""") + + first.query("""INSERT INTO db_move_to_volume.source SELECT * FROM generateRandom( + 'Column1 UInt64, Column2 Int32, Column3 Date, Column4 DateTime, Column5 String', 1, 10, 2) LIMIT 100;""") + + second = cluster.instances["second"] + second.query("CREATE DATABASE db_move_to_volume;") + second.query("""CREATE TABLE db_move_to_volume.destination + ( + Column1 UInt64, + Column2 Int32, + Column3 Date, + Column4 DateTime, + Column5 String + ) ENGINE = MergeTree() + PARTITION BY toYYYYMMDD(Column3) + ORDER BY (Column3, Column2, Column1) + TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' + SETTINGS storage_policy = 'external_with_jbods';""") + + print("Preparation completed") + + def check(self): + first = cluster.instances["first"] + second = cluster.instances["second"] + + a = first.query("SELECT count() from db_move_to_volume.source") + b = second.query("SELECT count() from db_move_to_volume.destination") + assert a == b, "Count" + + a = TSV(first.query("""SELECT sipHash64(*) from db_move_to_volume.source + ORDER BY (Column1, Column2, Column3, Column4, Column5)""")) + b = TSV(second.query("""SELECT sipHash64(*) from db_move_to_volume.destination + ORDER BY (Column1, Column2, Column3, Column4, Column5)""")) + assert a == b, "Data" + + + + def execute_task(task, cmd_options): task.start() @@ -311,4 +388,9 @@ def test_ttl_columns(started_cluster): @pytest.mark.timeout(600) def test_skip_index(started_cluster): - execute_task(TaskSkipIndex(started_cluster), []) \ No newline at end of file + execute_task(TaskSkipIndex(started_cluster), []) + + +@pytest.mark.timeout(600) +def test_ttl_move_to_volume(started_cluster): + execute_task(TaskTTLMoveToVolume(started_cluster), []) From fa22b41d5a549d6ceac1703cf6ec5953f5c1c333 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 18:46:52 +0300 Subject: [PATCH 283/652] added allow_to_drop_target_partitions with test --- programs/copier/ClusterCopier.cpp | 30 +++++---- programs/copier/TaskTableAndShard.h | 10 +-- .../task_drop_target_partition.xml | 42 +++++++++++++ .../test_cluster_copier/test_two_nodes.py | 61 +++++++++++++++++++ 4 files changed, 127 insertions(+), 16 deletions(-) create mode 100644 tests/integration/test_cluster_copier/task_drop_target_partition.xml diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 08a8749213a..3ff341949d2 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -594,6 +594,22 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent); } + + /// Try to drop destination partition in original table + DatabaseAndTableName original_table = task_table.table_push; + + if (task_table.allow_to_drop_target_partitions) + { + WriteBufferFromOwnString ss; + ss << "ALTER TABLE " << getQuotedTable(original_table) << ((partition_name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") << partition_name; + + UInt64 num_shards_drop_partition = executeQueryOnCluster(task_table.cluster_push, ss.str(), task_cluster->settings_push, ClusterExecutionMode::ON_EACH_SHARD); + + LOG_INFO(log, "Drop partiton {} in original table {} have been executed successfully on {} shards of {}", + partition_name, getQuotedTable(original_table), num_shards_drop_partition, task_table.cluster_push->getShardCount()); + } + + /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { @@ -602,7 +618,6 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ASTPtr query_alter_ast; String query_alter_ast_string; - DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); @@ -633,17 +648,11 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t if (settings_push.replication_alter_partitions_sync == 1) { - LOG_INFO( - log, - "Destination tables {} have been executed alter query successfully on {} shards of {}", - getQuotedTable(task_table.table_push), - num_nodes, - task_table.cluster_push->getShardCount()); + LOG_INFO(log, "Destination tables {} have been executed alter query successfully on {} shards of {}", + getQuotedTable(original_table), num_nodes, task_table.cluster_push->getShardCount()); if (num_nodes != task_table.cluster_push->getShardCount()) - { return TaskStatus::Error; - } } else { @@ -852,9 +861,6 @@ bool ClusterCopier::tryDropPartitionPiece( String query = "ALTER TABLE " + getQuotedTable(helping_table); query += ((task_partition.name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") + task_partition.name + ""; - /// TODO: use this statement after servers will be updated up to 1.1.54310 - // query += " DROP PARTITION ID '" + task_partition.name + "'"; - ClusterPtr & cluster_push = task_table.cluster_push; Settings settings_push = task_cluster->settings_push; diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 569aebae429..77a8db31525 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -36,19 +36,19 @@ struct TaskTable String getPartitionAttachIsDonePath(const String & partition_name) const; - String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; + String getPartitionPiecePath(const String & partition_name, size_t piece_number) const; String getCertainPartitionIsDirtyPath(const String & partition_name) const; - String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const; + String getCertainPartitionPieceIsDirtyPath(const String & partition_name, size_t piece_number) const; String getCertainPartitionIsCleanedPath(const String & partition_name) const; - String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const; + String getCertainPartitionPieceIsCleanedPath(const String & partition_name, size_t piece_number) const; String getCertainPartitionTaskStatusPath(const String & partition_name) const; - String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; + String getCertainPartitionPieceTaskStatusPath(const String & partition_name, size_t piece_number) const; bool isReplicatedTable() const { return is_replicated_table; } @@ -57,6 +57,7 @@ struct TaskTable size_t number_of_splits; bool allow_to_copy_alias_and_materialized_columns{false}; + bool allow_to_drop_target_partitions{false}; String name_in_config; @@ -255,6 +256,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 3); allow_to_copy_alias_and_materialized_columns = config.getBool(table_prefix + "allow_to_copy_alias_and_materialized_columns", false); + allow_to_drop_target_partitions = config.getBool(table_prefix + "allow_to_drop_target_partitions", false); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); diff --git a/tests/integration/test_cluster_copier/task_drop_target_partition.xml b/tests/integration/test_cluster_copier/task_drop_target_partition.xml new file mode 100644 index 00000000000..77605ee2648 --- /dev/null +++ b/tests/integration/test_cluster_copier/task_drop_target_partition.xml @@ -0,0 +1,42 @@ + + + + + + false + + first + 9000 + + + + + + false + + second + 9000 + + + + + + 2 + + + + source + db_drop_target_partition + source + + destination + db_drop_target_partition + destination + + true + + ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1) + rand() + + + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 2da2e1d6190..059b76c35ab 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -324,6 +324,67 @@ class TaskTTLMoveToVolume: assert a == b, "Data" +class TaskDropTargetPartition: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path = '/clickhouse-copier/task_drop_target_partition' + self.container_task_file = "/task_drop_target_partition.xml" + + for instance_name, _ in cluster.instances.items(): + instance = cluster.instances[instance_name] + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_drop_target_partition.xml'), self.container_task_file) + print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) + + def start(self): + first = cluster.instances["first"] + first.query("CREATE DATABASE db_drop_target_partition;") + first.query("""CREATE TABLE db_drop_target_partition.source + ( + Column1 UInt64, + Column2 Int32, + Column3 Date, + Column4 DateTime, + Column5 String + ) + ENGINE = MergeTree() + PARTITION BY (toYYYYMMDD(Column3), Column3) + PRIMARY KEY (Column1, Column2, Column3) + ORDER BY (Column1, Column2, Column3);""") + + first.query("""INSERT INTO db_drop_target_partition.source SELECT * FROM generateRandom( + 'Column1 UInt64, Column2 Int32, Column3 Date, Column4 DateTime, Column5 String', 1, 10, 2) LIMIT 100;""") + + second = cluster.instances["second"] + second.query("CREATE DATABASE db_drop_target_partition;") + second.query("""CREATE TABLE db_drop_target_partition.destination + ( + Column1 UInt64, + Column2 Int32, + Column3 Date, + Column4 DateTime, + Column5 String + ) ENGINE = MergeTree() + PARTITION BY toYYYYMMDD(Column3) + ORDER BY (Column3, Column2, Column1);""") + + # Insert data in target too. It has to be dropped. + first.query("""INSERT INTO db_drop_target_partition.destination SELECT * FROM db_drop_target_partition.source;""") + + print("Preparation completed") + + def check(self): + first = cluster.instances["first"] + second = cluster.instances["second"] + + a = first.query("SELECT count() from db_drop_target_partition.source") + b = second.query("SELECT count() from db_drop_target_partition.destination") + assert a == b, "Count" + + a = TSV(first.query("""SELECT sipHash64(*) from db_drop_target_partition.source + ORDER BY (Column1, Column2, Column3, Column4, Column5)""")) + b = TSV(second.query("""SELECT sipHash64(*) from db_drop_target_partition.destination + ORDER BY (Column1, Column2, Column3, Column4, Column5)""")) + assert a == b, "Data" def execute_task(task, cmd_options): From e9f66a0045134097933a72b037002f1967074366 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 19:06:10 +0300 Subject: [PATCH 284/652] delete alter table optimize --- programs/copier/ClusterCopier.cpp | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 3ff341949d2..20269d9c11f 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -668,30 +668,6 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t if (inject_fault) throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - try - { - String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) - { - query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + - ((partition_name == "'all'") ? " PARTITION ID " : " PARTITION ") + partition_name + " DEDUPLICATE;"; - - LOG_INFO(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_deduplicate_ast_string); - - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_deduplicate_ast_string, - task_cluster->settings_push, - ClusterExecutionMode::ON_EACH_SHARD); - - LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes)); - } - } - catch (...) - { - LOG_INFO(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name); - throw; - } } /// Create node to signal that we finished moving From 6d19dea761f2ec9f61bc8ecd79612608f663928d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Apr 2021 20:51:47 +0300 Subject: [PATCH 285/652] better --- programs/copier/ClusterCopier.cpp | 25 ++++-------------- programs/copier/ClusterCopierApp.cpp | 5 ---- src/Storages/StorageDistributed.cpp | 26 +++++++++---------- .../configs/config-copier.xml | 2 +- .../conf.d/clusters.xml | 0 .../conf.d/ddl.xml | 0 .../config-copier.xml | 2 +- .../users.xml | 0 .../configs_two_nodes/config-copier.xml | 2 +- .../test_cluster_copier/test_three_nodes.py | 4 +-- 10 files changed, 23 insertions(+), 43 deletions(-) rename tests/integration/test_cluster_copier/{configs_taxi => configs_three_nodes}/conf.d/clusters.xml (100%) rename tests/integration/test_cluster_copier/{configs_taxi => configs_three_nodes}/conf.d/ddl.xml (100%) rename tests/integration/test_cluster_copier/{configs_taxi => configs_three_nodes}/config-copier.xml (95%) rename tests/integration/test_cluster_copier/{configs_taxi => configs_three_nodes}/users.xml (100%) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 20269d9c11f..bd99b752b9a 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -22,12 +22,6 @@ namespace ErrorCodes } -std::string wrapWithColor(const std::string & value) -{ - return "\u001b[36;1m" + value + "\u001b[0m"; -} - - void ClusterCopier::init() { auto zookeeper = getContext()->getZooKeeper(); @@ -200,7 +194,6 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, LOG_INFO(log, "Waiting for {} setup jobs", thread_pool.active()); thread_pool.wait(); } - std::cout << "discoverTablePartitions finished" << std::endl; } void ClusterCopier::uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) @@ -624,12 +617,9 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t Settings settings_push = task_cluster->settings_push; ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_NODE; - UInt64 max_successful_executions_per_shard = 0; + if (settings_push.replication_alter_partitions_sync == 1) - { execution_mode = ClusterExecutionMode::ON_EACH_SHARD; - max_successful_executions_per_shard = 1; - } query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + ((partition_name == "'all'") ? " ATTACH PARTITION ID " : " ATTACH PARTITION ") + partition_name + @@ -1100,7 +1090,7 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t InterpreterCreateQuery::prepareOnClusterQuery(create, getContext(), task_table.cluster_push_name); String query = queryToString(create_query_push_ast); - LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); + LOG_INFO(log, "Create destination tables. Query: \n {}", query); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, @@ -1375,7 +1365,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto create_query_push_ast = rewriteCreateQueryStorage(create_query_ast, database_and_table_for_current_piece, new_engine_push_ast); String query = queryToString(create_query_push_ast); - LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); + LOG_INFO(log, "Create destination tables. Query: \n {}", query); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, @@ -1519,11 +1509,6 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext())); input = std::make_shared(pure_input, actions); - - std::cout << "Input:" << std::endl; - std::cout << input->getHeader().dumpStructure() << std::endl; - std::cout << "Output:" << std::endl; - std::cout << output->getHeader().dumpStructure() << std::endl; } /// Fail-fast optimization to abort copying when the current clean state expires @@ -1817,7 +1802,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); - LOG_INFO(log, "Computing destination partition set, executing query: \n {}", wrapWithColor(query)); + LOG_INFO(log, "Computing destination partition set, executing query: \n {}", query); auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); @@ -1973,7 +1958,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( } catch (...) { - LOG_WARNING(log, "An error occured while processing query : \n {}", wrapWithColor(query)); + LOG_WARNING(log, "An error occured while processing query : \n {}", query); tryLogCurrentException(log); } } diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 27e212d29cc..2422f5467a8 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -14,7 +14,6 @@ namespace DB void ClusterCopierApp::initialize(Poco::Util::Application & self) { - std::cout << "CopierApp::initialize" << std::endl; is_help = config().has("help"); if (is_help) return; @@ -105,13 +104,10 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) void ClusterCopierApp::mainImpl() { - std::cout << "ClusterCopierApp::mainImpl()" << std::endl; StatusFile status_file(process_path + "/status", StatusFile::write_full_info); ThreadStatus thread_status; auto * log = &logger(); - log->setLevel(6); /// Information - std::cout << log->getLevel() << std::endl; LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::getVersionRevision()); SharedContextHolder shared_context = Context::createShared(); @@ -138,7 +134,6 @@ void ClusterCopierApp::mainImpl() /// Initialize query scope just in case. CurrentThread::QueryScope query_scope(context); - std::cout << "Will construct copier" << std::endl; auto copier = std::make_unique(task_path, host_id, default_database, context, log); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 192765e33ea..366c3ad9e17 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -497,14 +497,14 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query); if (optimized_cluster) { - LOG_TRACE(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", + LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); cluster = optimized_cluster; query_info.optimized_cluster = cluster; } else { - LOG_TRACE(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", + LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", has_sharding_key ? "" : " (no sharding key)"); } } @@ -536,7 +536,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( auto stage = getOptimizedQueryProcessingStage(query_info, settings.extremes, sharding_key_block); if (stage) { - LOG_TRACE(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); + LOG_DEBUG(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); return *stage; } } @@ -789,7 +789,7 @@ void StorageDistributed::startup() if (inc > file_names_increment.value) file_names_increment.value.store(inc); } - LOG_TRACE(log, "Auto-increment is {}", file_names_increment.value); + LOG_DEBUG(log, "Auto-increment is {}", file_names_increment.value); } @@ -799,9 +799,9 @@ void StorageDistributed::shutdown() std::lock_guard lock(cluster_nodes_mutex); - LOG_TRACE(log, "Joining background threads for async INSERT"); + LOG_DEBUG(log, "Joining background threads for async INSERT"); cluster_nodes_data.clear(); - LOG_TRACE(log, "Background threads for async INSERT joined"); + LOG_DEBUG(log, "Background threads for async INSERT joined"); } void StorageDistributed::drop() { @@ -819,13 +819,13 @@ void StorageDistributed::drop() if (relative_data_path.empty()) return; - LOG_TRACE(log, "Removing pending blocks for async INSERT from filesystem on DROP TABLE"); + LOG_DEBUG(log, "Removing pending blocks for async INSERT from filesystem on DROP TABLE"); auto disks = data_volume->getDisks(); for (const auto & disk : disks) disk->removeRecursive(relative_data_path); - LOG_TRACE(log, "Removed"); + LOG_DEBUG(log, "Removed"); } Strings StorageDistributed::getDataPaths() const @@ -845,7 +845,7 @@ void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, Co { std::lock_guard lock(cluster_nodes_mutex); - LOG_TRACE(log, "Removing pending blocks for async INSERT from filesystem on TRUNCATE TABLE"); + LOG_DEBUG(log, "Removing pending blocks for async INSERT from filesystem on TRUNCATE TABLE"); for (auto it = cluster_nodes_data.begin(); it != cluster_nodes_data.end();) { @@ -853,7 +853,7 @@ void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, Co it = cluster_nodes_data.erase(it); } - LOG_TRACE(log, "Removed"); + LOG_DEBUG(log, "Removed"); } StoragePolicyPtr StorageDistributed::getStoragePolicy() const @@ -881,7 +881,7 @@ void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) if (std::filesystem::is_empty(dir_path)) { - LOG_TRACE(log, "Removing {} (used for async INSERT into Distributed)", dir_path.string()); + LOG_DEBUG(log, "Removing {} (used for async INSERT into Distributed)", dir_path.string()); /// Will be created by DistributedBlockOutputStream on demand. std::filesystem::remove(dir_path); } @@ -1040,7 +1040,7 @@ ClusterPtr StorageDistributed::skipUnusedShards( if (!limit) { - LOG_TRACE(log, + LOG_DEBUG(log, "Number of values for sharding key exceeds optimize_skip_unused_shards_limit={}, " "try to increase it, but note that this may increase query processing time.", local_context->getSettingsRef().optimize_skip_unused_shards_limit); @@ -1138,7 +1138,7 @@ void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) disk->moveDirectory(relative_data_path, new_path_to_table_data); auto new_path = disk->getPath() + new_path_to_table_data; - LOG_TRACE(log, "Updating path to {}", new_path); + LOG_DEBUG(log, "Updating path to {}", new_path); std::lock_guard lock(cluster_nodes_mutex); for (auto & node : cluster_nodes_data) diff --git a/tests/integration/test_cluster_copier/configs/config-copier.xml b/tests/integration/test_cluster_copier/configs/config-copier.xml index 12640034104..6db67efed6f 100644 --- a/tests/integration/test_cluster_copier/configs/config-copier.xml +++ b/tests/integration/test_cluster_copier/configs/config-copier.xml @@ -1,6 +1,6 @@ - trace + information /var/log/clickhouse-server/copier/log.log /var/log/clickhouse-server/copier/log.err.log 1000M diff --git a/tests/integration/test_cluster_copier/configs_taxi/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/clusters.xml similarity index 100% rename from tests/integration/test_cluster_copier/configs_taxi/conf.d/clusters.xml rename to tests/integration/test_cluster_copier/configs_three_nodes/conf.d/clusters.xml diff --git a/tests/integration/test_cluster_copier/configs_taxi/conf.d/ddl.xml b/tests/integration/test_cluster_copier/configs_three_nodes/conf.d/ddl.xml similarity index 100% rename from tests/integration/test_cluster_copier/configs_taxi/conf.d/ddl.xml rename to tests/integration/test_cluster_copier/configs_three_nodes/conf.d/ddl.xml diff --git a/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml b/tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml similarity index 95% rename from tests/integration/test_cluster_copier/configs_taxi/config-copier.xml rename to tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml index 09dd7a00762..ede3dcb1228 100644 --- a/tests/integration/test_cluster_copier/configs_taxi/config-copier.xml +++ b/tests/integration/test_cluster_copier/configs_three_nodes/config-copier.xml @@ -1,7 +1,7 @@ - fatal + information /var/log/clickhouse-server/copier/log.log /var/log/clickhouse-server/copier/log.err.log 1000M diff --git a/tests/integration/test_cluster_copier/configs_taxi/users.xml b/tests/integration/test_cluster_copier/configs_three_nodes/users.xml similarity index 100% rename from tests/integration/test_cluster_copier/configs_taxi/users.xml rename to tests/integration/test_cluster_copier/configs_three_nodes/users.xml diff --git a/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml b/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml index ebea79e92e6..642998c6d87 100644 --- a/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml +++ b/tests/integration/test_cluster_copier/configs_two_nodes/config-copier.xml @@ -1,7 +1,7 @@ - fatal + information /var/log/clickhouse-server/copier/log.log /var/log/clickhouse-server/copier/log.err.log 1000M diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index b30e63d9fb8..cd69e6e28f2 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -22,7 +22,7 @@ def started_cluster(): for name in ["first", "second", "third"]: cluster.add_instance(name, - main_configs=["configs_taxi/conf.d/clusters.xml", "configs_taxi/conf.d/ddl.xml"], user_configs=["configs_taxi/users.xml"], + main_configs=["configs_three_nodes/conf.d/clusters.xml", "configs_three_nodes/conf.d/ddl.xml"], user_configs=["configs_three_nodes/users.xml"], with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) cluster.start() @@ -196,7 +196,7 @@ def execute_task(task, cmd_options): for instance_name, instance in cluster.instances.items(): instance = cluster.instances[instance_name] container = instance.get_docker_handle() - instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_taxi/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_three_nodes/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") logging.info("Copied copier config to {}".format(instance.name)) exec_id = docker_api.exec_create(container.id, cmd, stderr=True) output = docker_api.exec_start(exec_id).decode('utf8') From b4895b1e7246f898b90ff8aa7e18f6f299524db7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 24 Apr 2021 00:34:01 +0300 Subject: [PATCH 286/652] better --- tests/integration/test_cluster_copier/test_two_nodes.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 059b76c35ab..28c52473366 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -410,7 +410,7 @@ def execute_task(task, cmd_options): for instance_name, instance in cluster.instances.items(): instance = cluster.instances[instance_name] container = instance.get_docker_handle() - instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_taxi/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs_two_nodes/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") logging.info("Copied copier config to {}".format(instance.name)) exec_id = docker_api.exec_create(container.id, cmd, stderr=True) output = docker_api.exec_start(exec_id).decode('utf8') From ac072243d6ab323497576b6cfde6eee6a5ca3413 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 27 Apr 2021 15:34:56 +0300 Subject: [PATCH 287/652] style + enable trivial test --- programs/copier/ClusterCopier.cpp | 79 +++++++++++++------ .../{trivial_test.py => test_trivial.py} | 76 ++++++++---------- 2 files changed, 87 insertions(+), 68 deletions(-) rename tests/integration/test_cluster_copier/{trivial_test.py => test_trivial.py} (75%) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index bd99b752b9a..08a8749213a 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -22,6 +22,12 @@ namespace ErrorCodes } +std::string wrapWithColor(const std::string & value) +{ + return "\u001b[36;1m" + value + "\u001b[0m"; +} + + void ClusterCopier::init() { auto zookeeper = getContext()->getZooKeeper(); @@ -194,6 +200,7 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, LOG_INFO(log, "Waiting for {} setup jobs", thread_pool.active()); thread_pool.wait(); } + std::cout << "discoverTablePartitions finished" << std::endl; } void ClusterCopier::uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) @@ -587,22 +594,6 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent); } - - /// Try to drop destination partition in original table - DatabaseAndTableName original_table = task_table.table_push; - - if (task_table.allow_to_drop_target_partitions) - { - WriteBufferFromOwnString ss; - ss << "ALTER TABLE " << getQuotedTable(original_table) << ((partition_name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") << partition_name; - - UInt64 num_shards_drop_partition = executeQueryOnCluster(task_table.cluster_push, ss.str(), task_cluster->settings_push, ClusterExecutionMode::ON_EACH_SHARD); - - LOG_INFO(log, "Drop partiton {} in original table {} have been executed successfully on {} shards of {}", - partition_name, getQuotedTable(original_table), num_shards_drop_partition, task_table.cluster_push->getShardCount()); - } - - /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { @@ -611,15 +602,19 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ASTPtr query_alter_ast; String query_alter_ast_string; + DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); Settings settings_push = task_cluster->settings_push; ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_NODE; - + UInt64 max_successful_executions_per_shard = 0; if (settings_push.replication_alter_partitions_sync == 1) + { execution_mode = ClusterExecutionMode::ON_EACH_SHARD; + max_successful_executions_per_shard = 1; + } query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + ((partition_name == "'all'") ? " ATTACH PARTITION ID " : " ATTACH PARTITION ") + partition_name + @@ -638,11 +633,17 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t if (settings_push.replication_alter_partitions_sync == 1) { - LOG_INFO(log, "Destination tables {} have been executed alter query successfully on {} shards of {}", - getQuotedTable(original_table), num_nodes, task_table.cluster_push->getShardCount()); + LOG_INFO( + log, + "Destination tables {} have been executed alter query successfully on {} shards of {}", + getQuotedTable(task_table.table_push), + num_nodes, + task_table.cluster_push->getShardCount()); if (num_nodes != task_table.cluster_push->getShardCount()) + { return TaskStatus::Error; + } } else { @@ -658,6 +659,30 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t if (inject_fault) throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) + { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + ((partition_name == "'all'") ? " PARTITION ID " : " PARTITION ") + partition_name + " DEDUPLICATE;"; + + LOG_INFO(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_deduplicate_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + task_cluster->settings_push, + ClusterExecutionMode::ON_EACH_SHARD); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes)); + } + } + catch (...) + { + LOG_INFO(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name); + throw; + } } /// Create node to signal that we finished moving @@ -827,6 +852,9 @@ bool ClusterCopier::tryDropPartitionPiece( String query = "ALTER TABLE " + getQuotedTable(helping_table); query += ((task_partition.name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") + task_partition.name + ""; + /// TODO: use this statement after servers will be updated up to 1.1.54310 + // query += " DROP PARTITION ID '" + task_partition.name + "'"; + ClusterPtr & cluster_push = task_table.cluster_push; Settings settings_push = task_cluster->settings_push; @@ -1090,7 +1118,7 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t InterpreterCreateQuery::prepareOnClusterQuery(create, getContext(), task_table.cluster_push_name); String query = queryToString(create_query_push_ast); - LOG_INFO(log, "Create destination tables. Query: \n {}", query); + LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, @@ -1365,7 +1393,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto create_query_push_ast = rewriteCreateQueryStorage(create_query_ast, database_and_table_for_current_piece, new_engine_push_ast); String query = queryToString(create_query_push_ast); - LOG_INFO(log, "Create destination tables. Query: \n {}", query); + LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, @@ -1509,6 +1537,11 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext())); input = std::make_shared(pure_input, actions); + + std::cout << "Input:" << std::endl; + std::cout << input->getHeader().dumpStructure() << std::endl; + std::cout << "Output:" << std::endl; + std::cout << output->getHeader().dumpStructure() << std::endl; } /// Fail-fast optimization to abort copying when the current clean state expires @@ -1802,7 +1835,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); - LOG_INFO(log, "Computing destination partition set, executing query: \n {}", query); + LOG_INFO(log, "Computing destination partition set, executing query: \n {}", wrapWithColor(query)); auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); @@ -1958,7 +1991,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( } catch (...) { - LOG_WARNING(log, "An error occured while processing query : \n {}", query); + LOG_WARNING(log, "An error occured while processing query : \n {}", wrapWithColor(query)); tryLogCurrentException(log); } } diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/test_trivial.py similarity index 75% rename from tests/integration/test_cluster_copier/trivial_test.py rename to tests/integration/test_cluster_copier/test_trivial.py index 8a43440ac90..fa103a9018b 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/test_trivial.py @@ -1,15 +1,18 @@ import os import sys import time -from contextlib import contextmanager -import docker +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +import kazoo import pytest +import docker + CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV + COPYING_FAIL_PROBABILITY = 0.33 MOVING_FAIL_PROBABILITY = 0.1 @@ -40,7 +43,6 @@ def started_cluster(): yield cluster finally: - pass cluster.shutdown() @@ -85,9 +87,14 @@ def execute_task(task, cmd_options): zk = cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) + try: + zk.delete("/clickhouse-copier", recursive=True) + except kazoo.exceptions.NoNodeError: + print("No node /clickhouse-copier. It is Ok in first test.") + zk_task_path = task.zk_task_path zk.ensure_path(zk_task_path) - zk.create(zk_task_path + "/description", task.copier_task_config) + zk.create(zk_task_path + "/description", task.copier_task_config.encode()) # Run cluster-copier processes on each node docker_api = docker.from_env().api @@ -99,23 +106,28 @@ def execute_task(task, cmd_options): '--base-dir', '/var/log/clickhouse-server/copier'] cmd += cmd_options - print(cmd) + copiers = list(cluster.instances.keys()) - for instance_name, instance in cluster.instances.items(): + for instance_name in copiers: + instance = cluster.instances[instance_name] container = instance.get_docker_handle() + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), + "/etc/clickhouse-server/config-copier.xml") + print("Copied copier config to {}".format(instance.name)) exec_id = docker_api.exec_create(container.id, cmd, stderr=True) - docker_api.exec_start(exec_id, detach=True) - + output = docker_api.exec_start(exec_id).decode('utf8') + print(output) copiers_exec_ids.append(exec_id) print("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) # Wait for copiers stopping and check their return codes - for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + for exec_id, instance_name in zip(copiers_exec_ids, copiers): + instance = cluster.instances[instance_name] while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: break - time.sleep(1) + time.sleep(0.5) assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) @@ -128,53 +140,27 @@ def execute_task(task, cmd_options): # Tests -@pytest.mark.parametrize( - ('use_sample_offset'), - [ - False, - True - ] -) +@pytest.mark.parametrize(('use_sample_offset'),[False,True]) def test_trivial_copy(started_cluster, use_sample_offset): if use_sample_offset: execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) else: - print("AAAAA") execute_task(TaskTrivial(started_cluster, use_sample_offset), []) -@pytest.mark.parametrize( - ('use_sample_offset'), - [ - False, - True - ] -) +@pytest.mark.parametrize(('use_sample_offset'),[False,True]) def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) -@pytest.mark.parametrize( - ('use_sample_offset'), - [ - False, - True - ] -) +@pytest.mark.parametrize(('use_sample_offset'),[False,True]) def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) - - -if __name__ == '__main__': - with contextmanager(started_cluster)() as cluster: - for name, instance in list(cluster.instances.items()): - print(name, instance.ip_address) - input("Cluster created, press any key to destroy...") + execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) From 1c548058e60126ee419e956658ed488dc64af5b1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 29 Apr 2021 22:16:51 +0300 Subject: [PATCH 288/652] add status json --- programs/copier/ClusterCopier.cpp | 66 +++++++++++++++---- programs/copier/ClusterCopier.h | 1 - programs/copier/ClusterCopierApp.cpp | 21 ++++++ programs/copier/ClusterCopierApp.h | 3 +- programs/copier/StatusAccumulator.h | 65 ++++++++++++++++++ programs/copier/TaskCluster.h | 4 ++ programs/copier/TaskTableAndShard.h | 14 ++++ .../test_cluster_copier/test_trivial.py | 4 ++ 8 files changed, 162 insertions(+), 16 deletions(-) create mode 100644 programs/copier/StatusAccumulator.h diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 08a8749213a..e747933ff46 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1,6 +1,7 @@ #include "ClusterCopier.h" #include "Internals.h" +#include "StatusAccumulator.h" #include #include @@ -44,10 +45,9 @@ void ClusterCopier::init() task_cluster = std::make_unique(task_zookeeper_path, working_database_name); reloadTaskDescription(); - task_cluster_initial_config = task_cluster_current_config; - task_cluster->loadTasks(*task_cluster_initial_config); - getContext()->setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); + task_cluster->loadTasks(*task_cluster_current_config); + getContext()->setClustersConfig(task_cluster_current_config, task_cluster->clusters_prefix); /// Set up shards and their priority task_cluster->random_engine.seed(task_cluster->random_device()); @@ -64,6 +64,8 @@ void ClusterCopier::init() zookeeper->createAncestors(getWorkersPathVersion() + "/"); zookeeper->createAncestors(getWorkersPath() + "/"); + /// Init status node + zookeeper->createIfNotExists(task_zookeeper_path + "/status", "{}"); } template @@ -231,23 +233,17 @@ void ClusterCopier::reloadTaskDescription() auto zookeeper = getContext()->getZooKeeper(); task_description_watch_zookeeper = zookeeper; - String task_config_str; Coordination::Stat stat{}; - Coordination::Error code; - zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); - if (code != Coordination::Error::ZOK) - throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); + /// It will throw exception if such a node doesn't exist. + auto task_config_str = zookeeper->get(task_description_path, &stat); - LOG_INFO(log, "Loading description, zxid={}", task_description_current_stat.czxid); - auto config = getConfigurationFromXMLString(task_config_str); + LOG_INFO(log, "Loading task description"); + task_cluster_current_config = getConfigurationFromXMLString(task_config_str); /// Setup settings - task_cluster->reloadSettings(*config); + task_cluster->reloadSettings(*task_cluster_current_config); getContext()->setSettings(task_cluster->settings_common); - - task_cluster_current_config = config; - task_description_current_stat = stat; } void ClusterCopier::updateConfigIfNeeded() @@ -689,6 +685,23 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); zookeeper->set(current_partition_attach_is_done, state_finished, 0); + /// Also increment a counter of processed partitions + while (true) + { + Coordination::Stat stat; + auto status_json = zookeeper->get(task_zookeeper_path + "/status", &stat); + auto statuses = StatusAccumulator::fromJSON(status_json); + + /// Increment status for table. + auto status_for_table = (*statuses)[task_table.name_in_config]; + status_for_table.processed_partitions_count += 1; + (*statuses)[task_table.name_in_config] = status_for_table; + + auto statuses_to_commit = StatusAccumulator::serializeToJSON(statuses); + auto error = zookeeper->trySet(task_zookeeper_path + "/status", statuses_to_commit, stat.version, &stat); + if (error == Coordination::Error::ZOK) + break; + } } return TaskStatus::Finished; @@ -907,6 +920,31 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab LOG_WARNING(log, "Create destination Tale Failed "); return false; } + + /// Set all_partitions_count for table in Zookeeper + auto zookeeper = getContext()->getZooKeeper(); + while (true) + { + Coordination::Stat stat; + auto status_json = zookeeper->get(task_zookeeper_path + "/status", &stat); + auto statuses = StatusAccumulator::fromJSON(status_json); + + /// Exit if someone already set the initial value for this table. + if (statuses->find(task_table.name_in_config) != statuses->end()) + break; + (*statuses)[task_table.name_in_config] = StatusAccumulator::TableStatus + { + /*all_partitions_count=*/task_table.ordered_partition_names.size(), + /*processed_partition_count=*/0 + }; + + auto statuses_to_commit = StatusAccumulator::serializeToJSON(statuses); + auto error = zookeeper->trySet(task_zookeeper_path + "/status", statuses_to_commit, stat.version); + if (error == Coordination::Error::ZOK) + break; + } + + /// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint bool previous_shard_is_instantly_finished = false; diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index ad44e3b0ec4..b553f253cde 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -210,7 +210,6 @@ private: ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_description_current_stat{}; std::unique_ptr task_cluster; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 2422f5467a8..5c4765855ef 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -22,6 +22,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) task_path = config().getString("task-path"); log_level = config().getString("log-level", "info"); is_safe_mode = config().has("safe-mode"); + is_status_mode = config().has("status"); if (config().has("copy-fault-probability")) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); if (config().has("move-fault-probability")) @@ -95,6 +96,7 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) .argument("base-dir").binding("base-dir")); options.addOption(Poco::Util::Option("experimental-use-sample-offset", "", "Use SAMPLE OFFSET query instead of cityHash64(PRIMARY KEY) % n == k") .argument("experimental-use-sample-offset").binding("experimental-use-sample-offset")); + options.addOption(Poco::Util::Option("status", "", "Get for status for current execution").binding("status")); using Me = std::decay_t; options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help") @@ -104,6 +106,25 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) void ClusterCopierApp::mainImpl() { + /// Status command + { + if (is_status_mode) + { + SharedContextHolder shared_context = Context::createShared(); + auto context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + SCOPE_EXIT_SAFE(context->shutdown()); + + auto zookeeper = context->getZooKeeper(); + auto status_json = zookeeper->get(task_path + "/status"); + + LOG_INFO(&logger(), "{}", status_json); + std::cout << status_json << std::endl; + + context->resetZooKeeper(); + return; + } + } StatusFile status_file(process_path + "/status", StatusFile::write_full_info); ThreadStatus thread_status; diff --git a/programs/copier/ClusterCopierApp.h b/programs/copier/ClusterCopierApp.h index 257b10cf196..cce07e338c0 100644 --- a/programs/copier/ClusterCopierApp.h +++ b/programs/copier/ClusterCopierApp.h @@ -76,8 +76,9 @@ private: std::string config_xml_path; std::string task_path; - std::string log_level = "trace"; + std::string log_level = "info"; bool is_safe_mode = false; + bool is_status_mode = false; double copy_fault_probability = 0.0; double move_fault_probability = 0.0; bool is_help = false; diff --git a/programs/copier/StatusAccumulator.h b/programs/copier/StatusAccumulator.h new file mode 100644 index 00000000000..6e20e3dc95d --- /dev/null +++ b/programs/copier/StatusAccumulator.h @@ -0,0 +1,65 @@ +#pragma once + + +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +class StatusAccumulator +{ + public: + struct TableStatus + { + size_t all_partitions_count; + size_t processed_partitions_count; + }; + + using Map = std::unordered_map; + using MapPtr = std::shared_ptr; + + static MapPtr fromJSON(std::string state_json) + { + Poco::JSON::Parser parser; + auto state = parser.parse(state_json).extract(); + MapPtr result_ptr = std::make_shared(); + for (const auto & table_name : state->getNames()) + { + auto table_status_json = state->getValue(table_name); + auto table_status = parser.parse(table_status_json).extract(); + /// Map entry will be created if it is absent + auto & map_table_status = (*result_ptr)[table_name]; + map_table_status.all_partitions_count += table_status->getValue("all_partitions_count"); + map_table_status.processed_partitions_count += table_status->getValue("processed_partitions_count"); + } + return result_ptr; + } + + static std::string serializeToJSON(MapPtr statuses) + { + Poco::JSON::Object result_json; + for (const auto & [table_name, table_status] : *statuses) + { + Poco::JSON::Object status_json; + status_json.set("all_partitions_count", table_status.all_partitions_count); + status_json.set("processed_partitions_count", table_status.processed_partitions_count); + + result_json.set(table_name, status_json); + } + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(result_json, oss); + auto result = oss.str(); + return result; + } +}; + +} diff --git a/programs/copier/TaskCluster.h b/programs/copier/TaskCluster.h index 5cf247abaf9..5fc990ccf48 100644 --- a/programs/copier/TaskCluster.h +++ b/programs/copier/TaskCluster.h @@ -97,8 +97,12 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat set_default_value(settings_pull.max_threads, 1); set_default_value(settings_pull.max_block_size, 8192UL); set_default_value(settings_pull.preferred_block_size_bytes, 0); + set_default_value(settings_pull.prefer_localhost_replica, 0); + set_default_value(settings_push.insert_distributed_timeout, 0); set_default_value(settings_push.replication_alter_partitions_sync, 2); + set_default_value(settings_push.prefer_localhost_replica, 0); + } } diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 77a8db31525..d15cb3fcd57 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -52,6 +52,10 @@ struct TaskTable bool isReplicatedTable() const { return is_replicated_table; } + /// These nodes are used for check-status option + String getStatusAllPartitionCount() const; + String getStatusProcessedPartitionsCount() const; + /// Partitions will be split into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) size_t number_of_splits; @@ -245,6 +249,16 @@ inline String TaskTable::getCertainPartitionPieceTaskStatusPath(const String & p return getPartitionPiecePath(partition_name, piece_number) + "/shards"; } +inline String TaskTable::getStatusAllPartitionCount() const +{ + return task_cluster.task_zookeeper_path + "/status/all_partitions_count"; +} + +inline String TaskTable::getStatusProcessedPartitionsCount() const +{ + return task_cluster.task_zookeeper_path + "/status/processed_partitions_count"; +} + inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, const String & table_key) : task_cluster(parent) diff --git a/tests/integration/test_cluster_copier/test_trivial.py b/tests/integration/test_cluster_copier/test_trivial.py index fa103a9018b..8502c0e7f67 100644 --- a/tests/integration/test_cluster_copier/test_trivial.py +++ b/tests/integration/test_cluster_copier/test_trivial.py @@ -71,6 +71,10 @@ class TaskTrivial: settings={"insert_distributed_sync": 1}) def check(self): + zk = cluster.get_kazoo_client('zoo1') + status_data, _ = zk.get(self.zk_task_path + "/status") + assert status_data == b'{"hits":{"all_partitions_count":5,"processed_partitions_count":5}}' + source = cluster.instances['s0_0_0'] destination = cluster.instances['s1_0_0'] From 01511d0cbefd19158a87b19ca6f10ff2520aba5c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 29 Apr 2021 22:33:34 +0300 Subject: [PATCH 289/652] better --- programs/copier/ClusterCopier.cpp | 85 ++++++++++++++++++------------- 1 file changed, 49 insertions(+), 36 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index e747933ff46..306016ca383 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1156,7 +1156,7 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t InterpreterCreateQuery::prepareOnClusterQuery(create, getContext(), task_table.cluster_push_name); String query = queryToString(create_query_push_ast); - LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); + LOG_INFO(log, "Create destination tables. Query: \n {}", query); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, @@ -1164,10 +1164,6 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); - // if (shards != task_table.cluster_push->getShardCount()) - // { - // return TaskStatus::Error; - // } } catch (...) { @@ -1431,19 +1427,20 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto create_query_push_ast = rewriteCreateQueryStorage(create_query_ast, database_and_table_for_current_piece, new_engine_push_ast); String query = queryToString(create_query_push_ast); +<<<<<<< HEAD LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, "Destination tables {} have been created on {} shards of {}", +======= + LOG_INFO(log, "Create destination tables. Query: \n {}", query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); + LOG_INFO(log, "Destination tables {} have been created on {} shards of {}", +>>>>>>> better getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); - - // if (shards != task_table.cluster_push->getShardCount()) - // { - // return TaskStatus::Error; - // } } @@ -1990,39 +1987,53 @@ UInt64 ClusterCopier::executeQueryOnCluster( for (const auto & replicas : cluster_for_query->getShardsAddresses()) { - const auto & node = replicas[0]; - try + for (const auto & node : replicas) { - connections.emplace_back(std::make_shared( - node.host_name, node.port, node.default_database, - node.user, node.password, node.cluster, node.cluster_secret, - "ClusterCopier", node.compression, node.secure - )); - - /// We execute only Alter, Create and Drop queries. - const auto header = Block{}; - - /// For unknown reason global context is passed to IStorage::read() method - /// So, task_identifier is passed as constructor argument. It is more obvious. - auto remote_query_executor = std::make_shared( - *connections.back(), query, header, getContext(), - /*throttler=*/nullptr, Scalars(), Tables(), QueryProcessingStage::Complete); - try { - remote_query_executor->sendQuery(); + connections.emplace_back(std::make_shared( + node.host_name, node.port, node.default_database, + node.user, node.password, node.cluster, node.cluster_secret, + "ClusterCopier", node.compression, node.secure + )); + + /// We execute only Alter, Create and Drop queries. + const auto header = Block{}; + + /// For unknown reason global context is passed to IStorage::read() method + /// So, task_identifier is passed as constructor argument. It is more obvious. + auto remote_query_executor = std::make_shared( + *connections.back(), query, header, getContext(), + /*throttler=*/nullptr, Scalars(), Tables(), QueryProcessingStage::Complete); + + try + { + remote_query_executor->sendQuery(); + } + catch (...) + { + LOG_WARNING(log, "Seemns like node with address {} is unreachable.", node.host_name); + continue; + } + + while (true) + { + auto block = remote_query_executor->read(); + if (!block) + break; + } + + remote_query_executor->finish(); + ++successfully_executed; + break; } catch (...) { - LOG_WARNING(log, "Seemns like node with address {} is unreachable {}", node.host_name); - } - - while (true) - { - auto block = remote_query_executor->read(); - if (!block) - break; + LOG_WARNING(log, "An error occurred while processing query : \n {}", query); + tryLogCurrentException(log); + continue; } +<<<<<<< HEAD remote_query_executor->finish(); ++successfully_executed; @@ -2031,6 +2042,8 @@ UInt64 ClusterCopier::executeQueryOnCluster( { LOG_WARNING(log, "An error occured while processing query : \n {}", wrapWithColor(query)); tryLogCurrentException(log); +======= +>>>>>>> better } } From 33dcebbb2456a309a2da3ea0d5c7f3feed82d550 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 30 Apr 2021 01:56:41 +0300 Subject: [PATCH 290/652] style --- programs/copier/ClusterCopier.cpp | 39 +------------------ programs/copier/ClusterCopier.h | 5 +-- programs/copier/TaskCluster.h | 8 ++-- .../test_cluster_copier/test_trivial.py | 19 +-------- 4 files changed, 9 insertions(+), 62 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 306016ca383..d7e37a1a5d3 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1405,11 +1405,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// 3) Create helping table on the whole destination cluster auto & settings_push = task_cluster->settings_push; - /// Get a connection to any shard to fetch `CREATE` query auto connection = task_table.cluster_push->getAnyShardInfo().pool->get(timeouts, &settings_push, true); - /// Execute a request and get `CREATE` query as a string. String create_query = getRemoteCreateTable(task_shard.task_table.table_push, *connection, settings_push); - /// Parse it to ASTPtr + ParserCreateQuery parser_create_query; auto create_query_ast = parseQuery(parser_create_query, create_query, settings_push.max_query_size, settings_push.max_parser_depth); /// Define helping table database and name for current partition piece @@ -1417,8 +1415,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( task_table.table_push.first, task_table.table_push.second + "_piece_" + toString(current_piece_number)}; - /// This is a bit of legacy, because we now could parse and engine AST from the whole create query. - /// But this is needed to make helping table non-replicated. We simply don't need this + auto new_engine_push_ast = task_table.engine_push_ast; if (task_table.isReplicatedTable()) new_engine_push_ast = task_table.rewriteReplicatedCreateQueryToPlain(); @@ -1427,17 +1424,11 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto create_query_push_ast = rewriteCreateQueryStorage(create_query_ast, database_and_table_for_current_piece, new_engine_push_ast); String query = queryToString(create_query_push_ast); -<<<<<<< HEAD LOG_INFO(log, "Create destination tables. Query: \n {}", wrapWithColor(query)); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); LOG_INFO( log, "Destination tables {} have been created on {} shards of {}", -======= - LOG_INFO(log, "Create destination tables. Query: \n {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, ClusterExecutionMode::ON_EACH_NODE); - LOG_INFO(log, "Destination tables {} have been created on {} shards of {}", ->>>>>>> better getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); @@ -1750,7 +1741,6 @@ String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, C } - ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) { /// Fetch and parse (possibly) new definition @@ -1766,20 +1756,6 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time } -ASTPtr ClusterCopier::getCreateTableForPushShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) -{ - /// Fetch and parse (possibly) new definition - auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_push, true); - String create_query_pull_str = getRemoteCreateTable( - task_shard.task_table.table_push, - *connection_entry, - task_cluster->settings_push); - - ParserCreateQuery parser_create_query; - const auto & settings = getContext()->getSettingsRef(); - return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth); -} - /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split) @@ -2033,17 +2009,6 @@ UInt64 ClusterCopier::executeQueryOnCluster( tryLogCurrentException(log); continue; } -<<<<<<< HEAD - - remote_query_executor->finish(); - ++successfully_executed; - } - catch (...) - { - LOG_WARNING(log, "An error occured while processing query : \n {}", wrapWithColor(query)); - tryLogCurrentException(log); -======= ->>>>>>> better } } diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index b553f253cde..bf3fb4f2ffa 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -24,9 +24,7 @@ public: task_zookeeper_path(task_path_), host_id(host_id_), working_database_name(proxy_database_name_), - log(log_) { - std::cout << "Level from constructor" << log->getLevel() << std::endl; - } + log(log_) {} void init(); @@ -162,7 +160,6 @@ protected: String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings & settings); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); - ASTPtr getCreateTableForPushShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true); diff --git a/programs/copier/TaskCluster.h b/programs/copier/TaskCluster.h index 5fc990ccf48..7d8f01ba15f 100644 --- a/programs/copier/TaskCluster.h +++ b/programs/copier/TaskCluster.h @@ -77,6 +77,8 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat if (config.has(prefix + "settings")) settings_common.loadSettingsFromConfig(prefix + "settings", config); + settings_common.prefer_localhost_replica = 0; + settings_pull = settings_common; if (config.has(prefix + "settings_pull")) settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); @@ -92,17 +94,17 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat /// Override important settings settings_pull.readonly = 1; + settings_pull.prefer_localhost_replica = false; settings_push.insert_distributed_sync = true; + settings_push.prefer_localhost_replica = false; + set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); set_default_value(settings_pull.max_threads, 1); set_default_value(settings_pull.max_block_size, 8192UL); set_default_value(settings_pull.preferred_block_size_bytes, 0); - set_default_value(settings_pull.prefer_localhost_replica, 0); set_default_value(settings_push.insert_distributed_timeout, 0); set_default_value(settings_push.replication_alter_partitions_sync, 2); - set_default_value(settings_push.prefer_localhost_replica, 0); - } } diff --git a/tests/integration/test_cluster_copier/test_trivial.py b/tests/integration/test_cluster_copier/test_trivial.py index 8502c0e7f67..947c4bee5ca 100644 --- a/tests/integration/test_cluster_copier/test_trivial.py +++ b/tests/integration/test_cluster_copier/test_trivial.py @@ -14,7 +14,7 @@ CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) -COPYING_FAIL_PROBABILITY = 0.33 +COPYING_FAIL_PROBABILITY = 0.1 MOVING_FAIL_PROBABILITY = 0.1 cluster = None @@ -151,20 +151,3 @@ def test_trivial_copy(started_cluster, use_sample_offset): else: execute_task(TaskTrivial(started_cluster, use_sample_offset), []) - -@pytest.mark.parametrize(('use_sample_offset'),[False,True]) -def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): - if use_sample_offset: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), - '--experimental-use-sample-offset', '1']) - else: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) - - -@pytest.mark.parametrize(('use_sample_offset'),[False,True]) -def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset): - if use_sample_offset: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), - '--experimental-use-sample-offset', '1']) - else: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) From 51bca62ff3e4c3aaacf9fb67e8578531c9d5b12a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 31 May 2021 18:01:55 +0300 Subject: [PATCH 291/652] Fix typo --- src/Functions/formatDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 12c2932c54f..b8d5fe7d281 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -309,7 +309,7 @@ public: "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + " when arguments size is 1. Should be integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)) + if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type))) throw Exception( "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + " when arguments size is 2 or 3. Should be a integer or a date with time", From 7bb1709185cc166907279f0faceddbeb8496a79d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 May 2021 18:46:32 +0300 Subject: [PATCH 292/652] smaller pack --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 20042ee596a..39ba416f147 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -14,7 +14,7 @@ import csv MAX_RETRY = 3 NUM_WORKERS = 5 SLEEP_BETWEEN_RETRIES = 5 -PARALLEL_GROUP_SIZE = 200 +PARALLEL_GROUP_SIZE = 100 CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge" DOCKERD_LOGS_PATH = "/ClickHouse/tests/integration/dockerd.log" From 655f8c4b1ccad42a8fe2249f86383dc0699b6740 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Mon, 31 May 2021 18:19:38 +0300 Subject: [PATCH 293/652] More fixes --- src/Common/examples/CMakeLists.txt | 2 -- src/Common/examples/YAML_fuzzer.cpp | 11 ++++------- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 288fcf7f87e..b154d5965ca 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -81,8 +81,6 @@ target_link_libraries (average PRIVATE clickhouse_common_io) add_executable (shell_command_inout shell_command_inout.cpp) target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) -set(SRCS) - if (ENABLE_FUZZING) add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp index b290bbc5fa8..f4b570e0e0a 100644 --- a/src/Common/examples/YAML_fuzzer.cpp +++ b/src/Common/examples/YAML_fuzzer.cpp @@ -11,26 +11,23 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) /// build ClickHouse with YAML_fuzzer.cpp /// ./YAML_fuzzer YAML_CORPUS /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer - char buf[L_tmpnam]; - char* file_name = std::tmpnam(buf); - if (file_name == nullptr) + char file_name[L_tmpnam]; + if (!std::tmpnam(file_name)) { std::cerr << "Cannot create temp file!\n"; return 1; } - std::string cur_file(file_name); - std::string input = std::string(reinterpret_cast(data), size); DB::YAMLParser parser; { - std::ofstream temp_file(cur_file); + std::ofstream temp_file(file_name); temp_file << input; } try { - DB::YAMLParser::parse(cur_file); + DB::YAMLParser::parse(std::string(file_name)); } catch (...) { From e03c09c51b4c8996c05362b3c911b8e5f50fabf9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 31 May 2021 21:08:16 +0300 Subject: [PATCH 294/652] Update DiskS3.cpp --- src/Disks/S3/DiskS3.cpp | 46 ----------------------------------------- 1 file changed, 46 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 78b0268ec48..5530ff4aa6b 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -139,52 +139,6 @@ public: } private: - std::unique_ptr initialize() - { - size_t offset = absolute_position; - for (size_t i = 0; i < metadata.s3_objects.size(); ++i) - { - current_buf_idx = i; - const auto & [path, size] = metadata.s3_objects[i]; - if (size > offset) - { - auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, max_single_read_retries, buf_size); - buf->seek(offset, SEEK_SET); - return buf; - } - offset -= size; - } - return nullptr; - } - - bool nextImpl() override - { - /// Find first available buffer that fits to given offset. - if (!current_buf) - current_buf = initialize(); - - /// If current buffer has remaining data - use it. - if (current_buf && current_buf->next()) - { - working_buffer = current_buf->buffer(); - absolute_position += working_buffer.size(); - return true; - } - - /// If there is no available buffers - nothing to read. - if (current_buf_idx + 1 >= metadata.s3_objects.size()) - return false; - - ++current_buf_idx; - const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, max_single_read_retries, buf_size); - current_buf->next(); - working_buffer = current_buf->buffer(); - absolute_position += working_buffer.size(); - - return true; - } - std::shared_ptr client_ptr; const String & bucket; DiskS3::Metadata metadata; From b9aa4045080adfa261f916b2161288a0725c2eb6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 May 2021 21:25:43 +0300 Subject: [PATCH 295/652] Review fixes. --- .../QueryPlan/ReadFromMergeTree.cpp | 194 ++++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 2 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 - 3 files changed, 82 insertions(+), 119 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0672d7ed040..fd5de98b4c0 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -230,63 +230,85 @@ Pipe ReadFromMergeTree::read( return pipe; } +namespace +{ + +struct PartRangesReadInfo +{ + std::vector sum_marks_in_parts; + + size_t sum_marks = 0; + size_t total_rows = 0; + size_t adaptive_parts = 0; + size_t index_granularity_bytes = 0; + size_t max_marks_to_use_cache = 0; + size_t min_marks_for_concurrent_read = 0; + + bool use_uncompressed_cache = false; + + PartRangesReadInfo( + const RangesInDataParts & parts, + const Settings & settings, + const MergeTreeSettings & data_settings) + { + /// Count marks for each part. + sum_marks_in_parts.resize(parts.size()); + for (size_t i = 0; i < parts.size(); ++i) + { + total_rows += parts[i].getRowsCount(); + sum_marks_in_parts[i] = parts[i].getMarksCount(); + sum_marks += sum_marks_in_parts[i]; + + if (parts[i].data_part->index_granularity_info.is_adaptive) + ++adaptive_parts; + } + + if (adaptive_parts > parts.size() / 2) + index_granularity_bytes = data_settings.index_granularity_bytes; + + max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( + settings.merge_tree_max_rows_to_use_cache, + settings.merge_tree_max_bytes_to_use_cache, + data_settings.index_granularity, + index_granularity_bytes); + + min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, + data_settings.index_granularity, + index_granularity_bytes, + sum_marks); + + use_uncompressed_cache = settings.use_uncompressed_cache; + if (sum_marks > max_marks_to_use_cache) + use_uncompressed_cache = false; + } +}; + +} + Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, const Names & column_names) { const auto & settings = context->getSettingsRef(); - - /// Count marks for each part. - std::vector sum_marks_in_parts(parts_with_ranges.size()); - size_t sum_marks = 0; - size_t total_rows = 0; - const auto data_settings = data.getSettings(); - size_t adaptive_parts = 0; - for (size_t i = 0; i < parts_with_ranges.size(); ++i) - { - total_rows += parts_with_ranges[i].getRowsCount(); - sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts_with_ranges.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - 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, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - bool use_uncompressed_cache = settings.use_uncompressed_cache; - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; - - if (0 == sum_marks) + if (0 == info.sum_marks) return {}; size_t num_streams = requested_num_streams; if (num_streams > 1) { /// Reduce the number of num_streams if the data is small. - if (sum_marks < num_streams * min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) - num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts_with_ranges.size()); + if (info.sum_marks < num_streams * info.min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) + num_streams = std::max((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read, parts_with_ranges.size()); } return read(std::move(parts_with_ranges), column_names, ReadType::Default, - num_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache); } static ActionsDAGPtr createProjection(const Block & header) @@ -305,44 +327,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( const InputOrderInfoPtr & input_order_info) { const auto & settings = context->getSettingsRef(); - size_t sum_marks = 0; - size_t adaptive_parts = 0; - std::vector sum_marks_in_parts(parts_with_ranges.size()); const auto data_settings = data.getSettings(); - for (size_t i = 0; i < parts_with_ranges.size(); ++i) - { - sum_marks_in_parts[i] = parts_with_ranges[i].getMarksCount(); - sum_marks += sum_marks_in_parts[i]; - - if (parts_with_ranges[i].data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts > parts_with_ranges.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - 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, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - - bool use_uncompressed_cache = settings.use_uncompressed_cache; - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); Pipes res; - if (sum_marks == 0) + if (info.sum_marks == 0) return {}; /// Let's split ranges to avoid reading much data. @@ -391,7 +382,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( return new_ranges; }; - const size_t min_marks_per_stream = (sum_marks - 1) / requested_num_streams + 1; + const size_t min_marks_per_stream = (info.sum_marks - 1) / requested_num_streams + 1; bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold); Pipes pipes; @@ -409,16 +400,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( RangesInDataPart part = parts_with_ranges.back(); parts_with_ranges.pop_back(); - size_t & marks_in_part = sum_marks_in_parts.back(); + size_t & marks_in_part = info.sum_marks_in_parts.back(); /// We will not take too few rows from a part. - if (marks_in_part >= min_marks_for_concurrent_read && - need_marks < min_marks_for_concurrent_read) - need_marks = min_marks_for_concurrent_read; + if (marks_in_part >= info.min_marks_for_concurrent_read && + need_marks < info.min_marks_for_concurrent_read) + need_marks = info.min_marks_for_concurrent_read; /// Do not leave too few rows in the part. if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) + marks_in_part - need_marks < info.min_marks_for_concurrent_read) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; @@ -429,7 +420,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( ranges_to_get_from_part = part.ranges; need_marks -= marks_in_part; - sum_marks_in_parts.pop_back(); + info.sum_marks_in_parts.pop_back(); } else { @@ -463,7 +454,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( : ReadFromMergeTree::ReadType::InReverseOrder; pipes.emplace_back(read(std::move(new_parts), column_names, read_type, - requested_num_streams, min_marks_for_concurrent_read, use_uncompressed_cache)); + requested_num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache)); } if (need_preliminary_merge) @@ -609,36 +600,14 @@ static void addMergingFinal( Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts_with_range, + RangesInDataParts && parts_with_ranges, const Names & column_names, ActionsDAGPtr & out_projection) { const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); - size_t sum_marks = 0; - size_t adaptive_parts = 0; - for (const auto & part : parts_with_range) - { - for (const auto & range : part.ranges) - sum_marks += range.end - range.begin; - if (part.data_part->index_granularity_info.is_adaptive) - ++adaptive_parts; - } - - size_t index_granularity_bytes = 0; - if (adaptive_parts >= parts_with_range.size() / 2) - index_granularity_bytes = data_settings->index_granularity_bytes; - - const size_t max_marks_to_use_cache = MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( - settings.merge_tree_max_rows_to_use_cache, - settings.merge_tree_max_bytes_to_use_cache, - data_settings->index_granularity, - index_granularity_bytes); - - bool use_uncompressed_cache = settings.use_uncompressed_cache; - if (sum_marks > max_marks_to_use_cache) - use_uncompressed_cache = false; + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); size_t num_streams = requested_num_streams; if (num_streams > settings.max_final_threads) @@ -650,15 +619,15 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// then we will create a pipe for each partition that will run selecting processor and merging processor /// for the parts with this partition. In the end we will unite all the pipes. std::vector parts_to_merge_ranges; - auto it = parts_with_range.begin(); + auto it = parts_with_ranges.begin(); parts_to_merge_ranges.push_back(it); if (settings.do_not_merge_across_partitions_select_final) { - while (it != parts_with_range.end()) + while (it != parts_with_ranges.end()) { it = std::find_if( - it, parts_with_range.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); + it, parts_with_ranges.end(), [&it](auto & part) { return it->data_part->info.partition_id != part.data_part->info.partition_id; }); parts_to_merge_ranges.push_back(it); } /// We divide threads for each partition equally. But we will create at least the number of partitions threads. @@ -668,7 +637,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( else { /// If do_not_merge_across_partitions_select_final is false we just merge all the parts. - parts_to_merge_ranges.push_back(parts_with_range.end()); + parts_to_merge_ranges.push_back(parts_with_ranges.end()); } Pipes partition_pipes; @@ -711,7 +680,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( continue; pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, - num_streams, 0, use_uncompressed_cache); + num_streams, 0, info.use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) @@ -761,11 +730,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( 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, data_settings->index_granularity, - index_granularity_bytes, + info.index_granularity_bytes, sum_marks_in_lonely_parts); /// Reduce the number of num_streams_for_lonely_parts if the data is small. @@ -773,7 +743,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, - num_streams_for_lonely_parts, min_marks_for_concurrent_read, use_uncompressed_cache); + num_streams_for_lonely_parts, min_marks_for_concurrent_read, info.use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a89563682bc..ca9f03d26f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3855,8 +3855,6 @@ static void selectBestProjection( if (projection_parts.empty()) return; - //candidate.merge_tree_data_select_base_cache = std::make_unique(); - //candidate.merge_tree_data_select_projection_cache = std::make_unique(); auto sum_marks = reader.estimateNumMarksToRead( projection_parts, candidate.required_columns, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index caf69b8ecfb..2d3722ce7cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -166,11 +166,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( ProjectionDescription::typeToString(query_info.projection->desc->type), query_info.projection->desc->name); - // if (query_info.projection->merge_tree_data_select_base_cache->sum_marks - // + query_info.projection->merge_tree_data_select_projection_cache->sum_marks - // == 0) - // return std::make_unique(); - MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; for (const auto & part : parts) From f9cf7c46e105c1d236c08d5762441c6fc675caf9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Jun 2021 02:22:05 +0300 Subject: [PATCH 296/652] better lock in StorageJoin --- src/Storages/StorageJoin.cpp | 67 +++++++++++-------- src/Storages/StorageJoin.h | 6 +- .../01821_join_table_mutation.reference | 2 + .../0_stateless/01821_join_table_mutation.sql | 18 ++++- .../01821_join_table_race_long.reference | 1 + .../0_stateless/01821_join_table_race_long.sh | 25 +++++++ 6 files changed, 85 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/01821_join_table_race_long.reference create mode 100755 tests/queries/0_stateless/01821_join_table_race_long.sh diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 6f7df6f4252..e271a50e505 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -68,11 +68,18 @@ StorageJoin::StorageJoin( restore(); } +BlockOutputStreamPtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +{ + std::lock_guard mutate_lock(mutate_mutex); + return StorageSetOrJoinBase::write(query, metadata_snapshot, context); +} void StorageJoin::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder&) { - std::lock_guard lock(mutex); + std::lock_guard mutate_lock(mutate_mutex); + std::unique_lock lock(rwlock); + disk->removeRecursive(path); disk->createDirectories(path); disk->createDirectories(path + "tmp/"); @@ -83,45 +90,47 @@ void StorageJoin::truncate( void StorageJoin::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const { - for (const auto & command: commands) - { - switch (command.type) - { - case MutationCommand::Type::DELETE: - break; - case MutationCommand::Type::UPDATE: - throw Exception("Table engine Join doesn't support update mutation, please use insert instead", ErrorCodes::NOT_IMPLEMENTED); - default: - throw Exception("Table engine Join doesn't support this mutation", ErrorCodes::NOT_IMPLEMENTED); - } - } + for (const auto & command : commands) + if (command.type != MutationCommand::DELETE) + throw Exception("Table engine Join supports only DELETE mutations", ErrorCodes::NOT_IMPLEMENTED); } void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) { - // Only delete is supported - std::lock_guard lock(mutex); + /// Firstly accuire lock for mutation, that locks changes of data. + /// We cannot accuire rwlock here, because read lock is needed + /// for execution of mutation interpreter. + std::lock_guard mutate_lock(mutate_mutex); + + constexpr auto tmp_backup_file_name = "tmp/mut.bin"; auto metadata_snapshot = getInMemoryMetadataPtr(); - auto storage = getStorageID(); - auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); - auto in = interpreter->execute(); - in->readPrefix(); - auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); - - const String backup_file_name = "1.bin"; // id starts from 1 - auto backup_buf = disk->writeFile(path + "tmp/" + backup_file_name); + auto backup_buf = disk->writeFile(path + tmp_backup_file_name); auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); - while (const Block & block = in->read()) + auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); + + // New scope controls lifetime of InputStream. { - new_data->addJoinedBlock(block, true); - if (persistent) - backup_stream.write(block); + auto storage_ptr = DatabaseCatalog::instance().getTable(getStorageID(), context); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); + auto in = interpreter->execute(); + in->readPrefix(); + + while (const Block & block = in->read()) + { + new_data->addJoinedBlock(block, true); + if (persistent) + backup_stream.write(block); + } + + in->readSuffix(); } + /// Now accuire exclusive lock and modify storage. + std::unique_lock lock(rwlock); + join = std::move(new_data); increment = 1; @@ -140,7 +149,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) disk->removeFileIfExists(path + file_name); } - disk->replaceFile(path + "tmp/" + backup_file_name, path + backup_file_name); + disk->replaceFile(path + tmp_backup_file_name, path + std::to_string(increment) + ".bin"); } } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 7315115520e..78d8b9768e9 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -29,6 +29,7 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; + /// Only delete is supported. void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void mutate(const MutationCommands & commands, ContextPtr context) override; @@ -44,6 +45,8 @@ public: /// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin) ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, @@ -71,8 +74,7 @@ private: /// Protect state for concurrent use in insertFromBlock and joinBlock. /// Lock is stored in HashJoin instance during query and blocks concurrent insertions. mutable std::shared_mutex rwlock; - - mutable std::mutex mutex; + mutable std::mutex mutate_mutex; void insertBlock(const Block & block) override; void finishInsert() override {} diff --git a/tests/queries/0_stateless/01821_join_table_mutation.reference b/tests/queries/0_stateless/01821_join_table_mutation.reference index e79d145b39b..8c446c806b5 100644 --- a/tests/queries/0_stateless/01821_join_table_mutation.reference +++ b/tests/queries/0_stateless/01821_join_table_mutation.reference @@ -3,3 +3,5 @@ 99 m10 50 +48 +0 diff --git a/tests/queries/0_stateless/01821_join_table_mutation.sql b/tests/queries/0_stateless/01821_join_table_mutation.sql index 9662a197b88..78903ebd6ec 100644 --- a/tests/queries/0_stateless/01821_join_table_mutation.sql +++ b/tests/queries/0_stateless/01821_join_table_mutation.sql @@ -4,13 +4,13 @@ CREATE TABLE join_table_mutation(id Int32, name String) ENGINE = Join(ANY, LEFT, INSERT INTO join_table_mutation select number, toString(number) from numbers(100); -SELECT count(1) FROM join_table_mutation; +SELECT count() FROM join_table_mutation; SELECT name FROM join_table_mutation WHERE id = 10; ALTER TABLE join_table_mutation DELETE WHERE id = 10; -SELECT count(1) FROM join_table_mutation; +SELECT count() FROM join_table_mutation; SELECT name FROM join_table_mutation WHERE id = 10; @@ -20,4 +20,16 @@ SELECT name FROM join_table_mutation WHERE id = 10; ALTER TABLE join_table_mutation DELETE WHERE id % 2 = 0; -SELECT count(1) FROM join_table_mutation; \ No newline at end of file +ALTER TABLE join_table_mutation UPDATE name = 'some' WHERE 1; -- {serverError 48} + +SELECT count() FROM join_table_mutation; + +ALTER TABLE join_table_mutation DELETE WHERE name IN ('1', '2', '3', '4'); + +SELECT count() FROM join_table_mutation; + +ALTER TABLE join_table_mutation DELETE WHERE 1; + +SELECT count() FROM join_table_mutation; + +DROP TABLE join_table_mutation; diff --git a/tests/queries/0_stateless/01821_join_table_race_long.reference b/tests/queries/0_stateless/01821_join_table_race_long.reference new file mode 100644 index 00000000000..f2018833bc6 --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_race_long.reference @@ -0,0 +1 @@ +1 foo diff --git a/tests/queries/0_stateless/01821_join_table_race_long.sh b/tests/queries/0_stateless/01821_join_table_race_long.sh new file mode 100755 index 00000000000..9602da1e12a --- /dev/null +++ b/tests/queries/0_stateless/01821_join_table_race_long.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS join_table_race" +$CLICKHOUSE_CLIENT -q "CREATE TABLE join_table_race(id Int32, name String) ENGINE = Join(ANY, LEFT, id)" + +for i in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & + +for i in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & + +for i in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & + +for i in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & + +wait + +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" +$CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES (1, 'foo')" +$CLICKHOUSE_CLIENT -q "SELECT id, name FROM join_table_race" + +$CLICKHOUSE_CLIENT -q "DROP TABLE join_table_race" From 41cd58efb1bf87d44f7d4a5ce6570b6118a981a7 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 31 May 2021 20:50:39 -0400 Subject: [PATCH 297/652] Adding window functions requirements for leadInFrame, lagInFrame and default frame. --- .../requirements/requirements.md | 328 ++++--- .../requirements/requirements.py | 885 ++++++++++-------- 2 files changed, 708 insertions(+), 505 deletions(-) diff --git a/tests/testflows/window_functions/requirements/requirements.md b/tests/testflows/window_functions/requirements/requirements.md index 1d6dbc28923..d8bfbe30ead 100644 --- a/tests/testflows/window_functions/requirements/requirements.md +++ b/tests/testflows/window_functions/requirements/requirements.md @@ -25,135 +25,136 @@ * 3.4.4 [RQ.SRS-019.ClickHouse.WindowFunctions.OrderClause.InvalidExpr.Error](#rqsrs-019clickhousewindowfunctionsorderclauseinvalidexprerror) * 3.5 [FRAME Clause](#frame-clause) * 3.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause](#rqsrs-019clickhousewindowfunctionsframeclause) - * 3.5.2 [ROWS](#rows) - * 3.5.2.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame](#rqsrs-019clickhousewindowfunctionsrowsframe) - * 3.5.2.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframemissingframeextenterror) - * 3.5.2.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframeinvalidframeextenterror) - * 3.5.2.4 [ROWS CURRENT ROW](#rows-current-row) - * 3.5.2.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframestartcurrentrow) - * 3.5.2.5 [ROWS UNBOUNDED PRECEDING](#rows-unbounded-preceding) - * 3.5.2.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedpreceding) - * 3.5.2.6 [ROWS `expr` PRECEDING](#rows-expr-preceding) - * 3.5.2.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartexprpreceding) - * 3.5.2.7 [ROWS UNBOUNDED FOLLOWING](#rows-unbounded-following) - * 3.5.2.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedfollowingerror) - * 3.5.2.8 [ROWS `expr` FOLLOWING](#rows-expr-following) - * 3.5.2.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartexprfollowingerror) - * 3.5.2.9 [ROWS BETWEEN CURRENT ROW](#rows-between-current-row) - * 3.5.2.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowcurrentrow) - * 3.5.2.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedprecedingerror) - * 3.5.2.9.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprprecedingerror) - * 3.5.2.9.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedfollowing) - * 3.5.2.9.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprfollowing) - * 3.5.2.10 [ROWS BETWEEN UNBOUNDED PRECEDING](#rows-between-unbounded-preceding) - * 3.5.2.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingcurrentrow) - * 3.5.2.10.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedprecedingerror) - * 3.5.2.10.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprpreceding) - * 3.5.2.10.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedfollowing) - * 3.5.2.10.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprfollowing) - * 3.5.2.11 [ROWS BETWEEN UNBOUNDED FOLLOWING](#rows-between-unbounded-following) - * 3.5.2.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedfollowingerror) - * 3.5.2.12 [ROWS BETWEEN `expr` FOLLOWING](#rows-between-expr-following) - * 3.5.2.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingerror) - * 3.5.2.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowingerror) - * 3.5.2.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingunboundedfollowing) - * 3.5.2.12.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowing) - * 3.5.2.13 [ROWS BETWEEN `expr` PRECEDING](#rows-between-expr-preceding) - * 3.5.2.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingcurrentrow) - * 3.5.2.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedprecedingerror) - * 3.5.2.13.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedfollowing) - * 3.5.2.13.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprprecedingerror) - * 3.5.2.13.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprpreceding) - * 3.5.2.13.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprfollowing) - * 3.5.3 [RANGE](#range) - * 3.5.3.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame](#rqsrs-019clickhousewindowfunctionsrangeframe) - * 3.5.3.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesdateanddatetime) - * 3.5.3.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesintanduint) - * 3.5.3.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframemultiplecolumnsinorderbyerror) - * 3.5.3.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframemissingframeextenterror) - * 3.5.3.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframeinvalidframeextenterror) - * 3.5.3.7 [`CURRENT ROW` Peers](#current-row-peers) - * 3.5.3.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers](#rqsrs-019clickhousewindowfunctionsrangeframecurrentrowpeers) - * 3.5.3.9 [RANGE CURRENT ROW](#range-current-row) - * 3.5.3.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithoutorderby) - * 3.5.3.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithorderby) - * 3.5.3.10 [RANGE UNBOUNDED FOLLOWING](#range-unbounded-following) - * 3.5.3.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedfollowingerror) - * 3.5.3.11 [RANGE UNBOUNDED PRECEDING](#range-unbounded-preceding) - * 3.5.3.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithoutorderby) - * 3.5.3.11.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithorderby) - * 3.5.3.12 [RANGE `expr` PRECEDING](#range-expr-preceding) - * 3.5.3.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithoutorderbyerror) - * 3.5.3.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingorderbynonnumericalcolumnerror) - * 3.5.3.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithorderby) - * 3.5.3.13 [RANGE `expr` FOLLOWING](#range-expr-following) - * 3.5.3.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithoutorderbyerror) - * 3.5.3.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithorderbyerror) - * 3.5.3.14 [RANGE BETWEEN CURRENT ROW](#range-between-current-row) - * 3.5.3.14.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowcurrentrow) - * 3.5.3.14.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedprecedingerror) - * 3.5.3.14.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedfollowing) - * 3.5.3.14.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithoutorderbyerror) - * 3.5.3.14.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithorderby) - * 3.5.3.14.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprprecedingerror) - * 3.5.3.15 [RANGE BETWEEN UNBOUNDED PRECEDING](#range-between-unbounded-preceding) - * 3.5.3.15.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingcurrentrow) - * 3.5.3.15.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedprecedingerror) - * 3.5.3.15.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedfollowing) - * 3.5.3.15.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithoutorderbyerror) - * 3.5.3.15.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithorderby) - * 3.5.3.15.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithoutorderbyerror) - * 3.5.3.15.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithorderby) - * 3.5.3.16 [RANGE BETWEEN UNBOUNDED FOLLOWING](#range-between-unbounded-following) - * 3.5.3.16.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingcurrentrowerror) - * 3.5.3.16.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedfollowingerror) - * 3.5.3.16.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedprecedingerror) - * 3.5.3.16.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprprecedingerror) - * 3.5.3.16.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprfollowingerror) - * 3.5.3.17 [RANGE BETWEEN expr PRECEDING](#range-between-expr-preceding) - * 3.5.3.17.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithorderby) - * 3.5.3.17.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithoutorderbyerror) - * 3.5.3.17.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedprecedingerror) - * 3.5.3.17.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithoutorderbyerror) - * 3.5.3.17.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithorderby) - * 3.5.3.17.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithoutorderbyerror) - * 3.5.3.17.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithorderby) - * 3.5.3.17.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithoutorderbyerror) - * 3.5.3.17.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderbyerror) - * 3.5.3.17.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderby) - * 3.5.3.18 [RANGE BETWEEN expr FOLLOWING](#range-between-expr-following) - * 3.5.3.18.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithoutorderbyerror) - * 3.5.3.18.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithorderbyerror) - * 3.5.3.18.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowzerospecialcase) - * 3.5.3.18.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithoutorderbyerror) - * 3.5.3.18.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithorderby) - * 3.5.3.18.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedprecedingerror) - * 3.5.3.18.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithoutorderbyerror) - * 3.5.3.18.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingerror) - * 3.5.3.18.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithorderbyzerospecialcase) - * 3.5.3.18.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithoutorderbyerror) - * 3.5.3.18.11 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderbyerror) - * 3.5.3.18.12 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderby) - * 3.5.4 [Frame Extent](#frame-extent) - * 3.5.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent](#rqsrs-019clickhousewindowfunctionsframeextent) - * 3.5.5 [Frame Start](#frame-start) - * 3.5.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start](#rqsrs-019clickhousewindowfunctionsframestart) - * 3.5.6 [Frame Between](#frame-between) - * 3.5.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between](#rqsrs-019clickhousewindowfunctionsframebetween) - * 3.5.7 [Frame End](#frame-end) - * 3.5.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End](#rqsrs-019clickhousewindowfunctionsframeend) - * 3.5.8 [`CURRENT ROW`](#current-row) - * 3.5.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow](#rqsrs-019clickhousewindowfunctionscurrentrow) - * 3.5.9 [`UNBOUNDED PRECEDING`](#unbounded-preceding) - * 3.5.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsunboundedpreceding) - * 3.5.10 [`UNBOUNDED FOLLOWING`](#unbounded-following) - * 3.5.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsunboundedfollowing) - * 3.5.11 [`expr PRECEDING`](#expr-preceding) - * 3.5.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding](#rqsrs-019clickhousewindowfunctionsexprpreceding) - * 3.5.11.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue](#rqsrs-019clickhousewindowfunctionsexprprecedingexprvalue) - * 3.5.12 [`expr FOLLOWING`](#expr-following) - * 3.5.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing](#rqsrs-019clickhousewindowfunctionsexprfollowing) - * 3.5.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue](#rqsrs-019clickhousewindowfunctionsexprfollowingexprvalue) + * 3.5.2 [RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause.DefaultFrame](#rqsrs-019clickhousewindowfunctionsframeclausedefaultframe) + * 3.5.3 [ROWS](#rows) + * 3.5.3.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame](#rqsrs-019clickhousewindowfunctionsrowsframe) + * 3.5.3.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframemissingframeextenterror) + * 3.5.3.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframeinvalidframeextenterror) + * 3.5.3.4 [ROWS CURRENT ROW](#rows-current-row) + * 3.5.3.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframestartcurrentrow) + * 3.5.3.5 [ROWS UNBOUNDED PRECEDING](#rows-unbounded-preceding) + * 3.5.3.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedpreceding) + * 3.5.3.6 [ROWS `expr` PRECEDING](#rows-expr-preceding) + * 3.5.3.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartexprpreceding) + * 3.5.3.7 [ROWS UNBOUNDED FOLLOWING](#rows-unbounded-following) + * 3.5.3.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedfollowingerror) + * 3.5.3.8 [ROWS `expr` FOLLOWING](#rows-expr-following) + * 3.5.3.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartexprfollowingerror) + * 3.5.3.9 [ROWS BETWEEN CURRENT ROW](#rows-between-current-row) + * 3.5.3.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowcurrentrow) + * 3.5.3.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedprecedingerror) + * 3.5.3.9.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprprecedingerror) + * 3.5.3.9.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedfollowing) + * 3.5.3.9.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprfollowing) + * 3.5.3.10 [ROWS BETWEEN UNBOUNDED PRECEDING](#rows-between-unbounded-preceding) + * 3.5.3.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingcurrentrow) + * 3.5.3.10.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedprecedingerror) + * 3.5.3.10.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprpreceding) + * 3.5.3.10.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedfollowing) + * 3.5.3.10.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprfollowing) + * 3.5.3.11 [ROWS BETWEEN UNBOUNDED FOLLOWING](#rows-between-unbounded-following) + * 3.5.3.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedfollowingerror) + * 3.5.3.12 [ROWS BETWEEN `expr` FOLLOWING](#rows-between-expr-following) + * 3.5.3.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingerror) + * 3.5.3.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowingerror) + * 3.5.3.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingunboundedfollowing) + * 3.5.3.12.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowing) + * 3.5.3.13 [ROWS BETWEEN `expr` PRECEDING](#rows-between-expr-preceding) + * 3.5.3.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingcurrentrow) + * 3.5.3.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedprecedingerror) + * 3.5.3.13.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedfollowing) + * 3.5.3.13.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprprecedingerror) + * 3.5.3.13.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprpreceding) + * 3.5.3.13.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprfollowing) + * 3.5.4 [RANGE](#range) + * 3.5.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame](#rqsrs-019clickhousewindowfunctionsrangeframe) + * 3.5.4.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesdateanddatetime) + * 3.5.4.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesintanduint) + * 3.5.4.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframemultiplecolumnsinorderbyerror) + * 3.5.4.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframemissingframeextenterror) + * 3.5.4.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframeinvalidframeextenterror) + * 3.5.4.7 [`CURRENT ROW` Peers](#current-row-peers) + * 3.5.4.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers](#rqsrs-019clickhousewindowfunctionsrangeframecurrentrowpeers) + * 3.5.4.9 [RANGE CURRENT ROW](#range-current-row) + * 3.5.4.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithoutorderby) + * 3.5.4.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithorderby) + * 3.5.4.10 [RANGE UNBOUNDED FOLLOWING](#range-unbounded-following) + * 3.5.4.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedfollowingerror) + * 3.5.4.11 [RANGE UNBOUNDED PRECEDING](#range-unbounded-preceding) + * 3.5.4.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithoutorderby) + * 3.5.4.11.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithorderby) + * 3.5.4.12 [RANGE `expr` PRECEDING](#range-expr-preceding) + * 3.5.4.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithoutorderbyerror) + * 3.5.4.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingorderbynonnumericalcolumnerror) + * 3.5.4.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithorderby) + * 3.5.4.13 [RANGE `expr` FOLLOWING](#range-expr-following) + * 3.5.4.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithoutorderbyerror) + * 3.5.4.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithorderbyerror) + * 3.5.4.14 [RANGE BETWEEN CURRENT ROW](#range-between-current-row) + * 3.5.4.14.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowcurrentrow) + * 3.5.4.14.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedprecedingerror) + * 3.5.4.14.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedfollowing) + * 3.5.4.14.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithoutorderbyerror) + * 3.5.4.14.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithorderby) + * 3.5.4.14.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprprecedingerror) + * 3.5.4.15 [RANGE BETWEEN UNBOUNDED PRECEDING](#range-between-unbounded-preceding) + * 3.5.4.15.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingcurrentrow) + * 3.5.4.15.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedprecedingerror) + * 3.5.4.15.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedfollowing) + * 3.5.4.15.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithoutorderbyerror) + * 3.5.4.15.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithorderby) + * 3.5.4.15.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithoutorderbyerror) + * 3.5.4.15.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithorderby) + * 3.5.4.16 [RANGE BETWEEN UNBOUNDED FOLLOWING](#range-between-unbounded-following) + * 3.5.4.16.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingcurrentrowerror) + * 3.5.4.16.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedfollowingerror) + * 3.5.4.16.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedprecedingerror) + * 3.5.4.16.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprprecedingerror) + * 3.5.4.16.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprfollowingerror) + * 3.5.4.17 [RANGE BETWEEN expr PRECEDING](#range-between-expr-preceding) + * 3.5.4.17.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithorderby) + * 3.5.4.17.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithoutorderbyerror) + * 3.5.4.17.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedprecedingerror) + * 3.5.4.17.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithoutorderbyerror) + * 3.5.4.17.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithorderby) + * 3.5.4.17.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithoutorderbyerror) + * 3.5.4.17.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithorderby) + * 3.5.4.17.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithoutorderbyerror) + * 3.5.4.17.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderbyerror) + * 3.5.4.17.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderby) + * 3.5.4.18 [RANGE BETWEEN expr FOLLOWING](#range-between-expr-following) + * 3.5.4.18.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithoutorderbyerror) + * 3.5.4.18.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithorderbyerror) + * 3.5.4.18.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowzerospecialcase) + * 3.5.4.18.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithoutorderbyerror) + * 3.5.4.18.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithorderby) + * 3.5.4.18.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedprecedingerror) + * 3.5.4.18.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithoutorderbyerror) + * 3.5.4.18.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingerror) + * 3.5.4.18.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithorderbyzerospecialcase) + * 3.5.4.18.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithoutorderbyerror) + * 3.5.4.18.11 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderbyerror) + * 3.5.4.18.12 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderby) + * 3.5.5 [Frame Extent](#frame-extent) + * 3.5.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent](#rqsrs-019clickhousewindowfunctionsframeextent) + * 3.5.6 [Frame Start](#frame-start) + * 3.5.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start](#rqsrs-019clickhousewindowfunctionsframestart) + * 3.5.7 [Frame Between](#frame-between) + * 3.5.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between](#rqsrs-019clickhousewindowfunctionsframebetween) + * 3.5.8 [Frame End](#frame-end) + * 3.5.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End](#rqsrs-019clickhousewindowfunctionsframeend) + * 3.5.9 [`CURRENT ROW`](#current-row) + * 3.5.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow](#rqsrs-019clickhousewindowfunctionscurrentrow) + * 3.5.10 [`UNBOUNDED PRECEDING`](#unbounded-preceding) + * 3.5.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsunboundedpreceding) + * 3.5.11 [`UNBOUNDED FOLLOWING`](#unbounded-following) + * 3.5.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsunboundedfollowing) + * 3.5.12 [`expr PRECEDING`](#expr-preceding) + * 3.5.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding](#rqsrs-019clickhousewindowfunctionsexprpreceding) + * 3.5.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue](#rqsrs-019clickhousewindowfunctionsexprprecedingexprvalue) + * 3.5.13 [`expr FOLLOWING`](#expr-following) + * 3.5.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing](#rqsrs-019clickhousewindowfunctionsexprfollowing) + * 3.5.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue](#rqsrs-019clickhousewindowfunctionsexprfollowingexprvalue) * 3.6 [WINDOW Clause](#window-clause) * 3.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause](#rqsrs-019clickhousewindowfunctionswindowclause) * 3.6.2 [RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause.MultipleWindows](#rqsrs-019clickhousewindowfunctionswindowclausemultiplewindows) @@ -179,12 +180,16 @@ * 3.8.1.3.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Lag.Workaround](#rqsrs-019clickhousewindowfunctionslagworkaround) * 3.8.1.4 [The `lead(value, offset)` Function Workaround](#the-leadvalue-offset-function-workaround) * 3.8.1.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Lead.Workaround](#rqsrs-019clickhousewindowfunctionsleadworkaround) - * 3.8.1.5 [The `rank()` Function](#the-rank-function) - * 3.8.1.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Rank](#rqsrs-019clickhousewindowfunctionsrank) - * 3.8.1.6 [The `dense_rank()` Function](#the-dense_rank-function) - * 3.8.1.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank](#rqsrs-019clickhousewindowfunctionsdenserank) - * 3.8.1.7 [The `row_number()` Function](#the-row_number-function) - * 3.8.1.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber](#rqsrs-019clickhousewindowfunctionsrownumber) + * 3.8.1.5 [The `leadInFrame(expr[, offset, [default]])`](#the-leadinframeexpr-offset-default) + * 3.8.1.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.LeadInFrame](#rqsrs-019clickhousewindowfunctionsleadinframe) + * 3.8.1.6 [The `lagInFrame(expr[, offset, [default]])`](#the-laginframeexpr-offset-default) + * 3.8.1.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.LagInFrame](#rqsrs-019clickhousewindowfunctionslaginframe) + * 3.8.1.7 [The `rank()` Function](#the-rank-function) + * 3.8.1.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Rank](#rqsrs-019clickhousewindowfunctionsrank) + * 3.8.1.8 [The `dense_rank()` Function](#the-dense_rank-function) + * 3.8.1.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank](#rqsrs-019clickhousewindowfunctionsdenserank) + * 3.8.1.9 [The `row_number()` Function](#the-row_number-function) + * 3.8.1.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber](#rqsrs-019clickhousewindowfunctionsrownumber) * 3.8.2 [Aggregate Functions](#aggregate-functions) * 3.8.2.1 [RQ.SRS-019.ClickHouse.WindowFunctions.AggregateFunctions](#rqsrs-019clickhousewindowfunctionsaggregatefunctions) * 3.8.2.2 [Combinators](#combinators) @@ -352,6 +357,17 @@ frame_clause: {ROWS | RANGE } frame_extent ``` +#### RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause.DefaultFrame +version: 1.0 + +[ClickHouse] SHALL support the default `frame_clause` to be `RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW`. + +If the `ORDER BY` clause is specified then this SHALL set the frame to be all rows from +the partition start up to and including current row and its peers. + +If the `ORDER BY` clause is not specified then this SHALL set the frame to include all rows +in the partition because all the rows are considered to be the peers of the current row. + #### ROWS ##### RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame @@ -2110,6 +2126,48 @@ If `N` or default are missing, the defaults are 1 and NULL, respectively. `N` SHALL be a literal non-negative integer. If `N` is 0, the value SHALL be returned for the current row. +##### The `leadInFrame(expr[, offset, [default]])` + +###### RQ.SRS-019.ClickHouse.WindowFunctions.LeadInFrame +version: 1.0 + +[ClickHouse] SHALL support the `leadInFrame(expr[, offset, [default]])` function. + +For example, +``` +leadInFrame(column) OVER (...) +``` + +The function SHALL return the value from the row that leads (follows) the current row +by the `offset` rows within the current frame. If there is no such row, +the return value SHALL be the `default` value. If the `default` value is not specified +then the default value for the corresponding column data type SHALL be returned. + +The `offset` SHALL be a literal non-negative integer. If the `offset` is set to `0`, then +the value SHALL be returned for the current row. If the `offset` is not specified, the default +value SHALL be `1`. + +##### The `lagInFrame(expr[, offset, [default]])` + +###### RQ.SRS-019.ClickHouse.WindowFunctions.LagInFrame +version: 1.0 + +[ClickHouse] SHALL support the `lagInFrame(expr[, offset, [default]])` function. + +For example, +``` +lagInFrame(column) OVER (...) +``` + +The function SHALL return the value from the row that lags (preceds) the current row +by the `offset` rows within the current frame. If there is no such row, +the return value SHALL be the `default` value. If the `default` value is not specified +then the default value for the corresponding column data type SHALL be returned. + +The `offset` SHALL be a literal non-negative integer. If the `offset` is set to `0`, then +the value SHALL be returned for the current row. If the `offset` is not specified, the default +value SHALL be `1`. + ##### The `rank()` Function ###### RQ.SRS-019.ClickHouse.WindowFunctions.Rank diff --git a/tests/testflows/window_functions/requirements/requirements.py b/tests/testflows/window_functions/requirements/requirements.py index 42389fd5452..9ed86380da2 100644 --- a/tests/testflows/window_functions/requirements/requirements.py +++ b/tests/testflows/window_functions/requirements/requirements.py @@ -259,6 +259,27 @@ RQ_SRS_019_ClickHouse_WindowFunctions_FrameClause = Requirement( level=3, num='3.5.1') +RQ_SRS_019_ClickHouse_WindowFunctions_FrameClause_DefaultFrame = Requirement( + name='RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause.DefaultFrame', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the default `frame_clause` to be `RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW`. \n' + '\n' + 'If the `ORDER BY` clause is specified then this SHALL set the frame to be all rows from \n' + 'the partition start up to and including current row and its peers. \n' + '\n' + 'If the `ORDER BY` clause is not specified then this SHALL set the frame to include all rows\n' + 'in the partition because all the rows are considered to be the peers of the current row.\n' + '\n' + ), + link=None, + level=3, + num='3.5.2') + RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame', version='1.0', @@ -279,7 +300,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame = Requirement( ), link=None, level=4, - num='3.5.2.1') + num='3.5.3.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_MissingFrameExtent_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error', @@ -300,7 +321,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_MissingFrameExtent_Error = Requi ), link=None, level=4, - num='3.5.2.2') + num='3.5.3.2') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_InvalidFrameExtent_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error', @@ -321,7 +342,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_InvalidFrameExtent_Error = Requi ), link=None, level=4, - num='3.5.2.3') + num='3.5.3.3') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow', @@ -350,7 +371,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_CurrentRow = Requirement( ), link=None, level=5, - num='3.5.2.4.1') + num='3.5.3.4.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_UnboundedPreceding = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding', @@ -380,7 +401,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_UnboundedPreceding = Requi ), link=None, level=5, - num='3.5.2.5.1') + num='3.5.3.5.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_ExprPreceding = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding', @@ -410,7 +431,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_ExprPreceding = Requiremen ), link=None, level=5, - num='3.5.2.6.1') + num='3.5.3.6.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_UnboundedFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error', @@ -431,7 +452,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_UnboundedFollowing_Error = ), link=None, level=5, - num='3.5.2.7.1') + num='3.5.3.7.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_ExprFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error', @@ -452,7 +473,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Start_ExprFollowing_Error = Requ ), link=None, level=5, - num='3.5.2.8.1') + num='3.5.3.8.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow', @@ -481,7 +502,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_CurrentRow = ), link=None, level=5, - num='3.5.2.9.1') + num='3.5.3.9.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error', @@ -496,7 +517,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_UnboundedPrec ), link=None, level=5, - num='3.5.2.9.2') + num='3.5.3.9.2') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_ExprPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error', @@ -511,7 +532,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_ExprPreceding ), link=None, level=5, - num='3.5.2.9.3') + num='3.5.3.9.3') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing', @@ -541,7 +562,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_UnboundedFoll ), link=None, level=5, - num='3.5.2.9.4') + num='3.5.3.9.4') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_ExprFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing', @@ -571,7 +592,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_CurrentRow_ExprFollowing ), link=None, level=5, - num='3.5.2.9.5') + num='3.5.3.9.5') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow', @@ -601,7 +622,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_Curre ), link=None, level=5, - num='3.5.2.10.1') + num='3.5.3.10.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error', @@ -622,7 +643,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_Unbou ), link=None, level=5, - num='3.5.2.10.2') + num='3.5.3.10.2') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_ExprPreceding = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding', @@ -652,7 +673,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_ExprP ), link=None, level=5, - num='3.5.2.10.3') + num='3.5.3.10.3') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing', @@ -682,7 +703,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_Unbou ), link=None, level=5, - num='3.5.2.10.4') + num='3.5.3.10.4') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_ExprFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing', @@ -712,7 +733,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedPreceding_ExprF ), link=None, level=5, - num='3.5.2.10.5') + num='3.5.3.10.5') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error', @@ -739,7 +760,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_UnboundedFollowing_Error ), link=None, level=5, - num='3.5.2.11.1') + num='3.5.3.11.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error', @@ -766,7 +787,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_Error = Re ), link=None, level=5, - num='3.5.2.12.1') + num='3.5.3.12.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_ExprFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error', @@ -787,7 +808,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_ExprFollow ), link=None, level=5, - num='3.5.2.12.2') + num='3.5.3.12.2') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing', @@ -818,7 +839,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_UnboundedF ), link=None, level=5, - num='3.5.2.12.3') + num='3.5.3.12.3') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_ExprFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing', @@ -850,7 +871,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprFollowing_ExprFollow ), link=None, level=5, - num='3.5.2.12.4') + num='3.5.3.12.4') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow', @@ -881,7 +902,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_CurrentRow ), link=None, level=5, - num='3.5.2.13.1') + num='3.5.3.13.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error', @@ -903,7 +924,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_UnboundedP ), link=None, level=5, - num='3.5.2.13.2') + num='3.5.3.13.2') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing', @@ -934,7 +955,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_UnboundedF ), link=None, level=5, - num='3.5.2.13.3') + num='3.5.3.13.3') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_ExprPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error', @@ -957,7 +978,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_ExprPreced ), link=None, level=5, - num='3.5.2.13.4') + num='3.5.3.13.4') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_ExprPreceding = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding', @@ -989,7 +1010,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_ExprPreced ), link=None, level=5, - num='3.5.2.13.5') + num='3.5.3.13.5') RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_ExprFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing', @@ -1020,7 +1041,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_Between_ExprPreceding_ExprFollow ), link=None, level=5, - num='3.5.2.13.6') + num='3.5.3.13.6') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame', @@ -1042,7 +1063,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame = Requirement( ), link=None, level=4, - num='3.5.3.1') + num='3.5.4.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_DataTypes_DateAndDateTime = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime', @@ -1058,7 +1079,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_DataTypes_DateAndDateTime = Req ), link=None, level=4, - num='3.5.3.2') + num='3.5.4.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_DataTypes_IntAndUInt = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt', @@ -1074,7 +1095,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_DataTypes_IntAndUInt = Requirem ), link=None, level=4, - num='3.5.3.3') + num='3.5.4.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_MultipleColumnsInOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error', @@ -1090,7 +1111,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_MultipleColumnsInOrderBy_Error ), link=None, level=4, - num='3.5.3.4') + num='3.5.4.4') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_MissingFrameExtent_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error', @@ -1105,7 +1126,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_MissingFrameExtent_Error = Requ ), link=None, level=4, - num='3.5.3.5') + num='3.5.4.5') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_InvalidFrameExtent_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error', @@ -1120,7 +1141,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_InvalidFrameExtent_Error = Requ ), link=None, level=4, - num='3.5.3.6') + num='3.5.4.6') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_CurrentRow_Peers = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers', @@ -1136,7 +1157,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_CurrentRow_Peers = Requirement( ), link=None, level=4, - num='3.5.3.8') + num='3.5.4.8') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_CurrentRow_WithoutOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy', @@ -1166,7 +1187,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_CurrentRow_WithoutOrderBy ), link=None, level=5, - num='3.5.3.9.1') + num='3.5.4.9.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_CurrentRow_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy', @@ -1197,7 +1218,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_CurrentRow_WithOrderBy = ), link=None, level=5, - num='3.5.3.9.2') + num='3.5.4.9.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_UnboundedFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error', @@ -1219,7 +1240,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_UnboundedFollowing_Error ), link=None, level=5, - num='3.5.3.10.1') + num='3.5.4.10.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_UnboundedPreceding_WithoutOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy', @@ -1249,7 +1270,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_UnboundedPreceding_Withou ), link=None, level=5, - num='3.5.3.11.1') + num='3.5.4.11.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_UnboundedPreceding_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy', @@ -1280,7 +1301,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_UnboundedPreceding_WithOr ), link=None, level=5, - num='3.5.3.11.2') + num='3.5.4.11.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprPreceding_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error', @@ -1301,7 +1322,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprPreceding_WithoutOrde ), link=None, level=5, - num='3.5.3.12.1') + num='3.5.4.12.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprPreceding_OrderByNonNumericalColumn_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error', @@ -1317,7 +1338,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprPreceding_OrderByNonN ), link=None, level=5, - num='3.5.3.12.2') + num='3.5.4.12.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprPreceding_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy', @@ -1348,7 +1369,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprPreceding_WithOrderBy ), link=None, level=5, - num='3.5.3.12.3') + num='3.5.4.12.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error', @@ -1369,7 +1390,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprFollowing_WithoutOrde ), link=None, level=5, - num='3.5.3.13.1') + num='3.5.4.13.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprFollowing_WithOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error', @@ -1391,7 +1412,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Start_ExprFollowing_WithOrderBy ), link=None, level=5, - num='3.5.3.13.2') + num='3.5.4.13.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow', @@ -1437,7 +1458,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_CurrentRow = ), link=None, level=5, - num='3.5.3.14.1') + num='3.5.4.14.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error', @@ -1467,7 +1488,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_UnboundedPre ), link=None, level=5, - num='3.5.3.14.2') + num='3.5.4.14.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing', @@ -1514,7 +1535,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_UnboundedFol ), link=None, level=5, - num='3.5.3.14.3') + num='3.5.4.14.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_ExprFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error', @@ -1536,7 +1557,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_ExprFollowin ), link=None, level=5, - num='3.5.3.14.4') + num='3.5.4.14.4') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_ExprFollowing_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy', @@ -1568,7 +1589,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_ExprFollowin ), link=None, level=5, - num='3.5.3.14.5') + num='3.5.4.14.5') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_ExprPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error', @@ -1598,7 +1619,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_CurrentRow_ExprPrecedin ), link=None, level=5, - num='3.5.3.14.6') + num='3.5.4.14.6') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow', @@ -1647,7 +1668,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Curr ), link=None, level=5, - num='3.5.3.15.1') + num='3.5.4.15.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error', @@ -1677,7 +1698,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Unbo ), link=None, level=5, - num='3.5.3.15.2') + num='3.5.4.15.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing', @@ -1725,7 +1746,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Unbo ), link=None, level=5, - num='3.5.3.15.3') + num='3.5.4.15.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_ExprPreceding_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error', @@ -1747,7 +1768,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Expr ), link=None, level=5, - num='3.5.3.15.4') + num='3.5.4.15.4') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_ExprPreceding_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy', @@ -1779,7 +1800,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Expr ), link=None, level=5, - num='3.5.3.15.5') + num='3.5.4.15.5') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_ExprFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error', @@ -1801,7 +1822,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Expr ), link=None, level=5, - num='3.5.3.15.6') + num='3.5.4.15.6') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_ExprFollowing_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy', @@ -1833,7 +1854,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedPreceding_Expr ), link=None, level=5, - num='3.5.3.15.7') + num='3.5.4.15.7') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_CurrentRow_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error', @@ -1849,7 +1870,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_Curr ), link=None, level=5, - num='3.5.3.16.1') + num='3.5.4.16.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_UnboundedFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error', @@ -1865,7 +1886,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_Unbo ), link=None, level=5, - num='3.5.3.16.2') + num='3.5.4.16.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error', @@ -1881,7 +1902,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_Unbo ), link=None, level=5, - num='3.5.3.16.3') + num='3.5.4.16.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_ExprPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error', @@ -1897,7 +1918,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_Expr ), link=None, level=5, - num='3.5.3.16.4') + num='3.5.4.16.4') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_ExprFollowing_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error', @@ -1913,7 +1934,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_UnboundedFollowing_Expr ), link=None, level=5, - num='3.5.3.16.5') + num='3.5.4.16.5') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_CurrentRow_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy', @@ -1945,7 +1966,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_CurrentRo ), link=None, level=5, - num='3.5.3.17.1') + num='3.5.4.17.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_CurrentRow_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error', @@ -1967,7 +1988,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_CurrentRo ), link=None, level=5, - num='3.5.3.17.2') + num='3.5.4.17.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error', @@ -1997,7 +2018,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_Unbounded ), link=None, level=5, - num='3.5.3.17.3') + num='3.5.4.17.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_UnboundedFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error', @@ -2019,7 +2040,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_Unbounded ), link=None, level=5, - num='3.5.3.17.4') + num='3.5.4.17.4') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_UnboundedFollowing_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy', @@ -2051,7 +2072,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_Unbounded ), link=None, level=5, - num='3.5.3.17.5') + num='3.5.4.17.5') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error', @@ -2073,7 +2094,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprFollo ), link=None, level=5, - num='3.5.3.17.6') + num='3.5.4.17.6') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprFollowing_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy', @@ -2105,7 +2126,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprFollo ), link=None, level=5, - num='3.5.3.17.7') + num='3.5.4.17.7') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprPreceding_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error', @@ -2127,7 +2148,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprPrece ), link=None, level=5, - num='3.5.3.17.8') + num='3.5.4.17.8') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprPreceding_WithOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error', @@ -2150,7 +2171,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprPrece ), link=None, level=5, - num='3.5.3.17.9') + num='3.5.4.17.9') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprPreceding_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy', @@ -2200,7 +2221,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprPreceding_ExprPrece ), link=None, level=5, - num='3.5.3.17.10') + num='3.5.4.17.10') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_CurrentRow_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error', @@ -2222,7 +2243,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_CurrentRo ), link=None, level=5, - num='3.5.3.18.1') + num='3.5.4.18.1') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_CurrentRow_WithOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error', @@ -2244,7 +2265,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_CurrentRo ), link=None, level=5, - num='3.5.3.18.2') + num='3.5.4.18.2') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_CurrentRow_ZeroSpecialCase = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase', @@ -2293,7 +2314,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_CurrentRo ), link=None, level=5, - num='3.5.3.18.3') + num='3.5.4.18.3') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_UnboundedFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error', @@ -2315,7 +2336,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_Unbounded ), link=None, level=5, - num='3.5.3.18.4') + num='3.5.4.18.4') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_UnboundedFollowing_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy', @@ -2347,7 +2368,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_Unbounded ), link=None, level=5, - num='3.5.3.18.5') + num='3.5.4.18.5') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_UnboundedPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error', @@ -2377,7 +2398,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_Unbounded ), link=None, level=5, - num='3.5.3.18.6') + num='3.5.4.18.6') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprPreceding_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error', @@ -2393,7 +2414,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprPrece ), link=None, level=5, - num='3.5.3.18.7') + num='3.5.4.18.7') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprPreceding_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error', @@ -2409,7 +2430,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprPrece ), link=None, level=5, - num='3.5.3.18.8') + num='3.5.4.18.8') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprPreceding_WithOrderBy_ZeroSpecialCase = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase', @@ -2441,7 +2462,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprPrece ), link=None, level=5, - num='3.5.3.18.9') + num='3.5.4.18.9') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprFollowing_WithoutOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error', @@ -2457,7 +2478,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprFollo ), link=None, level=5, - num='3.5.3.18.10') + num='3.5.4.18.10') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprFollowing_WithOrderBy_Error = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error', @@ -2479,7 +2500,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprFollo ), link=None, level=5, - num='3.5.3.18.11') + num='3.5.4.18.11') RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprFollowing_WithOrderBy = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy', @@ -2513,7 +2534,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RangeFrame_Between_ExprFollowing_ExprFollo ), link=None, level=5, - num='3.5.3.18.12') + num='3.5.4.18.12') RQ_SRS_019_ClickHouse_WindowFunctions_Frame_Extent = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent', @@ -2533,7 +2554,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_Frame_Extent = Requirement( ), link=None, level=4, - num='3.5.4.1') + num='3.5.5.1') RQ_SRS_019_ClickHouse_WindowFunctions_Frame_Start = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start', @@ -2558,7 +2579,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_Frame_Start = Requirement( ), link=None, level=4, - num='3.5.5.1') + num='3.5.6.1') RQ_SRS_019_ClickHouse_WindowFunctions_Frame_Between = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between', @@ -2578,7 +2599,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_Frame_Between = Requirement( ), link=None, level=4, - num='3.5.6.1') + num='3.5.7.1') RQ_SRS_019_ClickHouse_WindowFunctions_Frame_End = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End', @@ -2603,7 +2624,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_Frame_End = Requirement( ), link=None, level=4, - num='3.5.7.1') + num='3.5.8.1') RQ_SRS_019_ClickHouse_WindowFunctions_CurrentRow = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow', @@ -2621,7 +2642,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_CurrentRow = Requirement( ), link=None, level=4, - num='3.5.8.1') + num='3.5.9.1') RQ_SRS_019_ClickHouse_WindowFunctions_UnboundedPreceding = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding', @@ -2637,7 +2658,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_UnboundedPreceding = Requirement( ), link=None, level=4, - num='3.5.9.1') + num='3.5.10.1') RQ_SRS_019_ClickHouse_WindowFunctions_UnboundedFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing', @@ -2653,7 +2674,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_UnboundedFollowing = Requirement( ), link=None, level=4, - num='3.5.10.1') + num='3.5.11.1') RQ_SRS_019_ClickHouse_WindowFunctions_ExprPreceding = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding', @@ -2671,7 +2692,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_ExprPreceding = Requirement( ), link=None, level=4, - num='3.5.11.1') + num='3.5.12.1') RQ_SRS_019_ClickHouse_WindowFunctions_ExprPreceding_ExprValue = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue', @@ -2692,7 +2713,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_ExprPreceding_ExprValue = Requirement( ), link=None, level=4, - num='3.5.11.2') + num='3.5.12.2') RQ_SRS_019_ClickHouse_WindowFunctions_ExprFollowing = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing', @@ -2710,7 +2731,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_ExprFollowing = Requirement( ), link=None, level=4, - num='3.5.12.1') + num='3.5.13.1') RQ_SRS_019_ClickHouse_WindowFunctions_ExprFollowing_ExprValue = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue', @@ -2731,7 +2752,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_ExprFollowing_ExprValue = Requirement( ), link=None, level=4, - num='3.5.12.2') + num='3.5.13.2') RQ_SRS_019_ClickHouse_WindowFunctions_WindowClause = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause', @@ -3041,6 +3062,64 @@ RQ_SRS_019_ClickHouse_WindowFunctions_Lead_Workaround = Requirement( level=5, num='3.8.1.4.1') +RQ_SRS_019_ClickHouse_WindowFunctions_LeadInFrame = Requirement( + name='RQ.SRS-019.ClickHouse.WindowFunctions.LeadInFrame', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `leadInFrame(expr[, offset, [default]])` function.\n' + '\n' + 'For example,\n' + '```\n' + 'leadInFrame(column) OVER (...)\n' + '```\n' + '\n' + 'The function SHALL return the value from the row that leads (follows) the current row\n' + 'by the `offset` rows within the current frame. If there is no such row,\n' + 'the return value SHALL be the `default` value. If the `default` value is not specified \n' + 'then the default value for the corresponding column data type SHALL be returned.\n' + '\n' + 'The `offset` SHALL be a literal non-negative integer. If the `offset` is set to `0`, then\n' + 'the value SHALL be returned for the current row. If the `offset` is not specified, the default\n' + 'value SHALL be `1`.\n' + '\n' + ), + link=None, + level=5, + num='3.8.1.5.1') + +RQ_SRS_019_ClickHouse_WindowFunctions_LagInFrame = Requirement( + name='RQ.SRS-019.ClickHouse.WindowFunctions.LagInFrame', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `lagInFrame(expr[, offset, [default]])` function.\n' + '\n' + 'For example,\n' + '```\n' + 'lagInFrame(column) OVER (...)\n' + '```\n' + '\n' + 'The function SHALL return the value from the row that lags (preceds) the current row\n' + 'by the `offset` rows within the current frame. If there is no such row,\n' + 'the return value SHALL be the `default` value. If the `default` value is not specified \n' + 'then the default value for the corresponding column data type SHALL be returned.\n' + '\n' + 'The `offset` SHALL be a literal non-negative integer. If the `offset` is set to `0`, then\n' + 'the value SHALL be returned for the current row. If the `offset` is not specified, the default\n' + 'value SHALL be `1`.\n' + '\n' + ), + link=None, + level=5, + num='3.8.1.6.1') + RQ_SRS_019_ClickHouse_WindowFunctions_Rank = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.Rank', version='1.0', @@ -3066,7 +3145,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_Rank = Requirement( ), link=None, level=5, - num='3.8.1.5.1') + num='3.8.1.7.1') RQ_SRS_019_ClickHouse_WindowFunctions_DenseRank = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank', @@ -3093,7 +3172,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_DenseRank = Requirement( ), link=None, level=5, - num='3.8.1.6.1') + num='3.8.1.8.1') RQ_SRS_019_ClickHouse_WindowFunctions_RowNumber = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber', @@ -3118,7 +3197,7 @@ RQ_SRS_019_ClickHouse_WindowFunctions_RowNumber = Requirement( ), link=None, level=5, - num='3.8.1.7.1') + num='3.8.1.9.1') RQ_SRS_019_ClickHouse_WindowFunctions_AggregateFunctions = Requirement( name='RQ.SRS-019.ClickHouse.WindowFunctions.AggregateFunctions', @@ -3291,135 +3370,136 @@ SRS019_ClickHouse_Window_Functions = Specification( Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.OrderClause.InvalidExpr.Error', level=3, num='3.4.4'), Heading(name='FRAME Clause', level=2, num='3.5'), Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause', level=3, num='3.5.1'), - Heading(name='ROWS', level=3, num='3.5.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame', level=4, num='3.5.2.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error', level=4, num='3.5.2.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error', level=4, num='3.5.2.3'), - Heading(name='ROWS CURRENT ROW', level=4, num='3.5.2.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow', level=5, num='3.5.2.4.1'), - Heading(name='ROWS UNBOUNDED PRECEDING', level=4, num='3.5.2.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding', level=5, num='3.5.2.5.1'), - Heading(name='ROWS `expr` PRECEDING', level=4, num='3.5.2.6'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding', level=5, num='3.5.2.6.1'), - Heading(name='ROWS UNBOUNDED FOLLOWING', level=4, num='3.5.2.7'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error', level=5, num='3.5.2.7.1'), - Heading(name='ROWS `expr` FOLLOWING', level=4, num='3.5.2.8'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error', level=5, num='3.5.2.8.1'), - Heading(name='ROWS BETWEEN CURRENT ROW', level=4, num='3.5.2.9'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow', level=5, num='3.5.2.9.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error', level=5, num='3.5.2.9.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error', level=5, num='3.5.2.9.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing', level=5, num='3.5.2.9.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing', level=5, num='3.5.2.9.5'), - Heading(name='ROWS BETWEEN UNBOUNDED PRECEDING', level=4, num='3.5.2.10'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow', level=5, num='3.5.2.10.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error', level=5, num='3.5.2.10.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding', level=5, num='3.5.2.10.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing', level=5, num='3.5.2.10.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing', level=5, num='3.5.2.10.5'), - Heading(name='ROWS BETWEEN UNBOUNDED FOLLOWING', level=4, num='3.5.2.11'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error', level=5, num='3.5.2.11.1'), - Heading(name='ROWS BETWEEN `expr` FOLLOWING', level=4, num='3.5.2.12'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error', level=5, num='3.5.2.12.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error', level=5, num='3.5.2.12.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing', level=5, num='3.5.2.12.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing', level=5, num='3.5.2.12.4'), - Heading(name='ROWS BETWEEN `expr` PRECEDING', level=4, num='3.5.2.13'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow', level=5, num='3.5.2.13.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error', level=5, num='3.5.2.13.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing', level=5, num='3.5.2.13.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error', level=5, num='3.5.2.13.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding', level=5, num='3.5.2.13.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing', level=5, num='3.5.2.13.6'), - Heading(name='RANGE', level=3, num='3.5.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame', level=4, num='3.5.3.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime', level=4, num='3.5.3.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt', level=4, num='3.5.3.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error', level=4, num='3.5.3.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error', level=4, num='3.5.3.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error', level=4, num='3.5.3.6'), - Heading(name='`CURRENT ROW` Peers', level=4, num='3.5.3.7'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers', level=4, num='3.5.3.8'), - Heading(name='RANGE CURRENT ROW', level=4, num='3.5.3.9'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy', level=5, num='3.5.3.9.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy', level=5, num='3.5.3.9.2'), - Heading(name='RANGE UNBOUNDED FOLLOWING', level=4, num='3.5.3.10'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error', level=5, num='3.5.3.10.1'), - Heading(name='RANGE UNBOUNDED PRECEDING', level=4, num='3.5.3.11'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy', level=5, num='3.5.3.11.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy', level=5, num='3.5.3.11.2'), - Heading(name='RANGE `expr` PRECEDING', level=4, num='3.5.3.12'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.3.12.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error', level=5, num='3.5.3.12.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy', level=5, num='3.5.3.12.3'), - Heading(name='RANGE `expr` FOLLOWING', level=4, num='3.5.3.13'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.13.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error', level=5, num='3.5.3.13.2'), - Heading(name='RANGE BETWEEN CURRENT ROW', level=4, num='3.5.3.14'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow', level=5, num='3.5.3.14.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error', level=5, num='3.5.3.14.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing', level=5, num='3.5.3.14.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.14.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy', level=5, num='3.5.3.14.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error', level=5, num='3.5.3.14.6'), - Heading(name='RANGE BETWEEN UNBOUNDED PRECEDING', level=4, num='3.5.3.15'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow', level=5, num='3.5.3.15.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error', level=5, num='3.5.3.15.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing', level=5, num='3.5.3.15.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.3.15.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy', level=5, num='3.5.3.15.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.15.6'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy', level=5, num='3.5.3.15.7'), - Heading(name='RANGE BETWEEN UNBOUNDED FOLLOWING', level=4, num='3.5.3.16'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error', level=5, num='3.5.3.16.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error', level=5, num='3.5.3.16.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error', level=5, num='3.5.3.16.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error', level=5, num='3.5.3.16.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error', level=5, num='3.5.3.16.5'), - Heading(name='RANGE BETWEEN expr PRECEDING', level=4, num='3.5.3.17'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy', level=5, num='3.5.3.17.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error', level=5, num='3.5.3.17.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error', level=5, num='3.5.3.17.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.17.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy', level=5, num='3.5.3.17.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.17.6'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy', level=5, num='3.5.3.17.7'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.3.17.8'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error', level=5, num='3.5.3.17.9'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy', level=5, num='3.5.3.17.10'), - Heading(name='RANGE BETWEEN expr FOLLOWING', level=4, num='3.5.3.18'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error', level=5, num='3.5.3.18.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error', level=5, num='3.5.3.18.2'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase', level=5, num='3.5.3.18.3'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.18.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy', level=5, num='3.5.3.18.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error', level=5, num='3.5.3.18.6'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.3.18.7'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error', level=5, num='3.5.3.18.8'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase', level=5, num='3.5.3.18.9'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.3.18.10'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error', level=5, num='3.5.3.18.11'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy', level=5, num='3.5.3.18.12'), - Heading(name='Frame Extent', level=3, num='3.5.4'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent', level=4, num='3.5.4.1'), - Heading(name='Frame Start', level=3, num='3.5.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start', level=4, num='3.5.5.1'), - Heading(name='Frame Between', level=3, num='3.5.6'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between', level=4, num='3.5.6.1'), - Heading(name='Frame End', level=3, num='3.5.7'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End', level=4, num='3.5.7.1'), - Heading(name='`CURRENT ROW`', level=3, num='3.5.8'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow', level=4, num='3.5.8.1'), - Heading(name='`UNBOUNDED PRECEDING`', level=3, num='3.5.9'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding', level=4, num='3.5.9.1'), - Heading(name='`UNBOUNDED FOLLOWING`', level=3, num='3.5.10'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing', level=4, num='3.5.10.1'), - Heading(name='`expr PRECEDING`', level=3, num='3.5.11'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding', level=4, num='3.5.11.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue', level=4, num='3.5.11.2'), - Heading(name='`expr FOLLOWING`', level=3, num='3.5.12'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing', level=4, num='3.5.12.1'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue', level=4, num='3.5.12.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause.DefaultFrame', level=3, num='3.5.2'), + Heading(name='ROWS', level=3, num='3.5.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame', level=4, num='3.5.3.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error', level=4, num='3.5.3.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error', level=4, num='3.5.3.3'), + Heading(name='ROWS CURRENT ROW', level=4, num='3.5.3.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow', level=5, num='3.5.3.4.1'), + Heading(name='ROWS UNBOUNDED PRECEDING', level=4, num='3.5.3.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding', level=5, num='3.5.3.5.1'), + Heading(name='ROWS `expr` PRECEDING', level=4, num='3.5.3.6'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding', level=5, num='3.5.3.6.1'), + Heading(name='ROWS UNBOUNDED FOLLOWING', level=4, num='3.5.3.7'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error', level=5, num='3.5.3.7.1'), + Heading(name='ROWS `expr` FOLLOWING', level=4, num='3.5.3.8'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error', level=5, num='3.5.3.8.1'), + Heading(name='ROWS BETWEEN CURRENT ROW', level=4, num='3.5.3.9'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow', level=5, num='3.5.3.9.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error', level=5, num='3.5.3.9.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error', level=5, num='3.5.3.9.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing', level=5, num='3.5.3.9.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing', level=5, num='3.5.3.9.5'), + Heading(name='ROWS BETWEEN UNBOUNDED PRECEDING', level=4, num='3.5.3.10'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow', level=5, num='3.5.3.10.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error', level=5, num='3.5.3.10.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding', level=5, num='3.5.3.10.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing', level=5, num='3.5.3.10.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing', level=5, num='3.5.3.10.5'), + Heading(name='ROWS BETWEEN UNBOUNDED FOLLOWING', level=4, num='3.5.3.11'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error', level=5, num='3.5.3.11.1'), + Heading(name='ROWS BETWEEN `expr` FOLLOWING', level=4, num='3.5.3.12'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error', level=5, num='3.5.3.12.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error', level=5, num='3.5.3.12.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing', level=5, num='3.5.3.12.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing', level=5, num='3.5.3.12.4'), + Heading(name='ROWS BETWEEN `expr` PRECEDING', level=4, num='3.5.3.13'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow', level=5, num='3.5.3.13.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error', level=5, num='3.5.3.13.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing', level=5, num='3.5.3.13.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error', level=5, num='3.5.3.13.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding', level=5, num='3.5.3.13.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing', level=5, num='3.5.3.13.6'), + Heading(name='RANGE', level=3, num='3.5.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame', level=4, num='3.5.4.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime', level=4, num='3.5.4.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt', level=4, num='3.5.4.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error', level=4, num='3.5.4.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error', level=4, num='3.5.4.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error', level=4, num='3.5.4.6'), + Heading(name='`CURRENT ROW` Peers', level=4, num='3.5.4.7'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers', level=4, num='3.5.4.8'), + Heading(name='RANGE CURRENT ROW', level=4, num='3.5.4.9'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy', level=5, num='3.5.4.9.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy', level=5, num='3.5.4.9.2'), + Heading(name='RANGE UNBOUNDED FOLLOWING', level=4, num='3.5.4.10'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error', level=5, num='3.5.4.10.1'), + Heading(name='RANGE UNBOUNDED PRECEDING', level=4, num='3.5.4.11'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy', level=5, num='3.5.4.11.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy', level=5, num='3.5.4.11.2'), + Heading(name='RANGE `expr` PRECEDING', level=4, num='3.5.4.12'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.4.12.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error', level=5, num='3.5.4.12.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy', level=5, num='3.5.4.12.3'), + Heading(name='RANGE `expr` FOLLOWING', level=4, num='3.5.4.13'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.13.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error', level=5, num='3.5.4.13.2'), + Heading(name='RANGE BETWEEN CURRENT ROW', level=4, num='3.5.4.14'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow', level=5, num='3.5.4.14.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error', level=5, num='3.5.4.14.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing', level=5, num='3.5.4.14.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.14.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy', level=5, num='3.5.4.14.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error', level=5, num='3.5.4.14.6'), + Heading(name='RANGE BETWEEN UNBOUNDED PRECEDING', level=4, num='3.5.4.15'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow', level=5, num='3.5.4.15.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error', level=5, num='3.5.4.15.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing', level=5, num='3.5.4.15.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.4.15.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy', level=5, num='3.5.4.15.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.15.6'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy', level=5, num='3.5.4.15.7'), + Heading(name='RANGE BETWEEN UNBOUNDED FOLLOWING', level=4, num='3.5.4.16'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error', level=5, num='3.5.4.16.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error', level=5, num='3.5.4.16.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error', level=5, num='3.5.4.16.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error', level=5, num='3.5.4.16.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error', level=5, num='3.5.4.16.5'), + Heading(name='RANGE BETWEEN expr PRECEDING', level=4, num='3.5.4.17'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy', level=5, num='3.5.4.17.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error', level=5, num='3.5.4.17.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error', level=5, num='3.5.4.17.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.17.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy', level=5, num='3.5.4.17.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.17.6'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy', level=5, num='3.5.4.17.7'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.4.17.8'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error', level=5, num='3.5.4.17.9'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy', level=5, num='3.5.4.17.10'), + Heading(name='RANGE BETWEEN expr FOLLOWING', level=4, num='3.5.4.18'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error', level=5, num='3.5.4.18.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error', level=5, num='3.5.4.18.2'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase', level=5, num='3.5.4.18.3'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.18.4'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy', level=5, num='3.5.4.18.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error', level=5, num='3.5.4.18.6'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error', level=5, num='3.5.4.18.7'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error', level=5, num='3.5.4.18.8'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase', level=5, num='3.5.4.18.9'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error', level=5, num='3.5.4.18.10'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error', level=5, num='3.5.4.18.11'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy', level=5, num='3.5.4.18.12'), + Heading(name='Frame Extent', level=3, num='3.5.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent', level=4, num='3.5.5.1'), + Heading(name='Frame Start', level=3, num='3.5.6'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start', level=4, num='3.5.6.1'), + Heading(name='Frame Between', level=3, num='3.5.7'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between', level=4, num='3.5.7.1'), + Heading(name='Frame End', level=3, num='3.5.8'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End', level=4, num='3.5.8.1'), + Heading(name='`CURRENT ROW`', level=3, num='3.5.9'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow', level=4, num='3.5.9.1'), + Heading(name='`UNBOUNDED PRECEDING`', level=3, num='3.5.10'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding', level=4, num='3.5.10.1'), + Heading(name='`UNBOUNDED FOLLOWING`', level=3, num='3.5.11'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing', level=4, num='3.5.11.1'), + Heading(name='`expr PRECEDING`', level=3, num='3.5.12'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding', level=4, num='3.5.12.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue', level=4, num='3.5.12.2'), + Heading(name='`expr FOLLOWING`', level=3, num='3.5.13'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing', level=4, num='3.5.13.1'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue', level=4, num='3.5.13.2'), Heading(name='WINDOW Clause', level=2, num='3.6'), Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause', level=3, num='3.6.1'), Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause.MultipleWindows', level=3, num='3.6.2'), @@ -3445,12 +3525,16 @@ SRS019_ClickHouse_Window_Functions = Specification( Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Lag.Workaround', level=5, num='3.8.1.3.1'), Heading(name='The `lead(value, offset)` Function Workaround', level=4, num='3.8.1.4'), Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Lead.Workaround', level=5, num='3.8.1.4.1'), - Heading(name='The `rank()` Function', level=4, num='3.8.1.5'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Rank', level=5, num='3.8.1.5.1'), - Heading(name='The `dense_rank()` Function', level=4, num='3.8.1.6'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank', level=5, num='3.8.1.6.1'), - Heading(name='The `row_number()` Function', level=4, num='3.8.1.7'), - Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber', level=5, num='3.8.1.7.1'), + Heading(name='The `leadInFrame(expr[, offset, [default]])`', level=4, num='3.8.1.5'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.LeadInFrame', level=5, num='3.8.1.5.1'), + Heading(name='The `lagInFrame(expr[, offset, [default]])`', level=4, num='3.8.1.6'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.LagInFrame', level=5, num='3.8.1.6.1'), + Heading(name='The `rank()` Function', level=4, num='3.8.1.7'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.Rank', level=5, num='3.8.1.7.1'), + Heading(name='The `dense_rank()` Function', level=4, num='3.8.1.8'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank', level=5, num='3.8.1.8.1'), + Heading(name='The `row_number()` Function', level=4, num='3.8.1.9'), + Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber', level=5, num='3.8.1.9.1'), Heading(name='Aggregate Functions', level=3, num='3.8.2'), Heading(name='RQ.SRS-019.ClickHouse.WindowFunctions.AggregateFunctions', level=4, num='3.8.2.1'), Heading(name='Combinators', level=4, num='3.8.2.2'), @@ -3473,6 +3557,7 @@ SRS019_ClickHouse_Window_Functions = Specification( RQ_SRS_019_ClickHouse_WindowFunctions_OrderClause_MissingExpr_Error, RQ_SRS_019_ClickHouse_WindowFunctions_OrderClause_InvalidExpr_Error, RQ_SRS_019_ClickHouse_WindowFunctions_FrameClause, + RQ_SRS_019_ClickHouse_WindowFunctions_FrameClause_DefaultFrame, RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame, RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_MissingFrameExtent_Error, RQ_SRS_019_ClickHouse_WindowFunctions_RowsFrame_InvalidFrameExtent_Error, @@ -3584,6 +3669,8 @@ SRS019_ClickHouse_Window_Functions = Specification( RQ_SRS_019_ClickHouse_WindowFunctions_LastValue, RQ_SRS_019_ClickHouse_WindowFunctions_Lag_Workaround, RQ_SRS_019_ClickHouse_WindowFunctions_Lead_Workaround, + RQ_SRS_019_ClickHouse_WindowFunctions_LeadInFrame, + RQ_SRS_019_ClickHouse_WindowFunctions_LagInFrame, RQ_SRS_019_ClickHouse_WindowFunctions_Rank, RQ_SRS_019_ClickHouse_WindowFunctions_DenseRank, RQ_SRS_019_ClickHouse_WindowFunctions_RowNumber, @@ -3619,135 +3706,136 @@ SRS019_ClickHouse_Window_Functions = Specification( * 3.4.4 [RQ.SRS-019.ClickHouse.WindowFunctions.OrderClause.InvalidExpr.Error](#rqsrs-019clickhousewindowfunctionsorderclauseinvalidexprerror) * 3.5 [FRAME Clause](#frame-clause) * 3.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause](#rqsrs-019clickhousewindowfunctionsframeclause) - * 3.5.2 [ROWS](#rows) - * 3.5.2.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame](#rqsrs-019clickhousewindowfunctionsrowsframe) - * 3.5.2.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframemissingframeextenterror) - * 3.5.2.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframeinvalidframeextenterror) - * 3.5.2.4 [ROWS CURRENT ROW](#rows-current-row) - * 3.5.2.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframestartcurrentrow) - * 3.5.2.5 [ROWS UNBOUNDED PRECEDING](#rows-unbounded-preceding) - * 3.5.2.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedpreceding) - * 3.5.2.6 [ROWS `expr` PRECEDING](#rows-expr-preceding) - * 3.5.2.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartexprpreceding) - * 3.5.2.7 [ROWS UNBOUNDED FOLLOWING](#rows-unbounded-following) - * 3.5.2.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedfollowingerror) - * 3.5.2.8 [ROWS `expr` FOLLOWING](#rows-expr-following) - * 3.5.2.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartexprfollowingerror) - * 3.5.2.9 [ROWS BETWEEN CURRENT ROW](#rows-between-current-row) - * 3.5.2.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowcurrentrow) - * 3.5.2.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedprecedingerror) - * 3.5.2.9.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprprecedingerror) - * 3.5.2.9.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedfollowing) - * 3.5.2.9.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprfollowing) - * 3.5.2.10 [ROWS BETWEEN UNBOUNDED PRECEDING](#rows-between-unbounded-preceding) - * 3.5.2.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingcurrentrow) - * 3.5.2.10.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedprecedingerror) - * 3.5.2.10.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprpreceding) - * 3.5.2.10.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedfollowing) - * 3.5.2.10.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprfollowing) - * 3.5.2.11 [ROWS BETWEEN UNBOUNDED FOLLOWING](#rows-between-unbounded-following) - * 3.5.2.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedfollowingerror) - * 3.5.2.12 [ROWS BETWEEN `expr` FOLLOWING](#rows-between-expr-following) - * 3.5.2.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingerror) - * 3.5.2.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowingerror) - * 3.5.2.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingunboundedfollowing) - * 3.5.2.12.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowing) - * 3.5.2.13 [ROWS BETWEEN `expr` PRECEDING](#rows-between-expr-preceding) - * 3.5.2.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingcurrentrow) - * 3.5.2.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedprecedingerror) - * 3.5.2.13.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedfollowing) - * 3.5.2.13.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprprecedingerror) - * 3.5.2.13.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprpreceding) - * 3.5.2.13.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprfollowing) - * 3.5.3 [RANGE](#range) - * 3.5.3.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame](#rqsrs-019clickhousewindowfunctionsrangeframe) - * 3.5.3.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesdateanddatetime) - * 3.5.3.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesintanduint) - * 3.5.3.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframemultiplecolumnsinorderbyerror) - * 3.5.3.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframemissingframeextenterror) - * 3.5.3.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframeinvalidframeextenterror) - * 3.5.3.7 [`CURRENT ROW` Peers](#current-row-peers) - * 3.5.3.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers](#rqsrs-019clickhousewindowfunctionsrangeframecurrentrowpeers) - * 3.5.3.9 [RANGE CURRENT ROW](#range-current-row) - * 3.5.3.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithoutorderby) - * 3.5.3.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithorderby) - * 3.5.3.10 [RANGE UNBOUNDED FOLLOWING](#range-unbounded-following) - * 3.5.3.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedfollowingerror) - * 3.5.3.11 [RANGE UNBOUNDED PRECEDING](#range-unbounded-preceding) - * 3.5.3.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithoutorderby) - * 3.5.3.11.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithorderby) - * 3.5.3.12 [RANGE `expr` PRECEDING](#range-expr-preceding) - * 3.5.3.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithoutorderbyerror) - * 3.5.3.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingorderbynonnumericalcolumnerror) - * 3.5.3.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithorderby) - * 3.5.3.13 [RANGE `expr` FOLLOWING](#range-expr-following) - * 3.5.3.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithoutorderbyerror) - * 3.5.3.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithorderbyerror) - * 3.5.3.14 [RANGE BETWEEN CURRENT ROW](#range-between-current-row) - * 3.5.3.14.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowcurrentrow) - * 3.5.3.14.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedprecedingerror) - * 3.5.3.14.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedfollowing) - * 3.5.3.14.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithoutorderbyerror) - * 3.5.3.14.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithorderby) - * 3.5.3.14.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprprecedingerror) - * 3.5.3.15 [RANGE BETWEEN UNBOUNDED PRECEDING](#range-between-unbounded-preceding) - * 3.5.3.15.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingcurrentrow) - * 3.5.3.15.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedprecedingerror) - * 3.5.3.15.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedfollowing) - * 3.5.3.15.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithoutorderbyerror) - * 3.5.3.15.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithorderby) - * 3.5.3.15.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithoutorderbyerror) - * 3.5.3.15.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithorderby) - * 3.5.3.16 [RANGE BETWEEN UNBOUNDED FOLLOWING](#range-between-unbounded-following) - * 3.5.3.16.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingcurrentrowerror) - * 3.5.3.16.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedfollowingerror) - * 3.5.3.16.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedprecedingerror) - * 3.5.3.16.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprprecedingerror) - * 3.5.3.16.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprfollowingerror) - * 3.5.3.17 [RANGE BETWEEN expr PRECEDING](#range-between-expr-preceding) - * 3.5.3.17.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithorderby) - * 3.5.3.17.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithoutorderbyerror) - * 3.5.3.17.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedprecedingerror) - * 3.5.3.17.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithoutorderbyerror) - * 3.5.3.17.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithorderby) - * 3.5.3.17.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithoutorderbyerror) - * 3.5.3.17.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithorderby) - * 3.5.3.17.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithoutorderbyerror) - * 3.5.3.17.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderbyerror) - * 3.5.3.17.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderby) - * 3.5.3.18 [RANGE BETWEEN expr FOLLOWING](#range-between-expr-following) - * 3.5.3.18.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithoutorderbyerror) - * 3.5.3.18.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithorderbyerror) - * 3.5.3.18.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowzerospecialcase) - * 3.5.3.18.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithoutorderbyerror) - * 3.5.3.18.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithorderby) - * 3.5.3.18.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedprecedingerror) - * 3.5.3.18.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithoutorderbyerror) - * 3.5.3.18.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingerror) - * 3.5.3.18.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithorderbyzerospecialcase) - * 3.5.3.18.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithoutorderbyerror) - * 3.5.3.18.11 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderbyerror) - * 3.5.3.18.12 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderby) - * 3.5.4 [Frame Extent](#frame-extent) - * 3.5.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent](#rqsrs-019clickhousewindowfunctionsframeextent) - * 3.5.5 [Frame Start](#frame-start) - * 3.5.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start](#rqsrs-019clickhousewindowfunctionsframestart) - * 3.5.6 [Frame Between](#frame-between) - * 3.5.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between](#rqsrs-019clickhousewindowfunctionsframebetween) - * 3.5.7 [Frame End](#frame-end) - * 3.5.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End](#rqsrs-019clickhousewindowfunctionsframeend) - * 3.5.8 [`CURRENT ROW`](#current-row) - * 3.5.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow](#rqsrs-019clickhousewindowfunctionscurrentrow) - * 3.5.9 [`UNBOUNDED PRECEDING`](#unbounded-preceding) - * 3.5.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsunboundedpreceding) - * 3.5.10 [`UNBOUNDED FOLLOWING`](#unbounded-following) - * 3.5.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsunboundedfollowing) - * 3.5.11 [`expr PRECEDING`](#expr-preceding) - * 3.5.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding](#rqsrs-019clickhousewindowfunctionsexprpreceding) - * 3.5.11.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue](#rqsrs-019clickhousewindowfunctionsexprprecedingexprvalue) - * 3.5.12 [`expr FOLLOWING`](#expr-following) - * 3.5.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing](#rqsrs-019clickhousewindowfunctionsexprfollowing) - * 3.5.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue](#rqsrs-019clickhousewindowfunctionsexprfollowingexprvalue) + * 3.5.2 [RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause.DefaultFrame](#rqsrs-019clickhousewindowfunctionsframeclausedefaultframe) + * 3.5.3 [ROWS](#rows) + * 3.5.3.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame](#rqsrs-019clickhousewindowfunctionsrowsframe) + * 3.5.3.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframemissingframeextenterror) + * 3.5.3.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrowsframeinvalidframeextenterror) + * 3.5.3.4 [ROWS CURRENT ROW](#rows-current-row) + * 3.5.3.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframestartcurrentrow) + * 3.5.3.5 [ROWS UNBOUNDED PRECEDING](#rows-unbounded-preceding) + * 3.5.3.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedpreceding) + * 3.5.3.6 [ROWS `expr` PRECEDING](#rows-expr-preceding) + * 3.5.3.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframestartexprpreceding) + * 3.5.3.7 [ROWS UNBOUNDED FOLLOWING](#rows-unbounded-following) + * 3.5.3.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartunboundedfollowingerror) + * 3.5.3.8 [ROWS `expr` FOLLOWING](#rows-expr-following) + * 3.5.3.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Start.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframestartexprfollowingerror) + * 3.5.3.9 [ROWS BETWEEN CURRENT ROW](#rows-between-current-row) + * 3.5.3.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowcurrentrow) + * 3.5.3.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedprecedingerror) + * 3.5.3.9.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprprecedingerror) + * 3.5.3.9.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowunboundedfollowing) + * 3.5.3.9.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.CurrentRow.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweencurrentrowexprfollowing) + * 3.5.3.10 [ROWS BETWEEN UNBOUNDED PRECEDING](#rows-between-unbounded-preceding) + * 3.5.3.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingcurrentrow) + * 3.5.3.10.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedprecedingerror) + * 3.5.3.10.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprpreceding) + * 3.5.3.10.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingunboundedfollowing) + * 3.5.3.10.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedprecedingexprfollowing) + * 3.5.3.11 [ROWS BETWEEN UNBOUNDED FOLLOWING](#rows-between-unbounded-following) + * 3.5.3.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenunboundedfollowingerror) + * 3.5.3.12 [ROWS BETWEEN `expr` FOLLOWING](#rows-between-expr-following) + * 3.5.3.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingerror) + * 3.5.3.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowingerror) + * 3.5.3.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingunboundedfollowing) + * 3.5.3.12.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprFollowing.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprfollowingexprfollowing) + * 3.5.3.13 [ROWS BETWEEN `expr` PRECEDING](#rows-between-expr-preceding) + * 3.5.3.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingcurrentrow) + * 3.5.3.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedprecedingerror) + * 3.5.3.13.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingunboundedfollowing) + * 3.5.3.13.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprprecedingerror) + * 3.5.3.13.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprPreceding](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprpreceding) + * 3.5.3.13.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame.Between.ExprPreceding.ExprFollowing](#rqsrs-019clickhousewindowfunctionsrowsframebetweenexprprecedingexprfollowing) + * 3.5.4 [RANGE](#range) + * 3.5.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame](#rqsrs-019clickhousewindowfunctionsrangeframe) + * 3.5.4.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.DateAndDateTime](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesdateanddatetime) + * 3.5.4.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.DataTypes.IntAndUInt](#rqsrs-019clickhousewindowfunctionsrangeframedatatypesintanduint) + * 3.5.4.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MultipleColumnsInOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframemultiplecolumnsinorderbyerror) + * 3.5.4.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.MissingFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframemissingframeextenterror) + * 3.5.4.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.InvalidFrameExtent.Error](#rqsrs-019clickhousewindowfunctionsrangeframeinvalidframeextenterror) + * 3.5.4.7 [`CURRENT ROW` Peers](#current-row-peers) + * 3.5.4.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.CurrentRow.Peers](#rqsrs-019clickhousewindowfunctionsrangeframecurrentrowpeers) + * 3.5.4.9 [RANGE CURRENT ROW](#range-current-row) + * 3.5.4.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithoutorderby) + * 3.5.4.9.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartcurrentrowwithorderby) + * 3.5.4.10 [RANGE UNBOUNDED FOLLOWING](#range-unbounded-following) + * 3.5.4.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedfollowingerror) + * 3.5.4.11 [RANGE UNBOUNDED PRECEDING](#range-unbounded-preceding) + * 3.5.4.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithoutOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithoutorderby) + * 3.5.4.11.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.UnboundedPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartunboundedprecedingwithorderby) + * 3.5.4.12 [RANGE `expr` PRECEDING](#range-expr-preceding) + * 3.5.4.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithoutorderbyerror) + * 3.5.4.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.OrderByNonNumericalColumn.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingorderbynonnumericalcolumnerror) + * 3.5.4.12.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframestartexprprecedingwithorderby) + * 3.5.4.13 [RANGE `expr` FOLLOWING](#range-expr-following) + * 3.5.4.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithoutorderbyerror) + * 3.5.4.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Start.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframestartexprfollowingwithorderbyerror) + * 3.5.4.14 [RANGE BETWEEN CURRENT ROW](#range-between-current-row) + * 3.5.4.14.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowcurrentrow) + * 3.5.4.14.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedprecedingerror) + * 3.5.4.14.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowunboundedfollowing) + * 3.5.4.14.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithoutorderbyerror) + * 3.5.4.14.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprfollowingwithorderby) + * 3.5.4.14.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.CurrentRow.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweencurrentrowexprprecedingerror) + * 3.5.4.15 [RANGE BETWEEN UNBOUNDED PRECEDING](#range-between-unbounded-preceding) + * 3.5.4.15.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.CurrentRow](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingcurrentrow) + * 3.5.4.15.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedprecedingerror) + * 3.5.4.15.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingunboundedfollowing) + * 3.5.4.15.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithoutorderbyerror) + * 3.5.4.15.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprprecedingwithorderby) + * 3.5.4.15.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithoutorderbyerror) + * 3.5.4.15.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedprecedingexprfollowingwithorderby) + * 3.5.4.16 [RANGE BETWEEN UNBOUNDED FOLLOWING](#range-between-unbounded-following) + * 3.5.4.16.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.CurrentRow.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingcurrentrowerror) + * 3.5.4.16.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedfollowingerror) + * 3.5.4.16.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingunboundedprecedingerror) + * 3.5.4.16.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprprecedingerror) + * 3.5.4.16.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.UnboundedFollowing.ExprFollowing.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenunboundedfollowingexprfollowingerror) + * 3.5.4.17 [RANGE BETWEEN expr PRECEDING](#range-between-expr-preceding) + * 3.5.4.17.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithorderby) + * 3.5.4.17.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingcurrentrowwithoutorderbyerror) + * 3.5.4.17.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedprecedingerror) + * 3.5.4.17.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithoutorderbyerror) + * 3.5.4.17.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingunboundedfollowingwithorderby) + * 3.5.4.17.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithoutorderbyerror) + * 3.5.4.17.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprfollowingwithorderby) + * 3.5.4.17.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithoutorderbyerror) + * 3.5.4.17.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderbyerror) + * 3.5.4.17.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprPreceding.ExprPreceding.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprprecedingexprprecedingwithorderby) + * 3.5.4.18 [RANGE BETWEEN expr FOLLOWING](#range-between-expr-following) + * 3.5.4.18.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithoutorderbyerror) + * 3.5.4.18.2 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowwithorderbyerror) + * 3.5.4.18.3 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.CurrentRow.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingcurrentrowzerospecialcase) + * 3.5.4.18.4 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithoutorderbyerror) + * 3.5.4.18.5 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedfollowingwithorderby) + * 3.5.4.18.6 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.UnboundedPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingunboundedprecedingerror) + * 3.5.4.18.7 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithoutorderbyerror) + * 3.5.4.18.8 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingerror) + * 3.5.4.18.9 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprPreceding.WithOrderBy.ZeroSpecialCase](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprprecedingwithorderbyzerospecialcase) + * 3.5.4.18.10 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithoutOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithoutorderbyerror) + * 3.5.4.18.11 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy.Error](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderbyerror) + * 3.5.4.18.12 [RQ.SRS-019.ClickHouse.WindowFunctions.RangeFrame.Between.ExprFollowing.ExprFollowing.WithOrderBy](#rqsrs-019clickhousewindowfunctionsrangeframebetweenexprfollowingexprfollowingwithorderby) + * 3.5.5 [Frame Extent](#frame-extent) + * 3.5.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Extent](#rqsrs-019clickhousewindowfunctionsframeextent) + * 3.5.6 [Frame Start](#frame-start) + * 3.5.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Start](#rqsrs-019clickhousewindowfunctionsframestart) + * 3.5.7 [Frame Between](#frame-between) + * 3.5.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.Between](#rqsrs-019clickhousewindowfunctionsframebetween) + * 3.5.8 [Frame End](#frame-end) + * 3.5.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Frame.End](#rqsrs-019clickhousewindowfunctionsframeend) + * 3.5.9 [`CURRENT ROW`](#current-row) + * 3.5.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.CurrentRow](#rqsrs-019clickhousewindowfunctionscurrentrow) + * 3.5.10 [`UNBOUNDED PRECEDING`](#unbounded-preceding) + * 3.5.10.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedPreceding](#rqsrs-019clickhousewindowfunctionsunboundedpreceding) + * 3.5.11 [`UNBOUNDED FOLLOWING`](#unbounded-following) + * 3.5.11.1 [RQ.SRS-019.ClickHouse.WindowFunctions.UnboundedFollowing](#rqsrs-019clickhousewindowfunctionsunboundedfollowing) + * 3.5.12 [`expr PRECEDING`](#expr-preceding) + * 3.5.12.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding](#rqsrs-019clickhousewindowfunctionsexprpreceding) + * 3.5.12.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprPreceding.ExprValue](#rqsrs-019clickhousewindowfunctionsexprprecedingexprvalue) + * 3.5.13 [`expr FOLLOWING`](#expr-following) + * 3.5.13.1 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing](#rqsrs-019clickhousewindowfunctionsexprfollowing) + * 3.5.13.2 [RQ.SRS-019.ClickHouse.WindowFunctions.ExprFollowing.ExprValue](#rqsrs-019clickhousewindowfunctionsexprfollowingexprvalue) * 3.6 [WINDOW Clause](#window-clause) * 3.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause](#rqsrs-019clickhousewindowfunctionswindowclause) * 3.6.2 [RQ.SRS-019.ClickHouse.WindowFunctions.WindowClause.MultipleWindows](#rqsrs-019clickhousewindowfunctionswindowclausemultiplewindows) @@ -3773,12 +3861,16 @@ SRS019_ClickHouse_Window_Functions = Specification( * 3.8.1.3.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Lag.Workaround](#rqsrs-019clickhousewindowfunctionslagworkaround) * 3.8.1.4 [The `lead(value, offset)` Function Workaround](#the-leadvalue-offset-function-workaround) * 3.8.1.4.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Lead.Workaround](#rqsrs-019clickhousewindowfunctionsleadworkaround) - * 3.8.1.5 [The `rank()` Function](#the-rank-function) - * 3.8.1.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Rank](#rqsrs-019clickhousewindowfunctionsrank) - * 3.8.1.6 [The `dense_rank()` Function](#the-dense_rank-function) - * 3.8.1.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank](#rqsrs-019clickhousewindowfunctionsdenserank) - * 3.8.1.7 [The `row_number()` Function](#the-row_number-function) - * 3.8.1.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber](#rqsrs-019clickhousewindowfunctionsrownumber) + * 3.8.1.5 [The `leadInFrame(expr[, offset, [default]])`](#the-leadinframeexpr-offset-default) + * 3.8.1.5.1 [RQ.SRS-019.ClickHouse.WindowFunctions.LeadInFrame](#rqsrs-019clickhousewindowfunctionsleadinframe) + * 3.8.1.6 [The `lagInFrame(expr[, offset, [default]])`](#the-laginframeexpr-offset-default) + * 3.8.1.6.1 [RQ.SRS-019.ClickHouse.WindowFunctions.LagInFrame](#rqsrs-019clickhousewindowfunctionslaginframe) + * 3.8.1.7 [The `rank()` Function](#the-rank-function) + * 3.8.1.7.1 [RQ.SRS-019.ClickHouse.WindowFunctions.Rank](#rqsrs-019clickhousewindowfunctionsrank) + * 3.8.1.8 [The `dense_rank()` Function](#the-dense_rank-function) + * 3.8.1.8.1 [RQ.SRS-019.ClickHouse.WindowFunctions.DenseRank](#rqsrs-019clickhousewindowfunctionsdenserank) + * 3.8.1.9 [The `row_number()` Function](#the-row_number-function) + * 3.8.1.9.1 [RQ.SRS-019.ClickHouse.WindowFunctions.RowNumber](#rqsrs-019clickhousewindowfunctionsrownumber) * 3.8.2 [Aggregate Functions](#aggregate-functions) * 3.8.2.1 [RQ.SRS-019.ClickHouse.WindowFunctions.AggregateFunctions](#rqsrs-019clickhousewindowfunctionsaggregatefunctions) * 3.8.2.2 [Combinators](#combinators) @@ -3946,6 +4038,17 @@ frame_clause: {ROWS | RANGE } frame_extent ``` +#### RQ.SRS-019.ClickHouse.WindowFunctions.FrameClause.DefaultFrame +version: 1.0 + +[ClickHouse] SHALL support the default `frame_clause` to be `RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW`. + +If the `ORDER BY` clause is specified then this SHALL set the frame to be all rows from +the partition start up to and including current row and its peers. + +If the `ORDER BY` clause is not specified then this SHALL set the frame to include all rows +in the partition because all the rows are considered to be the peers of the current row. + #### ROWS ##### RQ.SRS-019.ClickHouse.WindowFunctions.RowsFrame @@ -5704,6 +5807,48 @@ If `N` or default are missing, the defaults are 1 and NULL, respectively. `N` SHALL be a literal non-negative integer. If `N` is 0, the value SHALL be returned for the current row. +##### The `leadInFrame(expr[, offset, [default]])` + +###### RQ.SRS-019.ClickHouse.WindowFunctions.LeadInFrame +version: 1.0 + +[ClickHouse] SHALL support the `leadInFrame(expr[, offset, [default]])` function. + +For example, +``` +leadInFrame(column) OVER (...) +``` + +The function SHALL return the value from the row that leads (follows) the current row +by the `offset` rows within the current frame. If there is no such row, +the return value SHALL be the `default` value. If the `default` value is not specified +then the default value for the corresponding column data type SHALL be returned. + +The `offset` SHALL be a literal non-negative integer. If the `offset` is set to `0`, then +the value SHALL be returned for the current row. If the `offset` is not specified, the default +value SHALL be `1`. + +##### The `lagInFrame(expr[, offset, [default]])` + +###### RQ.SRS-019.ClickHouse.WindowFunctions.LagInFrame +version: 1.0 + +[ClickHouse] SHALL support the `lagInFrame(expr[, offset, [default]])` function. + +For example, +``` +lagInFrame(column) OVER (...) +``` + +The function SHALL return the value from the row that lags (preceds) the current row +by the `offset` rows within the current frame. If there is no such row, +the return value SHALL be the `default` value. If the `default` value is not specified +then the default value for the corresponding column data type SHALL be returned. + +The `offset` SHALL be a literal non-negative integer. If the `offset` is set to `0`, then +the value SHALL be returned for the current row. If the `offset` is not specified, the default +value SHALL be `1`. + ##### The `rank()` Function ###### RQ.SRS-019.ClickHouse.WindowFunctions.Rank From aac97b65829cb3248232a5b1c22fbb0c629663fa Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Jun 2021 10:02:22 +0300 Subject: [PATCH 298/652] test_always_fetch_merged --- tests/integration/test_always_fetch_merged/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index 7ba8e05129b..e3b2d5ca392 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -22,15 +22,15 @@ def started_cluster(): def test_replica_always_download(started_cluster): - node1.query(""" - CREATE TABLE test_table( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS test_table( key UInt64, value String ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table/replicated', '1') ORDER BY tuple() """) - node2.query(""" - CREATE TABLE test_table( + node2.query_with_retry(""" + CREATE TABLE IF NOT EXISTS test_table( key UInt64, value String ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table/replicated', '2') @@ -42,12 +42,12 @@ def test_replica_always_download(started_cluster): node1.query("SYSTEM STOP MERGES") for i in range(0, 10): - node1.query("INSERT INTO test_table VALUES ({}, '{}')".format(i, i)) + node1.query_with_retry("INSERT INTO test_table VALUES ({}, '{}')".format(i, i)) assert node1.query("SELECT COUNT() FROM test_table") == "10\n" assert_eq_with_retry(node2, "SELECT COUNT() FROM test_table", "10\n") - time.sleep(3) + time.sleep(5) # Nothing is merged assert node1.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1") == "10\n" From 0bd323b3cd533a4cd4cc6cb998403d416c2b3ca8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Jun 2021 10:58:53 +0300 Subject: [PATCH 299/652] Fix build --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 62fed1c8e6f..2a1bf7d87b4 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 62fed1c8e6fb863fcd49829c812a76be005ab756 +Subproject commit 2a1bf7d87b4a03561fc66fbb49cee8a288983c5d From 739ff5fee7fbb68a0fc42d95f0312b942f6aa81a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Jun 2021 11:43:44 +0300 Subject: [PATCH 300/652] increase docker client timeout --- tests/integration/helpers/cluster.py | 2 +- tests/integration/helpers/network.py | 2 +- tests/integration/test_mysql_protocol/test.py | 8 ++++---- tests/integration/test_postgresql_protocol/test.py | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index bdb5cc78dff..c6fdbb49f5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1234,7 +1234,7 @@ class ClickHouseCluster: instance.create_dir(destroy_dir=destroy_dirs) _create_env_file(os.path.join(self.env_file), self.env_variables) - self.docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180) + self.docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=600) common_opts = ['up', '-d'] diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e4b9501a18f..c2ec2dd8089 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -164,7 +164,7 @@ class _NetworkManager: self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout - self._docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=docker_api_version, timeout=180) + self._docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=docker_api_version, timeout=600) self._container = None diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 5f76c9e098b..f2d3f46b9bc 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -36,7 +36,7 @@ def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') @@ -44,7 +44,7 @@ def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') @@ -52,7 +52,7 @@ def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') @@ -60,7 +60,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_java1_1') def test_mysql_client(started_cluster): diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 39ccb273299..7bea9569880 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') @@ -65,7 +65,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_java_1') def test_psql_is_ready(psql_server): From 7656ab3c850dd107b2ce652d16ffc2244cadc18e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Jun 2021 11:44:09 +0300 Subject: [PATCH 301/652] try tmpfs in runner --- tests/integration/ci-runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 39ba416f147..288ce89e7aa 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -261,7 +261,7 @@ class ClickhouseIntegrationTestsRunner: def _get_all_tests(self, repo_path): image_cmd = self._get_runner_image_cmd(repo_path) - cmd = "cd {}/tests/integration && ./runner {} ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt".format(repo_path, image_cmd) + cmd = "cd {}/tests/integration && ./runner --tmpfs {} ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt".format(repo_path, image_cmd) logging.info("Getting all tests with cmd '%s'", cmd) subprocess.check_call(cmd, shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL @@ -377,7 +377,7 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && ./runner {} -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, parallel_cmd, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: From feae36636cacd4986be48532c2ca661ff13c56fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Jun 2021 11:53:17 +0300 Subject: [PATCH 302/652] Simplify code around TraceCollector, alternative to #24829 --- src/Common/StackTrace.h | 17 +++++------------ src/Common/TraceCollector.cpp | 30 +++++++++++++++++++----------- 2 files changed, 24 insertions(+), 23 deletions(-) diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index ef90a0d587d..8f8c88c29fe 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -37,18 +37,11 @@ public: std::optional line; }; - static constexpr size_t capacity = -#ifndef NDEBUG - /* The stacks are normally larger in debug version due to less inlining. - * - * NOTE: it cannot be larger then 56 right now, since otherwise it will - * not fit into minimal PIPE_BUF (512) in TraceCollector. - */ - 56 -#else - 32 -#endif - ; + /* NOTE: It cannot be larger right now, since otherwise it + * will not fit into minimal PIPE_BUF (512) in TraceCollector. + */ + static constexpr size_t capacity = 45; + using FramePointers = std::array; using Frames = std::array; diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index 3cb4e5615c8..1c6074c5cf9 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -28,6 +28,7 @@ namespace /// `aggregating_merge_tree_simple_aggregate_function_string.query100.profile100`, /// so make some allowance for them as well. constexpr size_t QUERY_ID_MAX_LEN = 128; + static_assert(QUERY_ID_MAX_LEN <= std::numeric_limits::max()); } LazyPipeFDs pipe; @@ -61,17 +62,19 @@ TraceCollector::~TraceCollector() void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trace, Int64 size) { - constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag - 8 * sizeof(char) + // maximum VarUInt length for string size - QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length - sizeof(UInt8) + // number of stack frames - sizeof(StackTrace::FramePointers) + // collected stack trace, maximum capacity - sizeof(TraceType) + // trace type - sizeof(UInt64) + // thread_id - sizeof(Int64); // size + constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag + + sizeof(UInt8) /// String size + + QUERY_ID_MAX_LEN /// Maximum query_id length + + sizeof(UInt8) /// Number of stack frames + + sizeof(StackTrace::FramePointers) /// Collected stack trace, maximum capacity + + sizeof(TraceType) /// trace type + + sizeof(UInt64) /// thread_id + + sizeof(Int64); /// size + /// Write should be atomic to avoid overlaps /// (since recursive collect() is possible) - static_assert(buf_size < PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic"); + static_assert(PIPE_BUF >= 512); + static_assert(buf_size <= 512, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512"); char buffer[buf_size]; WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer); @@ -92,7 +95,9 @@ void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trac } writeChar(false, out); /// true if requested to stop the collecting thread. - writeStringBinary(query_id, out); + + writeBinary(static_cast(query_id.size), out); + out.write(query_id.data, query_id.size); size_t stack_trace_size = stack_trace.getSize(); size_t stack_trace_offset = stack_trace.getOffset(); @@ -137,7 +142,10 @@ void TraceCollector::run() break; std::string query_id; - readStringBinary(query_id, in); + UInt8 query_id_size = 0; + readBinary(query_id_size, in); + query_id.resize(query_id_size); + in.read(query_id.data(), query_id_size); UInt8 trace_size = 0; readIntBinary(trace_size, in); From 2983a9ba617bb524af2d48c2c33fe465b49e5aa5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Jun 2021 12:53:36 +0300 Subject: [PATCH 303/652] use cli for exec --- tests/integration/helpers/cluster.py | 65 ++++++++++++++++------------ 1 file changed, 37 insertions(+), 28 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c6fdbb49f5f..d4981b6f5de 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -51,14 +51,19 @@ def _create_env_file(path, variables): f.write("=".join([var, value]) + "\n") return path -def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE, timeout=180): +def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE, timeout=180, nothrow=False, detach=False): + if detach: + subprocess.Popen(args, stdout=subprocess.DEVNULL, stderr=subprocess.DEVNULL, env=env, shell=shell, timeout=timeout) + return + res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell, timeout=timeout) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually logging.debug(f"Stderr:\n{res.stderr.decode('utf-8')}\n") logging.debug(f"Stdout:\n{res.stdout.decode('utf-8')}\n") logging.debug(f"Env:\n{env}\n") - raise Exception(f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}") + if not nothrow: + raise Exception(f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}") # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 def get_free_port(): @@ -89,10 +94,10 @@ def retry_exception(num, delay, func, exception=Exception, *args, **kwargs): return raise StopIteration('Function did not finished successfully') -def subprocess_check_call(args): +def subprocess_check_call(args, detach=False, nothrow=False): # Uncomment for debugging - logging.info('run:' + ' '.join(args)) - run_and_check(args) + #logging.info('run:' + ' '.join(args)) + run_and_check(args, detach=detach, nothrow=nothrow) def subprocess_call(args): @@ -875,30 +880,34 @@ class ClickHouseCluster: container_id = self.get_container_id(instance_name) return self.docker_client.api.logs(container_id).decode() - def exec_in_container(self, container_id, cmd, detach=False, nothrow=False, **kwargs): - exec_id = self.docker_client.api.exec_create(container_id, cmd, **kwargs) - output = self.docker_client.api.exec_start(exec_id, detach=detach) + def exec_in_container(self, container_id, cmd, detach=False, nothrow=False, use_cli=True, **kwargs): + if use_cli: + result = subprocess_check_call(["docker", "exec", container_id] + cmd, detach=detach, nothrow=nothrow) + return result + else: + exec_id = self.docker_client.api.exec_create(container_id, cmd, **kwargs) + output = self.docker_client.api.exec_start(exec_id, detach=detach) - exit_code = self.docker_client.api.exec_inspect(exec_id)['ExitCode'] - if exit_code: - container_info = self.docker_client.api.inspect_container(container_id) - image_id = container_info.get('Image') - image_info = self.docker_client.api.inspect_image(image_id) - logging.debug(("Command failed in container {}: ".format(container_id))) - pprint.pprint(container_info) - logging.debug("") - logging.debug(("Container {} uses image {}: ".format(container_id, image_id))) - pprint.pprint(image_info) - logging.debug("") - message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, - exit_code, output) - if nothrow: - logging.debug(message) - else: - raise Exception(message) - if not detach: - return output.decode() - return output + exit_code = self.docker_client.api.exec_inspect(exec_id)['ExitCode'] + if exit_code: + container_info = self.docker_client.api.inspect_container(container_id) + image_id = container_info.get('Image') + image_info = self.docker_client.api.inspect_image(image_id) + logging.debug(("Command failed in container {}: ".format(container_id))) + pprint.pprint(container_info) + logging.debug("") + logging.debug(("Container {} uses image {}: ".format(container_id, image_id))) + pprint.pprint(image_info) + logging.debug("") + message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, + exit_code, output) + if nothrow: + logging.debug(message) + else: + raise Exception(message) + if not detach: + return output.decode() + return output def copy_file_to_container(self, container_id, local_path, dest_path): with open(local_path, "r") as fdata: From 94d2aed336ea551c2a8bd1752615bd75dee01529 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Jun 2021 13:24:06 +0300 Subject: [PATCH 304/652] fix tests --- src/Databases/DatabaseReplicated.cpp | 24 +++++++++++++++---- ...per_test_alter_compression_codecs_long.sql | 4 ++-- .../01710_projection_fetch.reference | 8 +++---- .../0_stateless/01710_projection_fetch.sql | 4 ++-- tests/queries/skip_list.json | 6 +++-- 5 files changed, 32 insertions(+), 14 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3482fad80b9..811114704c3 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -317,10 +318,25 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ if (!replicated_table || !create->storage->engine->arguments) return; - ASTs & args = create->storage->engine->arguments->children; + ASTs & args_ref = create->storage->engine->arguments->children; + ASTs args = args_ref; if (args.size() < 2) return; + /// It can be a constant expression. Try to evaluate it, ignore exception if we cannot. + bool has_expression_argument = args_ref[0]->as() || args_ref[0]->as(); + if (has_expression_argument) + { + try + { + args[0] = evaluateConstantExpressionAsLiteral(args_ref[0]->clone(), query_context); + args[1] = evaluateConstantExpressionAsLiteral(args_ref[1]->clone(), query_context); + } + catch (...) + { + } + } + ASTLiteral * arg1 = args[0]->as(); ASTLiteral * arg2 = args[1]->as(); if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String) @@ -348,12 +364,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ if (maybe_shard_macros && maybe_replica_macros) return; - if (enable_functional_tests_helper) + if (enable_functional_tests_helper && !has_expression_argument) { if (maybe_path.empty() || maybe_path.back() != '/') maybe_path += '/'; - arg1->value = maybe_path + "auto_{shard}"; - arg2->value = maybe_replica + "auto_{replica}"; + args_ref[0]->as()->value = maybe_path + "auto_{shard}"; + args_ref[1]->as()->value = maybe_replica + "auto_{replica}"; return; } diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql index 085a79485fb..548f26eadd0 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS alter_compression_codec2; CREATE TABLE alter_compression_codec1 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs', '1') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id; CREATE TABLE alter_compression_codec2 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs', '2') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id; INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2); diff --git a/tests/queries/0_stateless/01710_projection_fetch.reference b/tests/queries/0_stateless/01710_projection_fetch.reference index 54e5bff80a9..abce5410b26 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.reference +++ b/tests/queries/0_stateless/01710_projection_fetch.reference @@ -10,8 +10,8 @@ 3 3 4 4 0 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 2 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 -CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 +CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01710_projection_fetch.sql b/tests/queries/0_stateless/01710_projection_fetch.sql index 06790317808..7e4f6cc1d9a 100644 --- a/tests/queries/0_stateless/01710_projection_fetch.sql +++ b/tests/queries/0_stateless/01710_projection_fetch.sql @@ -1,9 +1,9 @@ drop table if exists tp_1; drop table if exists tp_2; -create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/01710_projection_fetch_' || currentDatabase(), '1') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; -create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/01710_projection_fetch_' || currentDatabase(), '2') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; +create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32; insert into tp_1 select number, number from numbers(3); diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 6a278316387..a9a41b1ac1f 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -109,6 +109,7 @@ "01153_attach_mv_uuid" ], "database-replicated": [ + /// Unclassified "memory_tracking", "memory_usage", "live_view", @@ -167,8 +168,9 @@ /// Does not support renaming of multiple tables in single query "00634_rename_view", "00140_rename", - "01783_http_chunk_size", - "01710_projection_fetch" + /// Requires investigation + "00953_zookeeper_suetin_deduplication_bug", + "01783_http_chunk_size" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed From 7eeea068452b540a952a9a27fe7909c16796c242 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Tue, 1 Jun 2021 20:32:40 +0900 Subject: [PATCH 305/652] sequenceNextNode() uses settings passed by creator. --- .../AggregateFunctionSequenceNextNode.cpp | 35 +++++++------------ .../AggregateFunctionSequenceNextNode.h | 1 + 2 files changed, 13 insertions(+), 23 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 2870f233b65..0c87a32a415 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -22,7 +23,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; - extern const int FUNCTION_NOT_ALLOWED; + extern const int UNKNOWN_AGGREGATE_FUNCTION; } namespace @@ -37,9 +38,14 @@ inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl( } AggregateFunctionPtr -createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, const DataTypes & argument_types, const Array & parameters) +createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) { - assert(max_events <= max_events_size); + if (settings == nullptr || !settings->allow_experimental_funnel_functions) + { + throw Exception( + "Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it", + ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); + } if (parameters.size() < 2) throw Exception("Aggregate function '" + name + "' requires 2 parameters (direction, head)", @@ -83,10 +89,10 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, "Aggregate function " + name + " requires at least " + toString(min_required_args + 1) + " arguments when base is first_match or last_match.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (argument_types.size() > max_events + min_required_args) + if (argument_types.size() > max_events_size + min_required_args) throw Exception(fmt::format( "Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.", - name, max_events + min_required_args, max_events), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + name, max_events_size + min_required_args, max_events_size), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg)) throw Exception("Illegal type " + cond_arg->getName() + " of third argument of aggregate function " @@ -127,29 +133,12 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } -auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) -{ - return [max_events](const std::string & name, const DataTypes & argument_types, const Array & parameters) - { - if (CurrentThread::isInitialized()) - { - const Context * query_context = CurrentThread::get().getQueryContext(); - if (query_context && !query_context->getSettingsRef().allow_experimental_funnel_functions) - throw Exception{"Cannot call 'sequenceNextNode' aggregate function because experimental_funnel_functions is not allowed. " - "Set 'allow_experimental_funnel_functions = 1' setting to enable", ErrorCodes::FUNCTION_NOT_ALLOWED}; - } - - return createAggregateFunctionSequenceNode(name, max_events, argument_types, parameters); - }; -} - } void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(max_events_size), properties }); - factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode, properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 629a130a0e3..116e53e95e8 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,6 +27,7 @@ namespace DB { +struct Settings; enum class SequenceDirection { From cdd46aa117642a0942cc0044a3dec2bef1812f53 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Jun 2021 14:52:25 +0300 Subject: [PATCH 306/652] Revert "try fix intersecting virtual parts" This reverts commit 2571ad7d438297d9aa799783c2301f520718572c. --- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 40 +-- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 14 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 273 +++--------------- .../MergeTree/ReplicatedMergeTreeQueue.h | 16 +- 4 files changed, 45 insertions(+), 298 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index ee802a2a5b1..dbd55bc4ff3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -152,10 +152,6 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << "sync_pinned_part_uuids\n"; break; - case PART_IS_LOST: - out << "lost\n" << new_part_name; - break; - default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown log entry type: {}", static_cast(type)); } @@ -330,11 +326,6 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) in >> new_part_name; in >> "\nsource_shard: " >> source_shard; } - else if (type_str == "lost") - { - type = PART_IS_LOST; - in >> new_part_name; - } if (!trailing_newline_found) in >> "\n"; @@ -398,6 +389,7 @@ void ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry::readText(ReadBuffer & i bool ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry::isMovePartitionOrAttachFrom(const MergeTreePartInfo & drop_range_info) { + assert(drop_range_info.getBlocksCount() != 0); return drop_range_info.getBlocksCount() == 1; } @@ -421,24 +413,6 @@ ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String return res; } -std::optional ReplicatedMergeTreeLogEntryData::getDropRange(MergeTreeDataFormatVersion format_version) const -{ - if (type == DROP_RANGE) - return new_part_name; - - if (type == REPLACE_RANGE) - { - auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version); - if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info)) - { - /// It's REPLACE, not MOVE or ATTACH, so drop range is real - return replace_range_entry->drop_range_part_name; - } - } - - return {}; -} - Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormatVersion format_version) const { /// Doesn't produce any part @@ -456,8 +430,12 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat if (type == REPLACE_RANGE) { Strings res = replace_range_entry->new_part_names; - if (auto drop_range = getDropRange(format_version)) - res.emplace_back(*drop_range); + auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version); + if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info)) + { + /// It's REPLACE, not MOVE or ATTACH, so drop range is real + res.emplace_back(replace_range_entry->drop_range_part_name); + } return res; } @@ -469,10 +447,6 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat if (type == CLONE_PART_FROM_SHARD) return {}; - /// Doesn't produce any part. - if (type == PART_IS_LOST) - return {}; - return {new_part_name}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 5d62f219924..12f1c78fb5d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -45,7 +45,6 @@ struct ReplicatedMergeTreeLogEntryData ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths SYNC_PINNED_PART_UUIDS, /// Synchronization point for ensuring that all replicas have up to date in-memory state. CLONE_PART_FROM_SHARD, /// Clone part from another shard. - PART_IS_LOST, /// Cancels previous operations with lost data part. Kinda "anti-merge". }; static String typeToString(Type type) @@ -63,7 +62,6 @@ struct ReplicatedMergeTreeLogEntryData case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; case ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS: return "SYNC_PINNED_PART_UUIDS"; case ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD: return "CLONE_PART_FROM_SHARD"; - case ReplicatedMergeTreeLogEntryData::PART_IS_LOST: return "PART_IS_LOST"; default: throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); } @@ -142,18 +140,6 @@ struct ReplicatedMergeTreeLogEntryData /// selection of merges. These parts are added to queue.virtual_parts. Strings getVirtualPartNames(MergeTreeDataFormatVersion format_version) const; - /// Returns name of part that will never appear and should be removed from virtual parts set. - /// It's required to correctly cancel merge which cannot be executed, because some source part is lost forever. - /// Do not use it for other purposes, it can be dangerous. - std::optional getAntiVirtualPartName() const - { - if (type == PART_IS_LOST) - return new_part_name; - return {}; - } - - std::optional getDropRange(MergeTreeDataFormatVersion format_version) const; - /// Returns set of parts that denote the block number ranges that should be blocked during the entry execution. /// These parts are added to future_parts. Strings getBlockingPartNames(MergeTreeDataFormatVersion format_version) const diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 84050f21528..30569e53f64 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -33,20 +33,14 @@ ReplicatedMergeTreeQueue::ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & log = &Poco::Logger::get(logger_name); } -void ReplicatedMergeTreeQueue::clear() -{ - auto locks = lockQueue(); - assert(future_parts.empty()); - current_parts.clear(); - virtual_parts.clear(); - queue.clear(); - inserts_by_time.clear(); - mutations_by_znode.clear(); - mutations_by_partition.clear(); - mutation_pointer.clear(); -} void ReplicatedMergeTreeQueue::initialize(const MergeTreeData::DataParts & parts) +{ + addVirtualParts(parts); +} + + +void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts & parts) { std::lock_guard lock(state_mutex); @@ -80,6 +74,9 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) /// Reset batch size on initialization to recover from possible errors of too large batch size. current_multi_batch_size = 1; + String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); + log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); + std::unordered_set already_loaded_paths; { std::lock_guard lock(state_mutex); @@ -135,45 +132,6 @@ void ReplicatedMergeTreeQueue::insertUnlocked( const LogEntryPtr & entry, std::optional & min_unprocessed_insert_time_changed, std::lock_guard & state_lock) { - if (auto lost_part = entry->getAntiVirtualPartName()) - { - LOG_TRACE(log, "Reading {}: PART_IS_LOST for {}", entry->znode_name, *lost_part); - QueueIters entries = findEntriesByNewPartName(*lost_part); - bool removed = virtual_parts.remove(*lost_part); - if (!removed && !entries.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Found {} entries for lost part {}, " - "but nothing is removed from virtual parts", entries.size(), *lost_part); - for (const auto & entry_it : entries) - { - for (const auto & source_part : (*entry_it)->source_parts) - { - String containing_current_part = current_parts.getContainingPart(source_part); - if (containing_current_part.empty()) - { - bool has_source_entry = std::any_of(queue.begin(), queue.end(), [&lost_part](auto e) - { - return e->new_part_name == *lost_part; - }); - if (!has_source_entry) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Source entry ({}) of lost virtual part {} does not exist", - source_part, *lost_part); - } - else - { - if (containing_current_part != source_part) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Source part {} of lost part {} is covered by current part {}. Entry: {}", - source_part, *lost_part, containing_current_part, (*entry_it)->toString()); - } - - virtual_parts.add(source_part); - } - } - - lost_forever_parts.insert(*lost_part); - } - for (const String & virtual_part_name : entry->getVirtualPartNames(format_version)) { virtual_parts.add(virtual_part_name); @@ -270,10 +228,16 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( removeCoveredPartsFromMutations(virtual_part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } - if (auto drop_range_part_name = entry->getDropRange(format_version)) + String drop_range_part_name; + if (entry->type == LogEntry::DROP_RANGE) + drop_range_part_name = entry->new_part_name; + else if (entry->type == LogEntry::REPLACE_RANGE) + drop_range_part_name = entry->replace_range_entry->drop_range_part_name; + + if (!drop_range_part_name.empty()) { - current_parts.remove(*drop_range_part_name); - virtual_parts.remove(*drop_range_part_name); + current_parts.remove(drop_range_part_name); + virtual_parts.remove(drop_range_part_name); } if (entry->type == LogEntry::ALTER_METADATA) @@ -337,7 +301,10 @@ void ReplicatedMergeTreeQueue::addPartToMutations(const String & part_name) { auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - if (part_info.isFakeDropRangePart()) + /// Do not add special virtual parts to parts_to_do + auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION + auto another_max_level = std::numeric_limits::max(); /// REPLACE/MOVE PARTITION + if (part_info.level == max_level || part_info.level == another_max_level) return; auto in_partition = mutations_by_partition.find(part_info.partition_id); @@ -425,8 +392,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep } if (!found && need_remove_from_zk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't find {} in the memory queue. It is a bug. Entry info: {}", - entry->znode_name, entry->toString()); + throw Exception("Can't find " + entry->znode_name + " in the memory queue. It is a bug", ErrorCodes::LOGICAL_ERROR); notifySubscribers(queue_size); @@ -495,162 +461,9 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri return true; } -bool ReplicatedMergeTreeQueue::markPartAsLostForever(zkutil::ZooKeeperPtr zookeeper, const String & part_name) + +bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & part_info) { - LogEntry entry; - entry.type = LogEntry::PART_IS_LOST; - entry.source_replica = storage.replica_name; - entry.new_part_name = part_name; - - size_t max_iters = 100; - while (--max_iters) - { - ReplicatedMergeTreeMergePredicate merge_pred = getMergePredicate(zookeeper); - - if (!merge_pred.ensurePartIsLost(part_name, log)) - return false; - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - Coordination::Responses responses; - Coordination::Error rc = zookeeper->tryMulti(ops, responses); - - if (rc == Coordination::Error::ZBADVERSION) - { - LOG_TRACE(log, "A new log entry appeared while trying to commit PART_IS_LOST. Retry."); - continue; - } else - zkutil::KeeperMultiException::check(rc, ops, responses); - - String path_created = dynamic_cast(*responses.back()).path_created; - LOG_TRACE(log, "Created PART_IS_LOST entry ({}) for part {}", path_created, part_name); - return true; - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create PART_IS_LOST entry for {}", part_name); -} - -ReplicatedMergeTreeQueue::QueueIters ReplicatedMergeTreeQueue::findEntriesByNewPartName(const String & part_name) const -{ - QueueIters entries; - size_t get_entries_num = 0; - for (auto it = queue.begin(); it != queue.end(); ++it) - { - if ((*it)->new_part_name == part_name && (*it)->type != LogEntry::PART_IS_LOST) - { - entries.push_back(it); - get_entries_num += (*it)->type == LogEntry::GET_PART; - } - } - - if (!entries.empty()) - { - if (entries.size() != 1) - { - /// Replication queue may contain multiple part producing entries with the same new_part_name - /// if replica was recently cloned. At first cloneReplica(...) copies source replica queue, - /// and after that it reads data parts set from source_path + "/parts" - /// to create GET_PART entries in own queue and download missing parts. - /// Therefore, some MERGE_PART (or MUTATE_PART, or even GET_PART) entry - /// may be duplicated with GET_PART with the same new_part_name. - if (1 < entries.size() - get_entries_num) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Found {} queue entries for {}, only {} are GET_PART", - entries.size(), part_name, get_entries_num); - - LOG_WARNING(log, "Found {} queue entries for {}, including {} GET_PART entries. " - "It may rarely happen after replica cloning", entries.size(), part_name, get_entries_num); - } - } - - return entries; -} - -void ReplicatedMergeTreeQueue::executePartIsLost(zkutil::ZooKeeperPtr zookeeper, LogEntry & entry_lost) -{ - LOG_TRACE(log, "Executing {}: PART_IS_LOST for part {} ({})", entry_lost.znode_name, entry_lost.new_part_name); - - /// There might be multiple cancelled parts if replica was cloned... - - size_t queue_size = 0; - std::optional min_unprocessed_insert_time_changed; - std::optional max_processed_insert_time_changed; - Coordination::Requests ops; - { - std::unique_lock lock(state_mutex); - QueueIters cancelled_entries = findEntriesByNewPartName(entry_lost.new_part_name); - - for (const auto entry_it : cancelled_entries) - { - LogEntryPtr canceled_entry; - canceled_entry = *entry_it; - LOG_TRACE(log, "Removing cancelled log entry {}: {}", canceled_entry->znode_name, canceled_entry->toString()); - ops.emplace_back(zkutil::makeRemoveRequest(replica_path + "/queue/" + canceled_entry->znode_name, -1)); - updateStateOnQueueEntryRemoval( - canceled_entry, /* is_successful = */ false, - min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); - queue.erase(entry_it); - queue_size = queue.size(); - - LOG_TRACE(log, "Waiting for {} to finish", canceled_entry->znode_name); - canceled_entry->execution_complete.wait(lock, [&canceled_entry] - { return !canceled_entry->currently_executing; }); - } - } - - notifySubscribers(queue_size); - - zookeeper->multi(ops); - - updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); - - { - std::unique_lock lock(state_mutex); - lost_forever_parts.erase(entry_lost.new_part_name); - } - - LOG_TRACE(log, "Executed {}: PART_IS_LOST for part {}", entry_lost.znode_name, entry_lost.new_part_name); -} - -bool ReplicatedMergeTreeMergePredicate::ensurePartIsLost(const String & part_name, Poco::Logger * log) const -{ - String containing_part; - { - std::lock_guard lock(queue.state_mutex); - containing_part = queue.virtual_parts.getContainingPart(part_name); - } - - if (containing_part.empty()) - { - LOG_WARNING(log, "Cannot find lost part {} in virtual parts set. " - "Probably it's already marked as lost by another replica.", part_name); - return false; - } - - if (containing_part != part_name) - { - auto info = MergeTreePartInfo::fromPartName(containing_part, queue.format_version); - if (info.isFakeDropRangePart()) - { - /// It does not matter anymore if part is lost, it should be removed anyway - LOG_INFO(log, "Lost part {} is covered by drop range {}, ignoring it", part_name, containing_part); - return false; - } - - /// Lost part is covered by real part. It means that merging/mutating operation - /// was assigned on lost part by some replica. It probably means that part is not lost. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Lost part {} is covered by real part {}", part_name, containing_part); - } - - /// We can mark part as lost - return true; -} - -bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & part_info) -{ - if (part_info.level != 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove part with non-zero level: {}", part_info.getPartName()); std::lock_guard lock(state_mutex); return virtual_parts.remove(part_info); } @@ -774,6 +587,8 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper { std::lock_guard state_lock(state_mutex); + log_pointer = last_entry_index + 1; + for (size_t copied_entry_idx = 0, num_copied_entries = copied_entries.size(); copied_entry_idx < num_copied_entries; ++copied_entry_idx) { String path_created = dynamic_cast(*responses[copied_entry_idx]).path_created; @@ -943,7 +758,9 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C /// Such parts do not exist and will never appear, so we should not add virtual parts to parts_to_do list. /// Fortunately, it's easy to distinguish virtual parts from normal parts by part level. /// See StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(...) - if (part_info.isFakeDropRangePart()) + auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION + auto another_max_level = std::numeric_limits::max(); /// REPLACE/MOVE PARTITION + if (part_info.level == max_level || part_info.level == another_max_level) continue; auto it = entry->block_numbers.find(part_info.partition_id); @@ -1124,6 +941,9 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( if ((*it)->currently_executing) to_wait.push_back(*it); auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); + /// FIXME it's probably unsafe to remove entries non-atomically + /// when this method called directly from alter query (not from replication queue task), + /// because entries will be lost if ALTER fails. if (code != Coordination::Error::ZOK) LOG_INFO(log, "Couldn't remove {}: {}", replica_path + "/queue/" + (*it)->znode_name, Coordination::errorMessage(code)); @@ -1218,26 +1038,11 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( || entry.type == LogEntry::ATTACH_PART || entry.type == LogEntry::MUTATE_PART) { - assert(entry.getBlockingPartNames(format_version).size() == 1); for (const String & new_part_name : entry.getBlockingPartNames(format_version)) { if (!isNotCoveredByFuturePartsImpl(entry.znode_name, new_part_name, out_postpone_reason, state_lock)) return false; } - - String covering_part = virtual_parts.getContainingPart(entry.new_part_name); - if (covering_part.empty()) - { - if (lost_forever_parts.count(entry.new_part_name)) - return false; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found in virtual parts, but it's not lost. " - "Entry {}: {}\nVirtual: {}\nLost:{}", - entry.new_part_name, entry.znode_name, entry.toString(), - fmt::join(virtual_parts.getParts(), ", "), fmt::join(lost_forever_parts, ", ")); - } - - /// NOTE: It's possible that (covering_part == entry.new_part_name), because isNotCoveredByFuturePartsImpl(...) - /// checks for future parts only, not for virtual parts. } /// Check that fetches pool is not overloaded @@ -1454,8 +1259,7 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(const Replicate for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version)) { if (!queue.future_parts.emplace(new_part_name, entry).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged future part {}. This is a bug. " - "It happened on attempt to execute {}: {}", new_part_name, entry->znode_name, entry->toString()); + throw Exception("Tagging already tagged future part " + new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); } } @@ -1473,8 +1277,7 @@ void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName(ReplicatedM return; if (!queue.future_parts.emplace(entry.actual_new_part_name, entry.shared_from_this()).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attaching already existing future part {}. This is a bug. " - "It happened on attempt to execute {}: {}", entry.actual_new_part_name, entry.znode_name, entry.toString()); + throw Exception("Attaching already existing future part " + entry.actual_new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); } @@ -1493,19 +1296,13 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version)) { if (!queue.future_parts.erase(new_part_name)) - { LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name); - assert(false); - } } if (!entry->actual_new_part_name.empty()) { if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name)) - { LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name); - assert(false); - } entry->actual_new_part_name.clear(); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index c1470e8ff01..496f277d132 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -92,8 +92,8 @@ private: using FuturePartsSet = std::map; FuturePartsSet future_parts; - using LostPartsSet = std::set; - LostPartsSet lost_forever_parts; + /// Index of the first log entry that we didn't see yet. + Int64 log_pointer = 0; /// Avoid parallel execution of queue enties, which may remove other entries from the queue. bool currently_executing_drop_or_replace_range = false; @@ -275,7 +275,6 @@ public: ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeMergeStrategyPicker & merge_strategy_picker_); ~ReplicatedMergeTreeQueue(); - void clear(); void initialize(const MergeTreeData::DataParts & parts); @@ -290,20 +289,13 @@ public: */ bool remove(zkutil::ZooKeeperPtr zookeeper, const String & part_name); - using QueueIters = std::vector; - QueueIters findEntriesByNewPartName(const String & part_name) const; - - bool markPartAsLostForever(zkutil::ZooKeeperPtr zookeeper, const String & part_name); - - void executePartIsLost(zkutil::ZooKeeperPtr zookeeper, LogEntry & entry_lost); - /** Load (initialize) a queue from ZooKeeper (/replicas/me/queue/). * If queue was not empty load() would not load duplicate records. * return true, if we update queue. */ bool load(zkutil::ZooKeeperPtr zookeeper); - bool removeFailedQuorumPart(const MergeTreePartInfo & part_info); + bool removeFromVirtualParts(const MergeTreePartInfo & part_info); /** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value. * If watch_callback is not empty, will call it when new entries appear in the log. @@ -486,8 +478,6 @@ public: /// The version of "log" node that is used to check that no new merges have appeared. int32_t getVersion() const { return merges_version; } - bool ensurePartIsLost(const String & part_name, Poco::Logger * log) const; - private: const ReplicatedMergeTreeQueue & queue; From e2941a8006bbcb0e34b9967aaae7e3da8ea821d3 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Jun 2021 15:15:13 +0300 Subject: [PATCH 307/652] Update DatabaseReplicated.cpp --- src/Databases/DatabaseReplicated.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 811114704c3..b3e5fc67151 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -324,7 +324,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ return; /// It can be a constant expression. Try to evaluate it, ignore exception if we cannot. - bool has_expression_argument = args_ref[0]->as() || args_ref[0]->as(); + bool has_expression_argument = args_ref[0]->as() || args_ref[1]->as(); if (has_expression_argument) { try From dbaa6ffc625287543fcd8bdef7249d4fe050db08 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Jun 2021 15:20:52 +0300 Subject: [PATCH 308/652] Rename ContextConstPtr to ContextPtr. --- .../AggregateFunctionFactory.cpp | 2 +- .../AggregateFunctionMLMethod.cpp | 6 +-- .../AggregateFunctionMLMethod.h | 10 ++--- src/AggregateFunctions/IAggregateFunction.h | 2 +- src/Bridge/IBridgeHelper.h | 4 +- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/Bridge/LibraryBridgeHelper.h | 2 +- src/Bridge/XDBCBridgeHelper.h | 2 +- src/Columns/ColumnAggregateFunction.cpp | 2 +- src/Columns/ColumnAggregateFunction.h | 2 +- src/DataTypes/DataTypeDecimalBase.cpp | 4 +- src/DataTypes/DataTypeDecimalBase.h | 4 +- .../CassandraDictionarySource.cpp | 2 +- .../ClickHouseDictionarySource.cpp | 6 +-- src/Dictionaries/ClickHouseDictionarySource.h | 2 +- src/Dictionaries/DictionaryFactory.cpp | 4 +- src/Dictionaries/DictionaryFactory.h | 6 +-- src/Dictionaries/DictionarySourceFactory.cpp | 2 +- src/Dictionaries/DictionarySourceFactory.h | 4 +- src/Dictionaries/DictionarySourceHelpers.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 2 +- src/Dictionaries/DirectDictionary.cpp | 2 +- .../ExecutableDictionarySource.cpp | 6 +-- src/Dictionaries/ExecutableDictionarySource.h | 4 +- .../ExecutablePoolDictionarySource.cpp | 4 +- .../ExecutablePoolDictionarySource.h | 4 +- src/Dictionaries/FileDictionarySource.cpp | 4 +- src/Dictionaries/FileDictionarySource.h | 4 +- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 4 +- src/Dictionaries/HTTPDictionarySource.h | 4 +- src/Dictionaries/HashedDictionary.cpp | 8 ++-- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 4 +- src/Dictionaries/LibraryDictionarySource.h | 4 +- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 2 +- .../PolygonDictionaryImplementations.cpp | 2 +- .../PostgreSQLDictionarySource.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.cpp | 2 +- src/Dictionaries/RedisDictionarySource.cpp | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 8 ++-- src/Dictionaries/XDBCDictionarySource.h | 2 +- .../getDictionaryConfigurationFromAST.cpp | 6 +-- .../getDictionaryConfigurationFromAST.h | 2 +- .../registerCacheDictionaries.cpp | 10 ++--- src/Disks/DiskDecorator.cpp | 2 +- src/Disks/DiskDecorator.h | 2 +- src/Disks/DiskFactory.cpp | 2 +- src/Disks/DiskFactory.h | 4 +- src/Disks/DiskLocal.cpp | 2 +- src/Disks/DiskMemory.cpp | 2 +- src/Disks/DiskSelector.cpp | 2 +- src/Disks/DiskSelector.h | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 2 +- src/Disks/IDisk.h | 2 +- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/S3/DiskS3.h | 4 +- src/Disks/S3/registerDiskS3.cpp | 6 +-- src/Formats/FormatFactory.cpp | 20 ++++----- src/Formats/FormatFactory.h | 16 +++---- src/Functions/FunctionBase64Conversion.h | 2 +- src/Functions/FunctionBinaryArithmetic.h | 20 ++++----- src/Functions/FunctionBitTestMany.h | 2 +- src/Functions/FunctionCustomWeekToSomething.h | 2 +- .../FunctionDateOrDateTimeAddInterval.h | 2 +- .../FunctionDateOrDateTimeToSomething.h | 2 +- src/Functions/FunctionFQDN.cpp | 2 +- src/Functions/FunctionFactory.cpp | 8 ++-- src/Functions/FunctionFactory.h | 12 +++--- src/Functions/FunctionFile.cpp | 6 +-- src/Functions/FunctionJoinGet.cpp | 2 +- src/Functions/FunctionJoinGet.h | 6 +-- src/Functions/FunctionMathBinaryFloat64.h | 2 +- src/Functions/FunctionMathConstFloat64.h | 2 +- src/Functions/FunctionMathUnary.h | 2 +- src/Functions/FunctionNumericPredicate.h | 2 +- src/Functions/FunctionStartsEndsWith.h | 4 +- src/Functions/FunctionStringOrArrayToT.h | 2 +- src/Functions/FunctionStringReplace.h | 2 +- src/Functions/FunctionStringToString.h | 2 +- src/Functions/FunctionUnaryArithmetic.h | 2 +- src/Functions/FunctionsAES.h | 4 +- src/Functions/FunctionsBitmap.h | 16 +++---- src/Functions/FunctionsCoding.h | 42 +++++++++---------- src/Functions/FunctionsComparison.h | 6 +-- src/Functions/FunctionsConsistentHashing.h | 2 +- src/Functions/FunctionsConversion.h | 6 +-- src/Functions/FunctionsEmbeddedDictionaries.h | 20 ++++----- src/Functions/FunctionsExternalDictionaries.h | 36 ++++++++-------- src/Functions/FunctionsExternalModels.cpp | 2 +- src/Functions/FunctionsExternalModels.h | 2 +- src/Functions/FunctionsHashing.h | 12 +++--- src/Functions/FunctionsJSON.h | 6 +-- src/Functions/FunctionsLogical.h | 4 +- .../FunctionsMultiStringFuzzySearch.h | 2 +- src/Functions/FunctionsMultiStringPosition.h | 2 +- src/Functions/FunctionsMultiStringSearch.h | 2 +- src/Functions/FunctionsRandom.h | 4 +- src/Functions/FunctionsRound.h | 4 +- src/Functions/FunctionsStringArray.h | 4 +- src/Functions/FunctionsStringHash.h | 2 +- src/Functions/FunctionsStringSearch.h | 2 +- src/Functions/FunctionsStringSearchToString.h | 2 +- src/Functions/FunctionsStringSimilarity.h | 2 +- src/Functions/LeastGreatestGeneric.h | 8 ++-- src/Functions/PerformanceAdaptors.h | 8 ++-- .../URL/FirstSignificantSubdomainCustomImpl.h | 2 +- src/Functions/URL/port.cpp | 2 +- src/Functions/abtesting.cpp | 2 +- src/Functions/addressToLine.cpp | 2 +- src/Functions/addressToSymbol.cpp | 2 +- src/Functions/appendTrailingCharIfAbsent.cpp | 2 +- src/Functions/array/FunctionArrayMapped.h | 2 +- src/Functions/array/array.cpp | 2 +- src/Functions/array/arrayConcat.cpp | 2 +- src/Functions/array/arrayDistinct.cpp | 2 +- src/Functions/array/arrayElement.cpp | 4 +- src/Functions/array/arrayEnumerate.cpp | 2 +- src/Functions/array/arrayEnumerateExtended.h | 2 +- src/Functions/array/arrayEnumerateRanked.h | 2 +- src/Functions/array/arrayFlatten.cpp | 2 +- src/Functions/array/arrayIndex.h | 2 +- src/Functions/array/arrayIntersect.cpp | 8 ++-- src/Functions/array/arrayJoin.cpp | 2 +- src/Functions/array/arrayPopBack.cpp | 2 +- src/Functions/array/arrayPopFront.cpp | 2 +- src/Functions/array/arrayPushBack.cpp | 2 +- src/Functions/array/arrayPushFront.cpp | 2 +- src/Functions/array/arrayReduce.cpp | 2 +- src/Functions/array/arrayReduceInRanges.cpp | 2 +- src/Functions/array/arrayResize.cpp | 2 +- src/Functions/array/arrayReverse.cpp | 2 +- src/Functions/array/arrayScalarProduct.h | 2 +- src/Functions/array/arraySlice.cpp | 2 +- src/Functions/array/arrayUniq.cpp | 2 +- src/Functions/array/arrayWithConstant.cpp | 2 +- src/Functions/array/arrayZip.cpp | 2 +- src/Functions/array/emptyArray.cpp | 2 +- src/Functions/array/emptyArrayToSingle.cpp | 2 +- src/Functions/array/hasAll.cpp | 2 +- src/Functions/array/hasAny.cpp | 2 +- src/Functions/array/hasSubstr.cpp | 2 +- src/Functions/array/mapOp.cpp | 2 +- src/Functions/array/mapPopulateSeries.cpp | 2 +- src/Functions/array/range.cpp | 2 +- src/Functions/assumeNotNull.cpp | 2 +- src/Functions/bar.cpp | 2 +- src/Functions/bitHammingDistance.cpp | 2 +- src/Functions/bitmaskToList.cpp | 2 +- src/Functions/blockNumber.cpp | 2 +- src/Functions/blockSerializedSize.cpp | 2 +- src/Functions/blockSize.cpp | 2 +- src/Functions/buildId.cpp | 2 +- src/Functions/byteSize.cpp | 2 +- src/Functions/caseWithExpression.cpp | 6 +-- src/Functions/coalesce.cpp | 6 +-- src/Functions/concat.cpp | 12 +++--- src/Functions/connectionId.cpp | 6 +-- src/Functions/convertCharset.cpp | 2 +- src/Functions/countDigits.cpp | 2 +- src/Functions/countMatches.h | 2 +- src/Functions/currentDatabase.cpp | 2 +- src/Functions/currentUser.cpp | 2 +- src/Functions/dateDiff.cpp | 2 +- src/Functions/date_trunc.cpp | 6 +-- src/Functions/defaultValueOfArgumentType.cpp | 2 +- src/Functions/defaultValueOfTypeName.cpp | 2 +- src/Functions/demange.cpp | 2 +- src/Functions/dumpColumnStructure.cpp | 2 +- src/Functions/errorCodeToName.cpp | 2 +- src/Functions/evalMLMethod.cpp | 6 +-- src/Functions/extractAllGroups.h | 2 +- src/Functions/extractGroups.cpp | 2 +- src/Functions/extractTextFromHTML.cpp | 2 +- src/Functions/filesystem.cpp | 2 +- src/Functions/finalizeAggregation.cpp | 2 +- src/Functions/formatDateTime.cpp | 2 +- src/Functions/formatReadable.h | 2 +- src/Functions/formatReadableTimeDelta.cpp | 2 +- src/Functions/formatRow.cpp | 10 ++--- src/Functions/formatString.cpp | 2 +- src/Functions/fromModifiedJulianDay.cpp | 2 +- src/Functions/fromUnixTimestamp64Micro.cpp | 2 +- src/Functions/fromUnixTimestamp64Milli.cpp | 2 +- src/Functions/fromUnixTimestamp64Nano.cpp | 2 +- src/Functions/fuzzBits.cpp | 2 +- src/Functions/generateUUIDv4.cpp | 4 +- src/Functions/geoToH3.cpp | 2 +- src/Functions/geohashDecode.cpp | 2 +- src/Functions/geohashEncode.cpp | 2 +- src/Functions/geohashesInBox.cpp | 2 +- src/Functions/getMacro.cpp | 2 +- src/Functions/getScalar.cpp | 8 ++-- src/Functions/getSetting.cpp | 6 +-- src/Functions/getSizeOfEnumType.cpp | 2 +- src/Functions/globalVariable.cpp | 2 +- src/Functions/greatCircleDistance.cpp | 4 +- src/Functions/h3EdgeAngle.cpp | 2 +- src/Functions/h3EdgeLengthM.cpp | 2 +- src/Functions/h3GetBaseCell.cpp | 2 +- src/Functions/h3GetResolution.cpp | 2 +- src/Functions/h3HexAreaM2.cpp | 2 +- src/Functions/h3IndexesAreNeighbors.cpp | 2 +- src/Functions/h3IsValid.cpp | 2 +- src/Functions/h3ToChildren.cpp | 2 +- src/Functions/h3ToParent.cpp | 2 +- src/Functions/h3ToString.cpp | 2 +- src/Functions/h3kRing.cpp | 2 +- src/Functions/hasColumnInTable.cpp | 6 +-- src/Functions/hasThreadFuzzer.cpp | 2 +- src/Functions/hostName.cpp | 2 +- src/Functions/identity.cpp | 2 +- src/Functions/if.cpp | 2 +- src/Functions/ifNotFinite.cpp | 6 +-- src/Functions/ifNull.cpp | 6 +-- src/Functions/ignore.cpp | 2 +- src/Functions/in.cpp | 2 +- src/Functions/indexHint.cpp | 2 +- src/Functions/initializeAggregation.cpp | 2 +- src/Functions/isConstant.cpp | 2 +- src/Functions/isDecimalOverflow.cpp | 2 +- src/Functions/isIPAddressContainedIn.cpp | 2 +- src/Functions/isNotNull.cpp | 2 +- src/Functions/isNull.cpp | 2 +- src/Functions/isZeroOrNull.cpp | 2 +- src/Functions/logTrace.cpp | 2 +- src/Functions/lowCardinalityIndices.cpp | 2 +- src/Functions/lowCardinalityKeys.cpp | 2 +- src/Functions/map.cpp | 8 ++-- src/Functions/materialize.h | 2 +- src/Functions/multiIf.cpp | 2 +- src/Functions/neighbor.cpp | 2 +- src/Functions/normalizedQueryHash.cpp | 2 +- src/Functions/now.cpp | 2 +- src/Functions/now64.cpp | 2 +- src/Functions/nullIf.cpp | 6 +-- src/Functions/partitionId.cpp | 2 +- src/Functions/pointInEllipses.cpp | 2 +- src/Functions/pointInPolygon.cpp | 2 +- src/Functions/polygonArea.cpp | 2 +- src/Functions/polygonConvexHull.cpp | 2 +- src/Functions/polygonPerimeter.cpp | 2 +- src/Functions/polygonsDistance.cpp | 2 +- src/Functions/polygonsEquals.cpp | 2 +- src/Functions/polygonsIntersection.cpp | 2 +- src/Functions/polygonsSymDifference.cpp | 2 +- src/Functions/polygonsUnion.cpp | 2 +- src/Functions/polygonsWithin.cpp | 2 +- src/Functions/randConstant.cpp | 2 +- src/Functions/randomFixedString.cpp | 4 +- src/Functions/randomPrintableASCII.cpp | 2 +- src/Functions/randomString.cpp | 4 +- src/Functions/randomStringUTF8.cpp | 2 +- src/Functions/readWkt.cpp | 2 +- src/Functions/regexpQuoteMeta.cpp | 2 +- src/Functions/reinterpretAs.cpp | 4 +- src/Functions/repeat.cpp | 2 +- src/Functions/replicate.h | 2 +- src/Functions/reverse.cpp | 8 ++-- src/Functions/rowNumberInAllBlocks.cpp | 2 +- src/Functions/rowNumberInBlock.cpp | 2 +- src/Functions/runningAccumulate.cpp | 2 +- src/Functions/runningConcurrency.cpp | 2 +- src/Functions/runningDifference.h | 2 +- src/Functions/sleep.h | 2 +- src/Functions/stringToH3.cpp | 2 +- src/Functions/substring.cpp | 2 +- src/Functions/svg.cpp | 2 +- src/Functions/tcpPort.cpp | 2 +- src/Functions/throwIf.cpp | 2 +- src/Functions/tid.cpp | 2 +- src/Functions/timeSlots.cpp | 2 +- src/Functions/timezone.cpp | 2 +- src/Functions/timezoneOf.cpp | 2 +- src/Functions/toColumnTypeName.cpp | 2 +- src/Functions/toFixedString.h | 2 +- src/Functions/toLowCardinality.cpp | 2 +- src/Functions/toModifiedJulianDay.cpp | 2 +- src/Functions/toNullable.cpp | 2 +- src/Functions/toStartOfInterval.cpp | 2 +- src/Functions/toTimezone.cpp | 2 +- src/Functions/toTypeName.cpp | 2 +- src/Functions/toUnixTimestamp64Micro.cpp | 2 +- src/Functions/toUnixTimestamp64Milli.cpp | 2 +- src/Functions/toUnixTimestamp64Nano.cpp | 2 +- src/Functions/today.cpp | 2 +- src/Functions/transform.cpp | 2 +- src/Functions/trap.cpp | 6 +-- src/Functions/tuple.cpp | 2 +- src/Functions/tupleElement.cpp | 2 +- src/Functions/tupleHammingDistance.cpp | 6 +-- src/Functions/uptime.cpp | 2 +- src/Functions/validateNestedArraySizes.cpp | 2 +- src/Functions/version.cpp | 2 +- src/Functions/visibleWidth.cpp | 2 +- src/Functions/wkt.cpp | 2 +- src/Functions/yesterday.cpp | 2 +- src/IO/ConnectionTimeouts.h | 2 +- src/IO/ConnectionTimeoutsContext.h | 2 +- src/Interpreters/ColumnAliasesVisitor.h | 4 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 2 +- src/Interpreters/Context_fwd.h | 3 -- src/Interpreters/DuplicateOrderByVisitor.h | 4 +- .../ExecuteScalarSubqueriesVisitor.h | 2 +- .../ExternalDictionariesLoader.cpp | 12 +++--- src/Interpreters/ExternalDictionariesLoader.h | 12 +++--- .../ExtractExpressionInfoVisitor.cpp | 4 +- .../ExtractExpressionInfoVisitor.h | 4 +- src/Interpreters/MonotonicityCheckVisitor.h | 2 +- .../PredicateExpressionsOptimizer.cpp | 8 ++-- .../PredicateExpressionsOptimizer.h | 4 +- src/Interpreters/PredicateRewriteVisitor.cpp | 4 +- src/Interpreters/PredicateRewriteVisitor.h | 4 +- .../RedundantFunctionsInOrderByVisitor.h | 2 +- .../RemoveInjectiveFunctionsVisitor.cpp | 2 +- .../RemoveInjectiveFunctionsVisitor.h | 4 +- src/Interpreters/TreeOptimizer.cpp | 12 +++--- src/Interpreters/TreeOptimizer.h | 2 +- src/Interpreters/TreeRewriter.cpp | 4 +- src/Interpreters/TreeRewriter.h | 4 +- src/Interpreters/addTypeConversionToAST.cpp | 2 +- src/Interpreters/addTypeConversionToAST.h | 2 +- .../replaceAliasColumnsInQuery.cpp | 2 +- src/Interpreters/replaceAliasColumnsInQuery.h | 2 +- .../Formats/Impl/MySQLOutputFormat.h | 4 +- .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- 328 files changed, 563 insertions(+), 566 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index bdc057817c9..d0e45053695 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -133,7 +133,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( is_case_insensitive = true; } - ContextConstPtr query_context; + ContextPtr query_context; if (CurrentThread::isInitialized()) query_context = CurrentThread::get().getQueryContext(); diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index e7f79c276ed..0b6764960fb 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -148,7 +148,7 @@ void LinearModelData::predict( const ColumnsWithTypeAndName & arguments, size_t offset, size_t limit, - ContextConstPtr context) const + ContextPtr context) const { gradient_computer->predict(container, arguments, offset, limit, weights, bias, context); } @@ -455,7 +455,7 @@ void LogisticRegression::predict( size_t limit, const std::vector & weights, Float64 bias, - ContextConstPtr /*context*/) const + ContextPtr /*context*/) const { size_t rows_num = arguments.front().column->size(); @@ -523,7 +523,7 @@ void LinearRegression::predict( size_t limit, const std::vector & weights, Float64 bias, - ContextConstPtr /*context*/) const + ContextPtr /*context*/) const { if (weights.size() + 1 != arguments.size()) { diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.h b/src/AggregateFunctions/AggregateFunctionMLMethod.h index a97b565eed3..91302269759 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.h +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.h @@ -46,7 +46,7 @@ public: size_t limit, const std::vector & weights, Float64 bias, - ContextConstPtr context) const = 0; + ContextPtr context) const = 0; }; @@ -71,7 +71,7 @@ public: size_t limit, const std::vector & weights, Float64 bias, - ContextConstPtr context) const override; + ContextPtr context) const override; }; @@ -96,7 +96,7 @@ public: size_t limit, const std::vector & weights, Float64 bias, - ContextConstPtr context) const override; + ContextPtr context) const override; }; @@ -266,7 +266,7 @@ public: const ColumnsWithTypeAndName & arguments, size_t offset, size_t limit, - ContextConstPtr context) const; + ContextPtr context) const; void returnWeights(IColumn & to) const; private: @@ -367,7 +367,7 @@ public: const ColumnsWithTypeAndName & arguments, size_t offset, size_t limit, - ContextConstPtr context) const override + ContextPtr context) const override { if (arguments.size() != param_num + 1) throw Exception( diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 2d47a1e7fe0..a05f71a61c8 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -123,7 +123,7 @@ public: const ColumnsWithTypeAndName & /*arguments*/, size_t /*offset*/, size_t /*limit*/, - ContextConstPtr /*context*/) const + ContextPtr /*context*/) const { throw Exception("Method predictValues is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Bridge/IBridgeHelper.h b/src/Bridge/IBridgeHelper.h index 34be69a7db7..caaf031b7d8 100644 --- a/src/Bridge/IBridgeHelper.h +++ b/src/Bridge/IBridgeHelper.h @@ -14,7 +14,7 @@ namespace DB /// Common base class for XDBC and Library bridge helpers. /// Contains helper methods to check/start bridge sync. -class IBridgeHelper: protected WithConstContext +class IBridgeHelper: protected WithContext { public: @@ -27,7 +27,7 @@ public: static const inline std::string PING_METHOD = Poco::Net::HTTPRequest::HTTP_GET; static const inline std::string MAIN_METHOD = Poco::Net::HTTPRequest::HTTP_POST; - explicit IBridgeHelper(ContextConstPtr context_) : WithConstContext(context_) {} + explicit IBridgeHelper(ContextPtr context_) : WithContext(context_) {} virtual ~IBridgeHelper() = default; void startBridgeSync() const; diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 8967b191b73..7d378941d25 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -20,7 +20,7 @@ namespace DB { LibraryBridgeHelper::LibraryBridgeHelper( - ContextConstPtr context_, + ContextPtr context_, const Block & sample_block_, const Field & dictionary_id_) : IBridgeHelper(context_->getGlobalContext()) diff --git a/src/Bridge/LibraryBridgeHelper.h b/src/Bridge/LibraryBridgeHelper.h index c3c42e85f93..12fe0c33363 100644 --- a/src/Bridge/LibraryBridgeHelper.h +++ b/src/Bridge/LibraryBridgeHelper.h @@ -17,7 +17,7 @@ class LibraryBridgeHelper : public IBridgeHelper public: static constexpr inline size_t DEFAULT_PORT = 9012; - LibraryBridgeHelper(ContextConstPtr context_, const Block & sample_block, const Field & dictionary_id_); + LibraryBridgeHelper(ContextPtr context_, const Block & sample_block, const Field & dictionary_id_); bool initLibrary(const std::string & library_path, std::string library_settings, std::string attributes_names); diff --git a/src/Bridge/XDBCBridgeHelper.h b/src/Bridge/XDBCBridgeHelper.h index 97a6251a9a0..b170a55e438 100644 --- a/src/Bridge/XDBCBridgeHelper.h +++ b/src/Bridge/XDBCBridgeHelper.h @@ -60,7 +60,7 @@ public: static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed"; XDBCBridgeHelper( - ContextConstPtr context_, + ContextPtr context_, Poco::Timespan http_timeout_, const std::string & connection_string_) : IXDBCBridgeHelper(context_->getGlobalContext()) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index a5bfcae05e4..8178802f3bd 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -162,7 +162,7 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr colum return res; } -MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, ContextConstPtr context) const +MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const { MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn(); res->reserve(data.size()); diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index e75af280ed5..8eb1a04b174 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -119,7 +119,7 @@ public: const char * getFamilyName() const override { return "AggregateFunction"; } TypeIndex getDataType() const override { return TypeIndex::AggregateFunction; } - MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, ContextConstPtr context) const; + MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const; size_t size() const override { diff --git a/src/DataTypes/DataTypeDecimalBase.cpp b/src/DataTypes/DataTypeDecimalBase.cpp index 830b22b5943..683710b8880 100644 --- a/src/DataTypes/DataTypeDecimalBase.cpp +++ b/src/DataTypes/DataTypeDecimalBase.cpp @@ -19,11 +19,11 @@ namespace ErrorCodes { } -bool decimalCheckComparisonOverflow(ContextConstPtr context) +bool decimalCheckComparisonOverflow(ContextPtr context) { return context->getSettingsRef().decimal_check_overflow; } -bool decimalCheckArithmeticOverflow(ContextConstPtr context) +bool decimalCheckArithmeticOverflow(ContextPtr context) { return context->getSettingsRef().decimal_check_overflow; } diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index 17b73a8e3d6..aea837f890f 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -18,8 +18,8 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } -bool decimalCheckComparisonOverflow(ContextConstPtr context); -bool decimalCheckArithmeticOverflow(ContextConstPtr context); +bool decimalCheckComparisonOverflow(ContextPtr context); +bool decimalCheckArithmeticOverflow(ContextPtr context); inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type) { diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index b91ce6c82c9..7605b86ef90 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - ContextConstPtr /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /*created_from_ddl*/) -> DictionarySourcePtr { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 9e44a7b442d..776e74672b4 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -27,7 +27,7 @@ namespace { constexpr size_t MAX_CONNECTIONS = 16; - inline UInt16 getPortFromContext(ContextConstPtr context, bool secure) + inline UInt16 getPortFromContext(ContextPtr context, bool secure) { return secure ? context->getTCPPortSecure().value_or(0) : context->getTCPPort(); } @@ -60,7 +60,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - ContextConstPtr context_) + ContextPtr context_) : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , configuration{configuration_} @@ -209,7 +209,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & default_database [[maybe_unused]], bool /* created_from_ddl */) -> DictionarySourcePtr { diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index cf954a9620d..69931ef41f4 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -37,7 +37,7 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - ContextConstPtr context); + ContextPtr context); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 1f23200a745..62b28ed7d14 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -31,7 +31,7 @@ DictionaryPtr DictionaryFactory::create( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextConstPtr context, + ContextPtr context, bool created_from_ddl) const { Poco::Util::AbstractConfiguration::Keys keys; @@ -68,7 +68,7 @@ DictionaryPtr DictionaryFactory::create( layout_type); } -DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextConstPtr context) const +DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const { auto configuration = getDictionaryConfigurationFromAST(ast, context); return DictionaryFactory::create(name, *configuration, "dictionary", context, true); diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index a261c377cad..b717009aa8a 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -36,13 +36,13 @@ public: const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextConstPtr context, + ContextPtr context, bool created_from_ddl) const; /// Create dictionary from DDL-query DictionaryPtr create(const std::string & name, const ASTCreateQuery & ast, - ContextConstPtr context) const; + ContextPtr context) const; using LayoutCreateFunction = std::function; bool isComplex(const std::string & layout_type) const; diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index 507d11eba8e..50ba6405074 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -79,7 +79,7 @@ DictionarySourcePtr DictionarySourceFactory::create( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - ContextConstPtr context, + ContextPtr context, const std::string & default_database, bool check_config) const { diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index 9a0235a5eb0..bb583927ac4 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -35,7 +35,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & default_database, bool check_config)>; @@ -48,7 +48,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - ContextConstPtr context, + ContextPtr context, const std::string & default_database, bool check_config) const; diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 723a0c338fd..2d53ac4321e 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -69,7 +69,7 @@ Block blockForKeys( ContextMutablePtr copyContextAndApplySettings( const std::string & config_prefix, - ContextConstPtr context, + ContextPtr context, const Poco::Util::AbstractConfiguration & config) { auto local_context = Context::createCopy(context); diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 1066574f8e9..6c9a321aa36 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -38,7 +38,7 @@ Block blockForKeys( /// Used for applying settings to copied context in some register[...]Source functions ContextMutablePtr copyContextAndApplySettings( const std::string & config_prefix, - ContextConstPtr context, + ContextPtr context, const Poco::Util::AbstractConfiguration & config); /** A stream, adds additional columns to each block that it will read from inner stream. diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 2cc1d6caf5f..0508a0d70ad 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -306,7 +306,7 @@ namespace const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr /* context */, + ContextPtr /* context */, bool /* created_from_ddl */) { const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct"; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 2fb1bcf89ac..55d06984df4 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -63,7 +63,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - ContextConstPtr context_) + ContextPtr context_) : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct{dict_struct_} , implicit_key{config.getBool(config_prefix + ".implicit_key", false)} @@ -140,7 +140,7 @@ namespace { public: BlockInputStreamWithBackgroundThread( - ContextConstPtr context, + ContextPtr context, const std::string & format, const Block & sample_block, const std::string & command_str, @@ -265,7 +265,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 4b2326b2a58..878cb086873 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -20,7 +20,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - ContextConstPtr context_); + ContextPtr context_); ExecutableDictionarySource(const ExecutableDictionarySource & other); ExecutableDictionarySource & operator=(const ExecutableDictionarySource &) = delete; @@ -58,7 +58,7 @@ private: const std::string update_field; const std::string format; Block sample_block; - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 21818baa658..2e01c75234c 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -32,7 +32,7 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - ContextConstPtr context_) + ContextPtr context_) : log(&Poco::Logger::get("ExecutablePoolDictionarySource")) , dict_struct{dict_struct_} , configuration{configuration_} @@ -273,7 +273,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 9bf6e89c1f0..7a0b8681a21 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -42,7 +42,7 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - ContextConstPtr context_); + ContextPtr context_); ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other); ExecutablePoolDictionarySource & operator=(const ExecutablePoolDictionarySource &) = delete; @@ -78,7 +78,7 @@ private: const Configuration configuration; Block sample_block; - ContextConstPtr context; + ContextPtr context; std::shared_ptr process_pool; }; diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 49e87e24543..239c13e71c2 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes FileDictionarySource::FileDictionarySource( const std::string & filepath_, const std::string & format_, - Block & sample_block_, ContextConstPtr context_, bool created_from_ddl) + Block & sample_block_, ContextPtr context_, bool created_from_ddl) : filepath{filepath_} , format{format_} , sample_block{sample_block_} @@ -74,7 +74,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index d31a6dedb7f..ffc29374f4f 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -17,7 +17,7 @@ class FileDictionarySource final : public IDictionarySource { public: FileDictionarySource(const std::string & filepath_, const std::string & format_, - Block & sample_block_, ContextConstPtr context_, bool created_from_ddl); + Block & sample_block_, ContextPtr context_, bool created_from_ddl); FileDictionarySource(const FileDictionarySource & other); @@ -61,7 +61,7 @@ private: const std::string filepath; const std::string format; Block sample_block; - ContextConstPtr context; + ContextPtr context; Poco::Timestamp last_modification; }; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index f58384a3b8e..1c7fd076e6e 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -506,7 +506,7 @@ void registerDictionaryFlat(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr /* context */, + ContextPtr /* context */, bool /* created_from_ddl */) -> DictionaryPtr { if (dict_struct.key) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index b3527b06117..7fc1778545c 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -31,7 +31,7 @@ HTTPDictionarySource::HTTPDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - ContextConstPtr context_, + ContextPtr context_, bool created_from_ddl) : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} @@ -231,7 +231,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { if (dict_struct.has_expressions) diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 9b93856a6bc..e348075edc3 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -27,7 +27,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - ContextConstPtr context_, + ContextPtr context_, bool created_from_ddl); HTTPDictionarySource(const HTTPDictionarySource & other); @@ -69,7 +69,7 @@ private: std::string update_field; const std::string format; Block sample_block; - ContextConstPtr context; + ContextPtr context; ConnectionTimeouts timeouts; }; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 0de28f18006..24450b87274 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -751,13 +751,13 @@ void registerDictionaryHashed(DictionaryFactory & factory) using namespace std::placeholders; factory.registerLayout("hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false); factory.registerLayout("sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); factory.registerLayout("complex_key_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true); factory.registerLayout("complex_key_sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextConstPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); } diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index a0630ab324c..648e756e9a7 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -935,7 +935,7 @@ void registerDictionaryTrie(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr /* context */, + ContextPtr /* context */, bool /*created_from_ddl*/) -> DictionaryPtr { if (!dict_struct.key || dict_struct.key->size() != 1) diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index f52d774f279..0b8b52a2d67 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -31,7 +31,7 @@ LibraryDictionarySource::LibraryDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix_, Block & sample_block_, - ContextConstPtr context_, + ContextPtr context_, bool created_from_ddl) : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{dict_struct_} @@ -172,7 +172,7 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index eb346d09fab..88e133666e6 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -39,7 +39,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix_, Block & sample_block_, - ContextConstPtr context_, + ContextPtr context_, bool created_from_ddl); LibraryDictionarySource(const LibraryDictionarySource & other); @@ -86,7 +86,7 @@ private: const Field dictionary_id; Block sample_block; - ContextConstPtr context; + ContextPtr context; LibraryBridgeHelperPtr bridge_helper; ExternalResultDescription description; diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index c0780616191..2a809f52817 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -13,7 +13,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & root_config_prefix, Block & sample_block, - ContextConstPtr, + ContextPtr, const std::string & /* default_database */, bool /* created_from_ddl */) { diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 40398a7ba42..afda8241b38 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -24,7 +24,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - [[maybe_unused]] ContextConstPtr context, + [[maybe_unused]] ContextPtr context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { #if USE_MYSQL diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index fd3b9efd532..7c3eb421a4a 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -167,7 +167,7 @@ DictionaryPtr createLayout(const std::string & , const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr /* context */, + ContextPtr /* context */, bool /*created_from_ddl*/) { const String database = config.getString(config_prefix + ".database", ""); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index c563f9aab45..d23c9a7e725 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -188,7 +188,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & root_config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index b771bca068f..4049bd3a027 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -637,7 +637,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr /* context */, + ContextPtr /* context */, bool /*created_from_ddl*/) -> DictionaryPtr { if (dict_struct.key) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index ce979462a36..4e79a75c1d4 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -12,7 +12,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const String & config_prefix, Block & sample_block, - ContextConstPtr /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { return std::make_unique(dict_struct, config, config_prefix + ".redis", sample_block); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index fa7138348c1..0ebd15ff515 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -37,7 +37,7 @@ namespace const Poco::URI & uri, std::function callback, const Block & sample_block, - ContextConstPtr context, + ContextPtr context, UInt64 max_block_size, const ConnectionTimeouts & timeouts, const String name_) @@ -102,7 +102,7 @@ XDBCDictionarySource::XDBCDictionarySource( const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix_, const Block & sample_block_, - ContextConstPtr context_, + ContextPtr context_, const BridgeHelperPtr bridge_) : WithContext(context_->getGlobalContext()) , log(&Poco::Logger::get(bridge_->getName() + "DictionarySource")) @@ -278,7 +278,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextConstPtr context, + ContextPtr context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_ODBC @@ -305,7 +305,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, Block & /* sample_block */, - ContextConstPtr /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 4a6f226102b..bd473e0db8a 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -31,7 +31,7 @@ public: const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix_, const Block & sample_block_, - ContextConstPtr context_, + ContextPtr context_, BridgeHelperPtr bridge); /// copy-constructor is provided in order to support cloneability diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 4816c6fd6b8..2063ebcbf79 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -397,7 +397,7 @@ void buildConfigurationFromFunctionWithKeyValueArguments( AutoPtr doc, AutoPtr root, const ASTExpressionList * ast_expr_list, - ContextConstPtr context) + ContextPtr context) { const auto & children = ast_expr_list->children; for (size_t i = 0; i != children.size(); ++i) @@ -464,7 +464,7 @@ void buildSourceConfiguration( AutoPtr root, const ASTFunctionWithKeyValueArguments * source, const ASTDictionarySettings * settings, - ContextConstPtr context) + ContextPtr context) { AutoPtr outer_element(doc->createElement("source")); root->appendChild(outer_element); @@ -525,7 +525,7 @@ void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs DictionaryConfigurationPtr -getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextConstPtr context, const std::string & database_) +getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr context, const std::string & database_) { checkAST(query); diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.h b/src/Dictionaries/getDictionaryConfigurationFromAST.h index 01f13ab4030..de8659e4d7b 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.h +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.h @@ -12,5 +12,5 @@ using DictionaryConfigurationPtr = Poco::AutoPtr DictionaryPtr { return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); @@ -278,7 +278,7 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr /* context */, + ContextPtr /* context */, bool /* created_from_ddl */) -> DictionaryPtr { return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); @@ -293,7 +293,7 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr context, + ContextPtr context, bool created_from_ddl) -> DictionaryPtr { return createSSDCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl); @@ -306,7 +306,7 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextConstPtr context, + ContextPtr context, bool created_from_ddl) -> DictionaryPtr { return createSSDCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl); }; diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index b524cd88f10..d1ff3f9f827 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -206,7 +206,7 @@ void DiskDecorator::startup() delegate->startup(); } -void DiskDecorator::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextConstPtr context) +void DiskDecorator::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context) { delegate->applyNewSettings(config, context); } diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index a413c1a3cfd..401078e6b2e 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -69,7 +69,7 @@ public: SyncGuardPtr getDirectorySyncGuard(const String & path) const override; void shutdown() override; void startup() override; - void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextConstPtr context) override; + void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context) override; protected: Executor & getExecutor() override; diff --git a/src/Disks/DiskFactory.cpp b/src/Disks/DiskFactory.cpp index 03412be9729..b0fb0bd7ca7 100644 --- a/src/Disks/DiskFactory.cpp +++ b/src/Disks/DiskFactory.cpp @@ -24,7 +24,7 @@ DiskPtr DiskFactory::create( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextConstPtr context) const + ContextPtr context) const { const auto disk_type = config.getString(config_prefix + ".type", "local"); diff --git a/src/Disks/DiskFactory.h b/src/Disks/DiskFactory.h index ba833d9e25f..1c05c8d0335 100644 --- a/src/Disks/DiskFactory.h +++ b/src/Disks/DiskFactory.h @@ -24,7 +24,7 @@ public: const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextConstPtr context)>; + ContextPtr context)>; static DiskFactory & instance(); @@ -34,7 +34,7 @@ public: const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextConstPtr context) const; + ContextPtr context) const; private: using DiskTypeRegistry = std::unordered_map; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 768d1c2354a..89c1514f5c8 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -367,7 +367,7 @@ void registerDiskLocal(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextConstPtr context) -> DiskPtr { + ContextPtr context) -> DiskPtr { String path = config.getString(config_prefix + ".path", ""); if (name == "default") { diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 423dba6bed6..77926b4e375 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -450,7 +450,7 @@ void registerDiskMemory(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & /*config*/, const String & /*config_prefix*/, - ContextConstPtr /*context*/) -> DiskPtr { return std::make_shared(name); }; + ContextPtr /*context*/) -> DiskPtr { return std::make_shared(name); }; factory.registerDiskType("memory", creator); } diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 96b906f3ffc..0d36cadc349 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int UNKNOWN_DISK; } -DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextConstPtr context) +DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 6b7e8174d3d..4652cc40ea3 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -19,7 +19,7 @@ using DisksMap = std::map; class DiskSelector { public: - DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextConstPtr context); + DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); DiskSelector(const DiskSelector & from) : disks(from.disks) { } DiskSelectorPtr updateFromConfig( diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index da6ccb024c7..dafd507ba1e 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -170,7 +170,7 @@ void registerDiskHDFS(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextConstPtr context_) -> DiskPtr + ContextPtr context_) -> DiskPtr { fs::path disk = fs::path(context_->getPath()) / "disks" / name; fs::create_directories(disk); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index cb718605dd6..ecaf7d63fdc 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -233,7 +233,7 @@ public: virtual SyncGuardPtr getDirectorySyncGuard(const String & path) const; /// Applies new settings for disk in runtime. - virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) {} + virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextPtr) {} protected: friend class DiskDecorator; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 89c2d20db9f..be4fe705496 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -925,7 +925,7 @@ void DiskS3::onFreeze(const String & path) revision_file_buf.finalize(); } -void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextConstPtr context) +void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context) { auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 8857a00d709..d4a3b03ace4 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -55,7 +55,7 @@ public: using Futures = std::vector>; using SettingsPtr = std::unique_ptr; - using GetDiskSettings = std::function; + using GetDiskSettings = std::function; struct RestoreInformation; @@ -107,7 +107,7 @@ public: /// Dumps current revision counter into file 'revision.txt' at given path. void onFreeze(const String & path) override; - void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextConstPtr context) override; + void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context) override; private: void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata); diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 21eff5f3d31..b955db5fd5d 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -112,7 +112,7 @@ std::shared_ptr getProxyConfiguration(const String & pre } std::shared_ptr -getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextConstPtr context) +getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), @@ -146,7 +146,7 @@ getClient(const Poco::Util::AbstractConfiguration & config, const String & confi config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false))); } -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextConstPtr context) +std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { return std::make_unique( getClient(config, config_prefix, context), @@ -168,7 +168,7 @@ void registerDiskS3(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextConstPtr context) -> DiskPtr { + ContextPtr context) -> DiskPtr { S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); if (uri.key.back() != '/') throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 02b125330c5..f1f60ae2ac4 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -43,7 +43,7 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT); } -FormatSettings getFormatSettings(ContextConstPtr context) +FormatSettings getFormatSettings(ContextPtr context) { const auto & settings = context->getSettingsRef(); @@ -51,7 +51,7 @@ FormatSettings getFormatSettings(ContextConstPtr context) } template -FormatSettings getFormatSettings(ContextConstPtr context, const Settings & settings) +FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) { FormatSettings format_settings; @@ -124,16 +124,16 @@ FormatSettings getFormatSettings(ContextConstPtr context, const Settings & setti return format_settings; } -template FormatSettings getFormatSettings(ContextConstPtr context, const FormatFactorySettings & settings); +template FormatSettings getFormatSettings(ContextPtr context, const FormatFactorySettings & settings); -template FormatSettings getFormatSettings(ContextConstPtr context, const Settings & settings); +template FormatSettings getFormatSettings(ContextPtr context, const Settings & settings); InputFormatPtr FormatFactory::getInput( const String & name, ReadBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, UInt64 max_block_size, const std::optional & _format_settings) const { @@ -202,7 +202,7 @@ BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible( const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback, const std::optional & _format_settings) const { @@ -239,7 +239,7 @@ BlockOutputStreamPtr FormatFactory::getOutputStream( const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback, const std::optional & _format_settings) const { @@ -268,7 +268,7 @@ InputFormatPtr FormatFactory::getInputFormat( const String & name, ReadBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, UInt64 max_block_size, const std::optional & _format_settings) const { @@ -302,7 +302,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback, const std::optional & _format_settings) const { @@ -337,7 +337,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback, const std::optional & _format_settings) const { diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d5e4b1b0341..6edbafd2d64 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -35,10 +35,10 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; -FormatSettings getFormatSettings(ContextConstPtr context); +FormatSettings getFormatSettings(ContextPtr context); template -FormatSettings getFormatSettings(ContextConstPtr context, const T & settings); +FormatSettings getFormatSettings(ContextPtr context, const T & settings); /** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. * Note: format and compression are independent things. @@ -113,7 +113,7 @@ public: const String & name, ReadBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, UInt64 max_block_size, const std::optional & format_settings = std::nullopt) const; @@ -123,7 +123,7 @@ public: const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; @@ -132,7 +132,7 @@ public: const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; @@ -140,7 +140,7 @@ public: const String & name, ReadBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, UInt64 max_block_size, const std::optional & format_settings = std::nullopt) const; @@ -149,7 +149,7 @@ public: const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; @@ -157,7 +157,7 @@ public: const String & name, WriteBuffer & buf, const Block & sample, - ContextConstPtr context, + ContextPtr context, WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 6a8e3d2d94a..29aa5913b83 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -61,7 +61,7 @@ class FunctionBase64Conversion : public IFunction public: static constexpr auto name = Func::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 4ad876fc461..927b870891f 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -514,7 +514,7 @@ class FunctionBinaryArithmetic : public IFunction static constexpr const bool is_multiply = IsOperation::multiply; static constexpr const bool is_division = IsOperation::division; - ContextConstPtr context; + ContextPtr context; bool check_decimal_overflow = true; template @@ -597,7 +597,7 @@ class FunctionBinaryArithmetic : public IFunction } static FunctionOverloadResolverPtr - getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextConstPtr context) + getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { bool first_is_date_or_datetime = isDateOrDateTime(type0); bool second_is_date_or_datetime = isDateOrDateTime(type1); @@ -945,9 +945,9 @@ class FunctionBinaryArithmetic : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context); } + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionBinaryArithmetic(ContextConstPtr context_) + explicit FunctionBinaryArithmetic(ContextPtr context_) : context(context_), check_decimal_overflow(decimalCheckArithmeticOverflow(context)) {} @@ -961,7 +961,7 @@ public: return getReturnTypeImplStatic(arguments, context); } - static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, ContextConstPtr context) + static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, ContextPtr context) { /// Special case when multiply aggregate function state if (isAggregateMultiply(arguments[0], arguments[1])) @@ -1373,7 +1373,7 @@ public: const ColumnWithTypeAndName & left_, const ColumnWithTypeAndName & right_, const DataTypePtr & return_type_, - ContextConstPtr context) + ContextPtr context) { return std::make_shared(left_, right_, return_type_, context); } @@ -1382,7 +1382,7 @@ public: const ColumnWithTypeAndName & left_, const ColumnWithTypeAndName & right_, const DataTypePtr & return_type_, - ContextConstPtr context_) + ContextPtr context_) : Base(context_), left(left_), right(right_), return_type(return_type_) { } @@ -1536,12 +1536,12 @@ class BinaryArithmeticOverloadResolver : public IFunctionOverloadResolver { public: static constexpr auto name = Name::name; - static FunctionOverloadResolverPtr create(ContextConstPtr context) + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } - explicit BinaryArithmeticOverloadResolver(ContextConstPtr context_) : context(context_) {} + explicit BinaryArithmeticOverloadResolver(ContextPtr context_) : context(context_) {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -1577,6 +1577,6 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/FunctionBitTestMany.h b/src/Functions/FunctionBitTestMany.h index 030919e8836..567be37cda8 100644 --- a/src/Functions/FunctionBitTestMany.h +++ b/src/Functions/FunctionBitTestMany.h @@ -24,7 +24,7 @@ struct FunctionBitTestMany : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionCustomWeekToSomething.h b/src/Functions/FunctionCustomWeekToSomething.h index d30ce7b5c86..84ecd4d7a5a 100644 --- a/src/Functions/FunctionCustomWeekToSomething.h +++ b/src/Functions/FunctionCustomWeekToSomething.h @@ -24,7 +24,7 @@ class FunctionCustomWeekToSomething : public IFunction { public: static constexpr auto name = Transform::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 4418f6e4b4b..14ab9f00452 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -352,7 +352,7 @@ class FunctionDateOrDateTimeAddInterval : public IFunction { public: static constexpr auto name = Transform::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index bdf3f425a0b..8b8f1b483c8 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -24,7 +24,7 @@ class FunctionDateOrDateTimeToSomething : public IFunction { public: static constexpr auto name = Transform::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionFQDN.cpp b/src/Functions/FunctionFQDN.cpp index 36fcdfacc99..304aad17d22 100644 --- a/src/Functions/FunctionFQDN.cpp +++ b/src/Functions/FunctionFQDN.cpp @@ -12,7 +12,7 @@ class FunctionFQDN : public IFunction { public: static constexpr auto name = "FQDN"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 1bb78ead872..b1437d58c09 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -52,7 +52,7 @@ void FunctionFactory::registerFunction(const FunctionOverloadResolverPtr FunctionFactory::getImpl( const std::string & name, - ContextConstPtr context) const + ContextPtr context) const { auto res = tryGetImpl(name, context); if (!res) @@ -82,14 +82,14 @@ std::vector FunctionFactory::getAllNames() const FunctionOverloadResolverPtr FunctionFactory::get( const std::string & name, - ContextConstPtr context) const + ContextPtr context) const { return getImpl(name, context); } FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( const std::string & name_param, - ContextConstPtr context) const + ContextPtr context) const { String name = getAliasToOrName(name_param); FunctionOverloadResolverPtr res; @@ -120,7 +120,7 @@ FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( FunctionOverloadResolverPtr FunctionFactory::tryGet( const std::string & name, - ContextConstPtr context) const + ContextPtr context) const { auto impl = tryGetImpl(name, context); return impl ? std::move(impl) : nullptr; diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index bf380d51312..bac55dade16 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -19,7 +19,7 @@ namespace DB * some dictionaries from Context. */ class FunctionFactory : private boost::noncopyable, - public IFactoryWithAliases> + public IFactoryWithAliases> { public: static FunctionFactory & instance(); @@ -44,14 +44,14 @@ public: std::vector getAllNames() const; /// Throws an exception if not found. - FunctionOverloadResolverPtr get(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr get(const std::string & name, ContextPtr context) const; /// Returns nullptr if not found. - FunctionOverloadResolverPtr tryGet(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr tryGet(const std::string & name, ContextPtr context) const; /// The same methods to get developer interface implementation. - FunctionOverloadResolverPtr getImpl(const std::string & name, ContextConstPtr context) const; - FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr getImpl(const std::string & name, ContextPtr context) const; + FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextPtr context) const; /// Register a function by its name. /// No locking, you must register all functions before usage of get. @@ -67,7 +67,7 @@ private: Functions case_insensitive_functions; template - static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextConstPtr context) + static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextPtr context) { return std::make_unique(Function::create(context)); } diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index cd81f2b4e0b..3239705281e 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -22,12 +22,12 @@ namespace ErrorCodes } /// A function to read file as a string. -class FunctionFile : public IFunction, WithConstContext +class FunctionFile : public IFunction, WithContext { public: static constexpr auto name = "file"; - static FunctionPtr create(ContextConstPtr context_) { return std::make_shared(context_); } - explicit FunctionFile(ContextConstPtr context_) : WithConstContext(context_) {} + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } + explicit FunctionFile(ContextPtr context_) : WithContext(context_) {} String getName() const override { return name; } diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index 3a17d049099..a78cd70f419 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -36,7 +36,7 @@ ExecutableFunctionPtr FunctionJoinGet::prepare(const ColumnsWithTypeAnd } static std::pair, String> -getJoin(const ColumnsWithTypeAndName & arguments, ContextConstPtr context) +getJoin(const ColumnsWithTypeAndName & arguments, ContextPtr context) { String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index 0328d1a16e1..2250fa3ccf0 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -74,13 +74,13 @@ private: }; template -class JoinGetOverloadResolver final : public IFunctionOverloadResolver, WithConstContext +class JoinGetOverloadResolver final : public IFunctionOverloadResolver, WithContext { public: static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; - static FunctionOverloadResolverPtr create(ContextConstPtr context_) { return std::make_unique(context_); } + static FunctionOverloadResolverPtr create(ContextPtr context_) { return std::make_unique(context_); } - explicit JoinGetOverloadResolver(ContextConstPtr context_) : WithConstContext(context_) {} + explicit JoinGetOverloadResolver(ContextPtr context_) : WithContext(context_) {} String getName() const override { return name; } diff --git a/src/Functions/FunctionMathBinaryFloat64.h b/src/Functions/FunctionMathBinaryFloat64.h index 340470fa0b8..8cc012d3ab2 100644 --- a/src/Functions/FunctionMathBinaryFloat64.h +++ b/src/Functions/FunctionMathBinaryFloat64.h @@ -28,7 +28,7 @@ class FunctionMathBinaryFloat64 : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration"); bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/FunctionMathConstFloat64.h b/src/Functions/FunctionMathConstFloat64.h index b85b65fc43d..ab7d401e99e 100644 --- a/src/Functions/FunctionMathConstFloat64.h +++ b/src/Functions/FunctionMathConstFloat64.h @@ -13,7 +13,7 @@ class FunctionMathConstFloat64 : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index d7804af4f7a..a637bbe3bd8 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -35,7 +35,7 @@ class FunctionMathUnary : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/FunctionNumericPredicate.h b/src/Functions/FunctionNumericPredicate.h index 006f74f531a..d5fa9f03aba 100644 --- a/src/Functions/FunctionNumericPredicate.h +++ b/src/Functions/FunctionNumericPredicate.h @@ -23,7 +23,7 @@ class FunctionNumericPredicate : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index cf3a406498a..65dbf393290 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -142,7 +142,7 @@ template class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith { public: - explicit FunctionStartsEndsWith(ContextConstPtr context) : selector(context) + explicit FunctionStartsEndsWith(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -164,7 +164,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared>(context); } diff --git a/src/Functions/FunctionStringOrArrayToT.h b/src/Functions/FunctionStringOrArrayToT.h index ceb58a5f7f6..158179fffe9 100644 --- a/src/Functions/FunctionStringOrArrayToT.h +++ b/src/Functions/FunctionStringOrArrayToT.h @@ -25,7 +25,7 @@ class FunctionStringOrArrayToT : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionStringReplace.h b/src/Functions/FunctionStringReplace.h index 2ff09f79f9e..2e99f58531a 100644 --- a/src/Functions/FunctionStringReplace.h +++ b/src/Functions/FunctionStringReplace.h @@ -23,7 +23,7 @@ class FunctionStringReplace : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionStringToString.h b/src/Functions/FunctionStringToString.h index 9b63eec4a60..26480a83995 100644 --- a/src/Functions/FunctionStringToString.h +++ b/src/Functions/FunctionStringToString.h @@ -23,7 +23,7 @@ class FunctionStringToString : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 4b5fdf86078..68fcfadfb84 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -111,7 +111,7 @@ class FunctionUnaryArithmetic : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index 4dfe5cdea91..b76b454fd77 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -141,7 +141,7 @@ class FunctionEncrypt : public IFunction public: static constexpr OpenSSLDetails::CompatibilityMode compatibility_mode = Impl::compatibility_mode; static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: using CipherMode = OpenSSLDetails::CipherMode; @@ -416,7 +416,7 @@ class FunctionDecrypt : public IFunction public: static constexpr OpenSSLDetails::CompatibilityMode compatibility_mode = Impl::compatibility_mode; static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: using CipherMode = OpenSSLDetails::CipherMode; diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index 368e5bccbd1..bbdc53c3006 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -93,7 +93,7 @@ class FunctionBitmapBuildImpl : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -221,7 +221,7 @@ class FunctionBitmapToArrayImpl : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -311,7 +311,7 @@ class FunctionBitmapSubset : public IFunction public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { return name; } @@ -469,7 +469,7 @@ class FunctionBitmapTransform : public IFunction public: static constexpr auto name = "bitmapTransform"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -635,7 +635,7 @@ class FunctionBitmapSelfCardinalityImpl : public IFunction public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { return name; } @@ -807,7 +807,7 @@ class FunctionBitmapContains : public IFunction public: static constexpr auto name = "bitmapContains"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -911,7 +911,7 @@ class FunctionBitmapCardinality : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -1054,7 +1054,7 @@ class FunctionBitmap : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsCoding.h b/src/Functions/FunctionsCoding.h index 6c9cd6d333b..01602cf1175 100644 --- a/src/Functions/FunctionsCoding.h +++ b/src/Functions/FunctionsCoding.h @@ -69,7 +69,7 @@ class FunctionIPv6NumToString : public IFunction { public: static constexpr auto name = "IPv6NumToString"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -139,7 +139,7 @@ class FunctionCutIPv6 : public IFunction { public: static constexpr auto name = "cutIPv6"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -262,7 +262,7 @@ class FunctionIPv6StringToNum : public IFunction { public: static constexpr auto name = "IPv6StringToNum"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static inline bool tryParseIPv4(const char * pos) { @@ -340,7 +340,7 @@ class FunctionIPv4NumToString : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { @@ -401,7 +401,7 @@ class FunctionIPv4StringToNum : public IFunction { public: static constexpr auto name = "IPv4StringToNum"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -464,7 +464,7 @@ class FunctionIPv4ToIPv6 : public IFunction { public: static constexpr auto name = "IPv4ToIPv6"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -519,7 +519,7 @@ class FunctionToIPv4 : public FunctionIPv4StringToNum { public: static constexpr auto name = "toIPv4"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -542,7 +542,7 @@ class FunctionToIPv6 : public FunctionIPv6StringToNum { public: static constexpr auto name = "toIPv6"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -560,7 +560,7 @@ class FunctionMACNumToString : public IFunction { public: static constexpr auto name = "MACNumToString"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -690,7 +690,7 @@ class FunctionMACStringTo : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { @@ -753,7 +753,7 @@ class FunctionUUIDNumToString : public IFunction public: static constexpr auto name = "UUIDNumToString"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -851,7 +851,7 @@ private: public: static constexpr auto name = "UUIDStringToNum"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -955,7 +955,7 @@ class FunctionHex : public IFunction { public: static constexpr auto name = "hex"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1238,7 +1238,7 @@ class FunctionUnhex : public IFunction { public: static constexpr auto name = "unhex"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1327,7 +1327,7 @@ class FunctionChar : public IFunction { public: static constexpr auto name = "char"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1422,7 +1422,7 @@ class FunctionBitmaskToArray : public IFunction { public: static constexpr auto name = "bitmaskToArray"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1508,7 +1508,7 @@ class FunctionToStringCutToZero : public IFunction { public: static constexpr auto name = "toStringCutToZero"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1673,7 +1673,7 @@ private: public: static constexpr auto name = "IPv6CIDRToRange"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -1787,7 +1787,7 @@ private: public: static constexpr auto name = "IPv4CIDRToRange"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -1868,7 +1868,7 @@ class FunctionIsIPv4String : public FunctionIPv4StringToNum public: static constexpr auto name = "isIPv4String"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -1914,7 +1914,7 @@ class FunctionIsIPv6String : public FunctionIPv6StringToNum public: static constexpr auto name = "isIPv6String"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 0696b0fd335..9ffb0cd0fc3 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -557,13 +557,13 @@ class FunctionComparison : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context); } + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionComparison(ContextConstPtr context_) + explicit FunctionComparison(ContextPtr context_) : context(context_), check_decimal_overflow(decimalCheckComparisonOverflow(context)) {} private: - ContextConstPtr context; + ContextPtr context; bool check_decimal_overflow = true; template diff --git a/src/Functions/FunctionsConsistentHashing.h b/src/Functions/FunctionsConsistentHashing.h index 69163917430..4c393f6ee01 100644 --- a/src/Functions/FunctionsConsistentHashing.h +++ b/src/Functions/FunctionsConsistentHashing.h @@ -24,7 +24,7 @@ class FunctionConsistentHashImpl : public IFunction public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5af86d9ff8b..589bd4cc16e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1277,7 +1277,7 @@ public: static constexpr bool to_string_or_fixed_string = std::is_same_v || std::is_same_v; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static FunctionPtr create() { return std::make_shared(); } String getName() const override @@ -1592,7 +1592,7 @@ public: static constexpr bool to_datetime64 = std::is_same_v; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static FunctionPtr create() { return std::make_shared(); } String getName() const override @@ -3196,7 +3196,7 @@ public: ? accurate_cast_name : (cast_type == CastType::accurateOrNull ? accurate_cast_or_null_name : cast_name); - static FunctionOverloadResolverPtr create(ContextConstPtr context) + static FunctionOverloadResolverPtr create(ContextPtr context) { return createImpl(context->getSettingsRef().cast_keep_nullable); } diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index a19c6226f91..884f53125eb 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -483,7 +483,7 @@ struct NameRegionIn { static constexpr auto name = "regionIn" struct FunctionRegionToCity : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -492,7 +492,7 @@ struct FunctionRegionToCity : struct FunctionRegionToArea : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -501,7 +501,7 @@ struct FunctionRegionToArea : struct FunctionRegionToDistrict : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -510,7 +510,7 @@ struct FunctionRegionToDistrict : struct FunctionRegionToCountry : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -519,7 +519,7 @@ struct FunctionRegionToCountry : struct FunctionRegionToContinent : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -528,7 +528,7 @@ struct FunctionRegionToContinent : struct FunctionRegionToTopContinent : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -537,7 +537,7 @@ struct FunctionRegionToTopContinent : struct FunctionRegionToPopulation : public FunctionTransformWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -546,7 +546,7 @@ struct FunctionRegionToPopulation : struct FunctionRegionIn : public FunctionIsInWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -555,7 +555,7 @@ struct FunctionRegionIn : struct FunctionRegionHierarchy : public FunctionHierarchyWithDictionary { - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsHierarchies()); } @@ -567,7 +567,7 @@ class FunctionRegionToName : public IFunction { public: static constexpr auto name = "regionToName"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getEmbeddedDictionaries().getRegionsNames()); } diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 89c76f93061..6d9e4ce6768 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -62,10 +62,10 @@ namespace ErrorCodes */ -class FunctionDictHelper : WithConstContext +class FunctionDictHelper : WithContext { public: - explicit FunctionDictHelper(ContextConstPtr context_) : WithConstContext(context_) {} + explicit FunctionDictHelper(ContextPtr context_) : WithContext(context_) {} std::shared_ptr getDictionary(const String & dictionary_name) { @@ -132,12 +132,12 @@ class FunctionDictHas final : public IFunction public: static constexpr auto name = "dictHas"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictHas(ContextConstPtr context_) : helper(context_) {} + explicit FunctionDictHas(ContextPtr context_) : helper(context_) {} String getName() const override { return name; } @@ -270,12 +270,12 @@ class FunctionDictGetNoType final : public IFunction public: static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictGetNoType(ContextConstPtr context_) : helper(context_) {} + explicit FunctionDictGetNoType(ContextPtr context_) : helper(context_) {} String getName() const override { return name; } @@ -604,12 +604,12 @@ class FunctionDictGetImpl final : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictGetImpl(ContextConstPtr context_) : impl(context_) {} + explicit FunctionDictGetImpl(ContextPtr context_) : impl(context_) {} String getName() const override { return name; } @@ -743,12 +743,12 @@ class FunctionDictGetOrNull final : public IFunction public: static constexpr auto name = "dictGetOrNull"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictGetOrNull(ContextConstPtr context_) + explicit FunctionDictGetOrNull(ContextPtr context_) : dictionary_get_func_impl(context_) , dictionary_has_func_impl(context_) {} @@ -906,12 +906,12 @@ class FunctionDictGetHierarchy final : public IFunction public: static constexpr auto name = "dictGetHierarchy"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictGetHierarchy(ContextConstPtr context_) : helper(context_) {} + explicit FunctionDictGetHierarchy(ContextPtr context_) : helper(context_) {} String getName() const override { return name; } @@ -966,12 +966,12 @@ class FunctionDictIsIn final : public IFunction public: static constexpr auto name = "dictIsIn"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictIsIn(ContextConstPtr context_) + explicit FunctionDictIsIn(ContextPtr context_) : helper(context_) {} String getName() const override { return name; } @@ -1032,12 +1032,12 @@ class FunctionDictGetChildren final : public IFunction public: static constexpr auto name = "dictGetChildren"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictGetChildren(ContextConstPtr context_) + explicit FunctionDictGetChildren(ContextPtr context_) : helper(context_) {} String getName() const override { return name; } @@ -1091,12 +1091,12 @@ class FunctionDictGetDescendants final : public IFunction public: static constexpr auto name = "dictGetDescendants"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionDictGetDescendants(ContextConstPtr context_) + explicit FunctionDictGetDescendants(ContextPtr context_) : helper(context_) {} String getName() const override { return name; } diff --git a/src/Functions/FunctionsExternalModels.cpp b/src/Functions/FunctionsExternalModels.cpp index 6bcaf13cd69..da46f7fcb38 100644 --- a/src/Functions/FunctionsExternalModels.cpp +++ b/src/Functions/FunctionsExternalModels.cpp @@ -18,7 +18,7 @@ namespace DB { -FunctionPtr FunctionModelEvaluate::create(ContextConstPtr context) +FunctionPtr FunctionModelEvaluate::create(ContextPtr context) { return std::make_shared(context->getExternalModelsLoader()); } diff --git a/src/Functions/FunctionsExternalModels.h b/src/Functions/FunctionsExternalModels.h index b9d611934e6..8f8b0e0c860 100644 --- a/src/Functions/FunctionsExternalModels.h +++ b/src/Functions/FunctionsExternalModels.h @@ -17,7 +17,7 @@ class FunctionModelEvaluate final : public IFunction public: static constexpr auto name = "modelEvaluate"; - static FunctionPtr create(ContextConstPtr context); + static FunctionPtr create(ContextPtr context); explicit FunctionModelEvaluate(const ExternalModelsLoader & models_loader_) : models_loader(models_loader_) {} diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 25f57e06f2e..690991759a3 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -535,7 +535,7 @@ class FunctionStringHashFixedString : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -700,7 +700,7 @@ template class FunctionIntHash : public TargetSpecific::Default::FunctionIntHash { public: - explicit FunctionIntHash(ContextConstPtr context) : selector(context) + explicit FunctionIntHash(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -718,7 +718,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -1077,7 +1077,7 @@ template class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash { public: - explicit FunctionAnyHash(ContextConstPtr context) : selector(context) + explicit FunctionAnyHash(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -1095,7 +1095,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -1182,7 +1182,7 @@ class FunctionURLHash : public IFunction { public: static constexpr auto name = "URLHash"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 6024658dd6f..2b49d01de2c 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -270,11 +270,11 @@ private: template typename Impl> -class FunctionJSON : public IFunction, WithConstContext +class FunctionJSON : public IFunction, WithContext { public: - static FunctionPtr create(ContextConstPtr context_) { return std::make_shared(context_); } - FunctionJSON(ContextConstPtr context_) : WithConstContext(context_) {} + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } + FunctionJSON(ContextPtr context_) : WithContext(context_) {} static constexpr auto name = Name::name; String getName() const override { return Name::name; } diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 339cc25510d..264eeeef0fe 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -139,7 +139,7 @@ class FunctionAnyArityLogical : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } public: String getName() const override @@ -206,7 +206,7 @@ class FunctionUnaryLogical : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } public: String getName() const override diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 3c6e67c1546..209efb0fc2f 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -34,7 +34,7 @@ class FunctionsMultiStringFuzzySearch : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { if (Impl::is_using_hyperscan && !context->getSettingsRef().allow_hyperscan) throw Exception( diff --git a/src/Functions/FunctionsMultiStringPosition.h b/src/Functions/FunctionsMultiStringPosition.h index 68c658c0aa7..f36f7639ccd 100644 --- a/src/Functions/FunctionsMultiStringPosition.h +++ b/src/Functions/FunctionsMultiStringPosition.h @@ -51,7 +51,7 @@ class FunctionsMultiStringPosition : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 4d391124de9..08b4668940e 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -47,7 +47,7 @@ class FunctionsMultiStringSearch : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { if (Impl::is_using_hyperscan && !context->getSettingsRef().allow_hyperscan) throw Exception( diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 0eae9ef183f..75037d02a2d 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -91,7 +91,7 @@ template class FunctionRandom : public FunctionRandomImpl { public: - explicit FunctionRandom(ContextConstPtr context) : selector(context) + explicit FunctionRandom(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -107,7 +107,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared>(context); } diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 98f35e52a4c..f56b92d6db5 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -520,7 +520,7 @@ class FunctionRounding : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -629,7 +629,7 @@ class FunctionRoundDown : public IFunction { public: static constexpr auto name = "roundDown"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index 2141adbc72f..27f10797651 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -427,7 +427,7 @@ class FunctionTokens : public IFunction { public: static constexpr auto name = Generator::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -590,7 +590,7 @@ private: public: static constexpr auto name = "arrayStringConcat"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionsStringHash.h b/src/Functions/FunctionsStringHash.h index 8336a4ee85f..37fa7d618b9 100644 --- a/src/Functions/FunctionsStringHash.h +++ b/src/Functions/FunctionsStringHash.h @@ -35,7 +35,7 @@ public: static constexpr size_t max_shingle_size = 25; static constexpr size_t max_num_hashes = 25; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index 524fe43f42f..0789247e2d4 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -51,7 +51,7 @@ class FunctionsStringSearch : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsStringSearchToString.h b/src/Functions/FunctionsStringSearchToString.h index 3b3383cf162..af91a9511e1 100644 --- a/src/Functions/FunctionsStringSearchToString.h +++ b/src/Functions/FunctionsStringSearchToString.h @@ -36,7 +36,7 @@ class FunctionsStringSearchToString : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsStringSimilarity.h b/src/Functions/FunctionsStringSimilarity.h index 49fc8a21129..6efb373aace 100644 --- a/src/Functions/FunctionsStringSimilarity.h +++ b/src/Functions/FunctionsStringSimilarity.h @@ -33,7 +33,7 @@ class FunctionsStringSimilarity : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/LeastGreatestGeneric.h b/src/Functions/LeastGreatestGeneric.h index f33bda801cb..51564ad7cdc 100644 --- a/src/Functions/LeastGreatestGeneric.h +++ b/src/Functions/LeastGreatestGeneric.h @@ -30,7 +30,7 @@ class FunctionLeastGreatestGeneric : public IFunction { public: static constexpr auto name = kind == LeastGreatest::Least ? "least" : "greatest"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } private: String getName() const override { return name; } @@ -92,12 +92,12 @@ class LeastGreatestOverloadResolver : public IFunctionOverloadResolver public: static constexpr auto name = kind == LeastGreatest::Least ? "least" : "greatest"; - static FunctionOverloadResolverPtr create(ContextConstPtr context) + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique>(context); } - explicit LeastGreatestOverloadResolver(ContextConstPtr context_) : context(context_) {} + explicit LeastGreatestOverloadResolver(ContextPtr context_) : context(context_) {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -127,7 +127,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 04bf19bbd88..9ef6454d085 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -172,7 +172,7 @@ namespace detail * /// default implementation. * class MyFunction : public MyDefaultImpl * { - * MyFunction(ContextConstPtr context) : selector(context) { + * MyFunction(ContextPtr context) : selector(context) { * /// Register all implementations in constructor. * /// There could be as many implementation for every target as you want. * selector.registerImplementation(); @@ -185,7 +185,7 @@ namespace detail * selector.selectAndExecute(...); * } * - * static FunctionPtr create(ContextConstPtr context) { + * static FunctionPtr create(ContextPtr context) { * return std::make_shared(context); * } * private: @@ -193,12 +193,12 @@ namespace detail * }; */ template -class ImplementationSelector : WithConstContext +class ImplementationSelector : WithContext { public: using ImplementationPtr = std::shared_ptr; - ImplementationSelector(ContextConstPtr context_) : WithConstContext(context_) {} + ImplementationSelector(ContextPtr context_) : WithContext(context_) {} /* Select the best implementation based on previous runs. * If FunctionInterface is IFunction, then "executeImpl" method of the implementation will be called diff --git a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h index fa985f194f3..4670d610725 100644 --- a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h +++ b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h @@ -36,7 +36,7 @@ class FunctionCutToFirstSignificantSubdomainCustomImpl : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index 6f277de45a5..179a2be4471 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes struct FunctionPort : public IFunction { static constexpr auto name = "port"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } diff --git a/src/Functions/abtesting.cpp b/src/Functions/abtesting.cpp index fa0e5e8a6bd..871357fe450 100644 --- a/src/Functions/abtesting.cpp +++ b/src/Functions/abtesting.cpp @@ -168,7 +168,7 @@ class FunctionBayesAB : public IFunction public: static constexpr auto name = "bayesAB"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/addressToLine.cpp b/src/Functions/addressToLine.cpp index 1ae8d9126bb..aabf388b428 100644 --- a/src/Functions/addressToLine.cpp +++ b/src/Functions/addressToLine.cpp @@ -36,7 +36,7 @@ class FunctionAddressToLine : public IFunction { public: static constexpr auto name = "addressToLine"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { context->checkAccess(AccessType::addressToLine); return std::make_shared(); diff --git a/src/Functions/addressToSymbol.cpp b/src/Functions/addressToSymbol.cpp index 1c744a11755..7ffdc6d4260 100644 --- a/src/Functions/addressToSymbol.cpp +++ b/src/Functions/addressToSymbol.cpp @@ -28,7 +28,7 @@ class FunctionAddressToSymbol : public IFunction { public: static constexpr auto name = "addressToSymbol"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { context->checkAccess(AccessType::addressToSymbol); return std::make_shared(); diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index 89d7f12f780..266b2a44f4a 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -24,7 +24,7 @@ class FunctionAppendTrailingCharIfAbsent : public IFunction { public: static constexpr auto name = "appendTrailingCharIfAbsent"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 9b51000144f..3609398bc3f 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -44,7 +44,7 @@ class FunctionArrayMapped : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index 62ae0b4f2d8..1f513defc6f 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -14,7 +14,7 @@ class FunctionArray : public IFunction { public: static constexpr auto name = "array"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/array/arrayConcat.cpp b/src/Functions/array/arrayConcat.cpp index 23193e28816..57a783a3244 100644 --- a/src/Functions/array/arrayConcat.cpp +++ b/src/Functions/array/arrayConcat.cpp @@ -26,7 +26,7 @@ class FunctionArrayConcat : public IFunction { public: static constexpr auto name = "arrayConcat"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayDistinct.cpp b/src/Functions/array/arrayDistinct.cpp index 164e3b19c83..916af560c8f 100644 --- a/src/Functions/array/arrayDistinct.cpp +++ b/src/Functions/array/arrayDistinct.cpp @@ -26,7 +26,7 @@ class FunctionArrayDistinct : public IFunction public: static constexpr auto name = "arrayDistinct"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 5ec5c4d4213..f3d3f558d7b 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -41,7 +41,7 @@ class FunctionArrayElement : public IFunction { public: static constexpr auto name = "arrayElement"; - static FunctionPtr create(ContextConstPtr context); + static FunctionPtr create(ContextPtr context); String getName() const override; @@ -449,7 +449,7 @@ struct ArrayElementGenericImpl } -FunctionPtr FunctionArrayElement::create(ContextConstPtr) +FunctionPtr FunctionArrayElement::create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/array/arrayEnumerate.cpp b/src/Functions/array/arrayEnumerate.cpp index a80e4f9e016..6d59a2c3805 100644 --- a/src/Functions/array/arrayEnumerate.cpp +++ b/src/Functions/array/arrayEnumerate.cpp @@ -22,7 +22,7 @@ class FunctionArrayEnumerate : public IFunction public: static constexpr auto name = "arrayEnumerate"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/array/arrayEnumerateExtended.h b/src/Functions/array/arrayEnumerateExtended.h index 921a0f0c75d..7e4fe24873a 100644 --- a/src/Functions/array/arrayEnumerateExtended.h +++ b/src/Functions/array/arrayEnumerateExtended.h @@ -30,7 +30,7 @@ template class FunctionArrayEnumerateExtended : public IFunction { public: - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return Derived::name; } diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 3dbb2d2d0c9..2c999415f33 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -90,7 +90,7 @@ template class FunctionArrayEnumerateRankedExtended : public IFunction { public: - static FunctionPtr create(ContextConstPtr /* context */) { return std::make_shared(); } + static FunctionPtr create(ContextPtr /* context */) { return std::make_shared(); } String getName() const override { return Derived::name; } diff --git a/src/Functions/array/arrayFlatten.cpp b/src/Functions/array/arrayFlatten.cpp index dfd1811d91d..3d286aa0bb4 100644 --- a/src/Functions/array/arrayFlatten.cpp +++ b/src/Functions/array/arrayFlatten.cpp @@ -20,7 +20,7 @@ class ArrayFlatten : public IFunction public: static constexpr auto name = "arrayFlatten"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index fbd3501298e..f3b279faaef 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -355,7 +355,7 @@ class FunctionArrayIndex : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } /// Get function name. String getName() const override { return name; } diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 1d3bcdd6fcc..93c1ee06403 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -38,8 +38,8 @@ class FunctionArrayIntersect : public IFunction { public: static constexpr auto name = "arrayIntersect"; - static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context); } - explicit FunctionArrayIntersect(ContextConstPtr context_) : context(context_) {} + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionArrayIntersect(ContextPtr context_) : context(context_) {} String getName() const override { return name; } @@ -53,7 +53,7 @@ public: bool useDefaultImplementationForConstants() const override { return true; } private: - ContextConstPtr context; + ContextPtr context; /// Initially allocate a piece of memory for 64 elements. NOTE: This is just a guess. static constexpr size_t INITIAL_SIZE_DEGREE = 6; @@ -281,7 +281,7 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns( return {.initial = initial_columns, .casted = casted_columns}; } -static ColumnPtr callFunctionNotEquals(ColumnWithTypeAndName first, ColumnWithTypeAndName second, ContextConstPtr context) +static ColumnPtr callFunctionNotEquals(ColumnWithTypeAndName first, ColumnWithTypeAndName second, ContextPtr context) { ColumnsWithTypeAndName args{first, second}; auto eq_func = FunctionFactory::instance().get("notEquals", context)->build(args); diff --git a/src/Functions/array/arrayJoin.cpp b/src/Functions/array/arrayJoin.cpp index 953c29ae67d..da8c4e6e80b 100644 --- a/src/Functions/array/arrayJoin.cpp +++ b/src/Functions/array/arrayJoin.cpp @@ -20,7 +20,7 @@ class FunctionArrayJoin : public IFunction { public: static constexpr auto name = "arrayJoin"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/array/arrayPopBack.cpp b/src/Functions/array/arrayPopBack.cpp index dea16c228fb..0b532c22661 100644 --- a/src/Functions/array/arrayPopBack.cpp +++ b/src/Functions/array/arrayPopBack.cpp @@ -9,7 +9,7 @@ class FunctionArrayPopBack : public FunctionArrayPop { public: static constexpr auto name = "arrayPopBack"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayPopBack() : FunctionArrayPop(false, name) {} }; diff --git a/src/Functions/array/arrayPopFront.cpp b/src/Functions/array/arrayPopFront.cpp index 14a2d5f583e..57eedf9b44a 100644 --- a/src/Functions/array/arrayPopFront.cpp +++ b/src/Functions/array/arrayPopFront.cpp @@ -9,7 +9,7 @@ class FunctionArrayPopFront : public FunctionArrayPop { public: static constexpr auto name = "arrayPopFront"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayPopFront() : FunctionArrayPop(true, name) {} }; diff --git a/src/Functions/array/arrayPushBack.cpp b/src/Functions/array/arrayPushBack.cpp index 1e12bf72d87..3171bca9d2c 100644 --- a/src/Functions/array/arrayPushBack.cpp +++ b/src/Functions/array/arrayPushBack.cpp @@ -9,7 +9,7 @@ class FunctionArrayPushBack : public FunctionArrayPush { public: static constexpr auto name = "arrayPushBack"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayPushBack() : FunctionArrayPush(false, name) {} }; diff --git a/src/Functions/array/arrayPushFront.cpp b/src/Functions/array/arrayPushFront.cpp index 08b4efa207e..d12dfe02af1 100644 --- a/src/Functions/array/arrayPushFront.cpp +++ b/src/Functions/array/arrayPushFront.cpp @@ -10,7 +10,7 @@ class FunctionArrayPushFront : public FunctionArrayPush { public: static constexpr auto name = "arrayPushFront"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayPushFront() : FunctionArrayPush(true, name) {} }; diff --git a/src/Functions/array/arrayReduce.cpp b/src/Functions/array/arrayReduce.cpp index 3126554c127..bbdd142c062 100644 --- a/src/Functions/array/arrayReduce.cpp +++ b/src/Functions/array/arrayReduce.cpp @@ -37,7 +37,7 @@ class FunctionArrayReduce : public IFunction { public: static constexpr auto name = "arrayReduce"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayReduceInRanges.cpp b/src/Functions/array/arrayReduceInRanges.cpp index dfeb79027f2..bcb34385d17 100644 --- a/src/Functions/array/arrayReduceInRanges.cpp +++ b/src/Functions/array/arrayReduceInRanges.cpp @@ -40,7 +40,7 @@ class FunctionArrayReduceInRanges : public IFunction public: static const size_t minimum_step = 64; static constexpr auto name = "arrayReduceInRanges"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayResize.cpp b/src/Functions/array/arrayResize.cpp index 03b0430d6ec..f8eea06335b 100644 --- a/src/Functions/array/arrayResize.cpp +++ b/src/Functions/array/arrayResize.cpp @@ -25,7 +25,7 @@ class FunctionArrayResize : public IFunction { public: static constexpr auto name = "arrayResize"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayReverse.cpp b/src/Functions/array/arrayReverse.cpp index 2a2e24828ac..fb4a559b37f 100644 --- a/src/Functions/array/arrayReverse.cpp +++ b/src/Functions/array/arrayReverse.cpp @@ -24,7 +24,7 @@ class FunctionArrayReverse : public IFunction { public: static constexpr auto name = "arrayReverse"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 0d35157d133..6a23d6a45d8 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -25,7 +25,7 @@ class FunctionArrayScalarProduct : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: using ResultColumnType = ColumnVector; diff --git a/src/Functions/array/arraySlice.cpp b/src/Functions/array/arraySlice.cpp index 4fc32bfa981..567135de884 100644 --- a/src/Functions/array/arraySlice.cpp +++ b/src/Functions/array/arraySlice.cpp @@ -34,7 +34,7 @@ class FunctionArraySlice : public IFunction { public: static constexpr auto name = "arraySlice"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayUniq.cpp b/src/Functions/array/arrayUniq.cpp index 6d461206f38..62de746f136 100644 --- a/src/Functions/array/arrayUniq.cpp +++ b/src/Functions/array/arrayUniq.cpp @@ -31,7 +31,7 @@ class FunctionArrayUniq : public IFunction public: static constexpr auto name = "arrayUniq"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/arrayWithConstant.cpp b/src/Functions/array/arrayWithConstant.cpp index 7d00c2227ee..578e8bf2296 100644 --- a/src/Functions/array/arrayWithConstant.cpp +++ b/src/Functions/array/arrayWithConstant.cpp @@ -30,7 +30,7 @@ class FunctionArrayWithConstant : public IFunction public: static constexpr auto name = "arrayWithConstant"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 8e4d41ffca2..f77b4f6dfe9 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -23,7 +23,7 @@ class FunctionArrayZip : public IFunction { public: static constexpr auto name = "arrayZip"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/array/emptyArray.cpp b/src/Functions/array/emptyArray.cpp index 38b68043628..370c51f3e8d 100644 --- a/src/Functions/array/emptyArray.cpp +++ b/src/Functions/array/emptyArray.cpp @@ -22,7 +22,7 @@ class FunctionEmptyArray : public IFunction { public: static String getNameImpl() { return "emptyArray" + DataType().getName(); } - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override diff --git a/src/Functions/array/emptyArrayToSingle.cpp b/src/Functions/array/emptyArrayToSingle.cpp index a116adca4e7..be387620e60 100644 --- a/src/Functions/array/emptyArrayToSingle.cpp +++ b/src/Functions/array/emptyArrayToSingle.cpp @@ -27,7 +27,7 @@ class FunctionEmptyArrayToSingle : public IFunction { public: static constexpr auto name = "emptyArrayToSingle"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/array/hasAll.cpp b/src/Functions/array/hasAll.cpp index 7b5b83334b9..c1959515e9f 100644 --- a/src/Functions/array/hasAll.cpp +++ b/src/Functions/array/hasAll.cpp @@ -10,7 +10,7 @@ class FunctionArrayHasAll : public FunctionArrayHasAllAny { public: static constexpr auto name = "hasAll"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayHasAll() : FunctionArrayHasAllAny(GatherUtils::ArraySearchType::All, name) {} }; diff --git a/src/Functions/array/hasAny.cpp b/src/Functions/array/hasAny.cpp index 94d429d6c0e..fac810c7ba1 100644 --- a/src/Functions/array/hasAny.cpp +++ b/src/Functions/array/hasAny.cpp @@ -10,7 +10,7 @@ class FunctionArrayHasAny : public FunctionArrayHasAllAny { public: static constexpr auto name = "hasAny"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayHasAny() : FunctionArrayHasAllAny(GatherUtils::ArraySearchType::Any, name) {} }; diff --git a/src/Functions/array/hasSubstr.cpp b/src/Functions/array/hasSubstr.cpp index f304e8bc3ba..886f82f577b 100644 --- a/src/Functions/array/hasSubstr.cpp +++ b/src/Functions/array/hasSubstr.cpp @@ -10,7 +10,7 @@ class FunctionArrayHasSubstr : public FunctionArrayHasAllAny { public: static constexpr auto name = "hasSubstr"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } FunctionArrayHasSubstr() : FunctionArrayHasAllAny(GatherUtils::ArraySearchType::Substr, name) {} }; diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index 0709c1f2664..1a19ee41d2f 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -43,7 +43,7 @@ class FunctionMapOp : public IFunction { public: static constexpr auto name = (op_type == OpTypes::ADD) ? "mapAdd" : "mapSubtract"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/array/mapPopulateSeries.cpp b/src/Functions/array/mapPopulateSeries.cpp index 7e05eb81150..eb2f6192346 100644 --- a/src/Functions/array/mapPopulateSeries.cpp +++ b/src/Functions/array/mapPopulateSeries.cpp @@ -23,7 +23,7 @@ class FunctionMapPopulateSeries : public IFunction { public: static constexpr auto name = "mapPopulateSeries"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index 9f4c9fb0676..5b9886580dc 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -32,7 +32,7 @@ class FunctionRange : public IFunction public: static constexpr auto name = "range"; static constexpr size_t max_elements = 100'000'000; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/assumeNotNull.cpp b/src/Functions/assumeNotNull.cpp index 2b8743b1507..019c637e50c 100644 --- a/src/Functions/assumeNotNull.cpp +++ b/src/Functions/assumeNotNull.cpp @@ -19,7 +19,7 @@ class FunctionAssumeNotNull : public IFunction public: static constexpr auto name = "assumeNotNull"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/bar.cpp b/src/Functions/bar.cpp index 49f3be46d60..2e26f7af0bb 100644 --- a/src/Functions/bar.cpp +++ b/src/Functions/bar.cpp @@ -28,7 +28,7 @@ class FunctionBar : public IFunction { public: static constexpr auto name = "bar"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/bitHammingDistance.cpp b/src/Functions/bitHammingDistance.cpp index a89ffa16010..25b902b8bd5 100644 --- a/src/Functions/bitHammingDistance.cpp +++ b/src/Functions/bitHammingDistance.cpp @@ -74,7 +74,7 @@ class FunctionBitHammingDistance : public IFunction public: static constexpr auto name = "bitHammingDistance"; using ResultType = UInt8; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/bitmaskToList.cpp b/src/Functions/bitmaskToList.cpp index 0f3362aa031..8c3105724ac 100644 --- a/src/Functions/bitmaskToList.cpp +++ b/src/Functions/bitmaskToList.cpp @@ -30,7 +30,7 @@ class FunctionBitmaskToList : public IFunction { public: static constexpr auto name = "bitmaskToList"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/blockNumber.cpp b/src/Functions/blockNumber.cpp index 6998c02b3b6..9a57f8a96b0 100644 --- a/src/Functions/blockNumber.cpp +++ b/src/Functions/blockNumber.cpp @@ -18,7 +18,7 @@ private: public: static constexpr auto name = "blockNumber"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/blockSerializedSize.cpp b/src/Functions/blockSerializedSize.cpp index bd65f6dd084..30f77bbf627 100644 --- a/src/Functions/blockSerializedSize.cpp +++ b/src/Functions/blockSerializedSize.cpp @@ -15,7 +15,7 @@ class FunctionBlockSerializedSize : public IFunction public: static constexpr auto name = "blockSerializedSize"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/blockSize.cpp b/src/Functions/blockSize.cpp index 481a89b1b82..294686054f0 100644 --- a/src/Functions/blockSize.cpp +++ b/src/Functions/blockSize.cpp @@ -15,7 +15,7 @@ class FunctionBlockSize : public IFunction { public: static constexpr auto name = "blockSize"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp index 4f452ac38a2..cc0c21350ca 100644 --- a/src/Functions/buildId.cpp +++ b/src/Functions/buildId.cpp @@ -18,7 +18,7 @@ class FunctionBuildId : public IFunction { public: static constexpr auto name = "buildId"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/byteSize.cpp b/src/Functions/byteSize.cpp index 5d43ddb2929..54a7da59b9c 100644 --- a/src/Functions/byteSize.cpp +++ b/src/Functions/byteSize.cpp @@ -17,7 +17,7 @@ class FunctionByteSize : public IFunction { public: static constexpr auto name = "byteSize"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/caseWithExpression.cpp b/src/Functions/caseWithExpression.cpp index 95c06ee683a..24b5855cc5b 100644 --- a/src/Functions/caseWithExpression.cpp +++ b/src/Functions/caseWithExpression.cpp @@ -20,9 +20,9 @@ class FunctionCaseWithExpression : public IFunction { public: static constexpr auto name = "caseWithExpression"; - static FunctionPtr create(ContextConstPtr context_) { return std::make_shared(context_); } + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - explicit FunctionCaseWithExpression(ContextConstPtr context_) : context(context_) {} + explicit FunctionCaseWithExpression(ContextPtr context_) : context(context_) {} bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } String getName() const override { return name; } @@ -98,7 +98,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/coalesce.cpp b/src/Functions/coalesce.cpp index 69b69b7d3f1..791b9d993b4 100644 --- a/src/Functions/coalesce.cpp +++ b/src/Functions/coalesce.cpp @@ -24,12 +24,12 @@ class FunctionCoalesce : public IFunction public: static constexpr auto name = "coalesce"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionCoalesce(ContextConstPtr context_) : context(context_) {} + explicit FunctionCoalesce(ContextPtr context_) : context(context_) {} std::string getName() const override { @@ -160,7 +160,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index cc5636ffb5f..c1b9f8e7967 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -33,8 +33,8 @@ class ConcatImpl : public IFunction { public: static constexpr auto name = Name::name; - explicit ConcatImpl(ContextConstPtr context_) : context(context_) {} - static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context); } + explicit ConcatImpl(ContextPtr context_) : context(context_) {} + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } String getName() const override { return name; } @@ -85,7 +85,7 @@ public: } private: - ContextWeakConstPtr context; + ContextWeakPtr context; ColumnPtr executeBinary(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { @@ -191,9 +191,9 @@ class ConcatOverloadResolver : public IFunctionOverloadResolver { public: static constexpr auto name = "concat"; - static FunctionOverloadResolverPtr create(ContextConstPtr context) { return std::make_unique(context); } + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } - explicit ConcatOverloadResolver(ContextConstPtr context_) : context(context_) {} + explicit ConcatOverloadResolver(ContextPtr context_) : context(context_) {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -223,7 +223,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/connectionId.cpp b/src/Functions/connectionId.cpp index 4d22bb6ffad..ae04cfd1af6 100644 --- a/src/Functions/connectionId.cpp +++ b/src/Functions/connectionId.cpp @@ -8,14 +8,14 @@ namespace DB { /// Get the connection Id. It's used for MySQL handler only. -class FunctionConnectionId : public IFunction, WithConstContext +class FunctionConnectionId : public IFunction, WithContext { public: static constexpr auto name = "connectionId"; - explicit FunctionConnectionId(ContextConstPtr context_) : WithConstContext(context_) {} + explicit FunctionConnectionId(ContextPtr context_) : WithContext(context_) {} - static FunctionPtr create(ContextConstPtr context_) { return std::make_shared(context_); } + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } String getName() const override { return name; } diff --git a/src/Functions/convertCharset.cpp b/src/Functions/convertCharset.cpp index e7a43e73a80..b204875b4ff 100644 --- a/src/Functions/convertCharset.cpp +++ b/src/Functions/convertCharset.cpp @@ -162,7 +162,7 @@ private: public: static constexpr auto name = "convertCharset"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/countDigits.cpp b/src/Functions/countDigits.cpp index 76038a88d41..597a2c625b9 100644 --- a/src/Functions/countDigits.cpp +++ b/src/Functions/countDigits.cpp @@ -28,7 +28,7 @@ class FunctionCountDigits : public IFunction public: static constexpr auto name = "countDigits"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/countMatches.h b/src/Functions/countMatches.h index 0c6a95a94ea..6ae69520cb9 100644 --- a/src/Functions/countMatches.h +++ b/src/Functions/countMatches.h @@ -27,7 +27,7 @@ class FunctionCountMatches : public IFunction { public: static constexpr auto name = CountMatchesBase::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } diff --git a/src/Functions/currentDatabase.cpp b/src/Functions/currentDatabase.cpp index efb98260e02..16eff20cfd5 100644 --- a/src/Functions/currentDatabase.cpp +++ b/src/Functions/currentDatabase.cpp @@ -16,7 +16,7 @@ class FunctionCurrentDatabase : public IFunction public: static constexpr auto name = "currentDatabase"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getCurrentDatabase()); } diff --git a/src/Functions/currentUser.cpp b/src/Functions/currentUser.cpp index ea9943b3647..22ad49fb29d 100644 --- a/src/Functions/currentUser.cpp +++ b/src/Functions/currentUser.cpp @@ -16,7 +16,7 @@ class FunctionCurrentUser : public IFunction public: static constexpr auto name = "currentUser"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getClientInfo().initial_user); } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 9d8b98aab41..cbad9dc1ce4 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -47,7 +47,7 @@ class FunctionDateDiff : public IFunction using ColumnDateTime64 = ColumnDecimal; public: static constexpr auto name = "dateDiff"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index d90586d6703..7952eda4f2b 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -25,9 +25,9 @@ class FunctionDateTrunc : public IFunction public: static constexpr auto name = "date_trunc"; - explicit FunctionDateTrunc(ContextConstPtr context_) : context(context_) {} + explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {} - static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context); } + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } String getName() const override { return name; } @@ -146,7 +146,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second; }; diff --git a/src/Functions/defaultValueOfArgumentType.cpp b/src/Functions/defaultValueOfArgumentType.cpp index d7f6be1b55f..c0abe1b63d9 100644 --- a/src/Functions/defaultValueOfArgumentType.cpp +++ b/src/Functions/defaultValueOfArgumentType.cpp @@ -13,7 +13,7 @@ class FunctionDefaultValueOfArgumentType : public IFunction { public: static constexpr auto name = "defaultValueOfArgumentType"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/defaultValueOfTypeName.cpp b/src/Functions/defaultValueOfTypeName.cpp index fec72067069..1bf978ab17b 100644 --- a/src/Functions/defaultValueOfTypeName.cpp +++ b/src/Functions/defaultValueOfTypeName.cpp @@ -21,7 +21,7 @@ class FunctionDefaultValueOfTypeName : public IFunction { public: static constexpr auto name = "defaultValueOfTypeName"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/demange.cpp b/src/Functions/demange.cpp index b2df742b30a..755a50dccbb 100644 --- a/src/Functions/demange.cpp +++ b/src/Functions/demange.cpp @@ -25,7 +25,7 @@ class FunctionDemangle : public IFunction { public: static constexpr auto name = "demangle"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { context->checkAccess(AccessType::demangle); return std::make_shared(); diff --git a/src/Functions/dumpColumnStructure.cpp b/src/Functions/dumpColumnStructure.cpp index 28e1b874f3e..3189e343beb 100644 --- a/src/Functions/dumpColumnStructure.cpp +++ b/src/Functions/dumpColumnStructure.cpp @@ -14,7 +14,7 @@ class FunctionDumpColumnStructure : public IFunction { public: static constexpr auto name = "dumpColumnStructure"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp index 617731ba78c..d8e8e0f0d29 100644 --- a/src/Functions/errorCodeToName.cpp +++ b/src/Functions/errorCodeToName.cpp @@ -19,7 +19,7 @@ class FunctionErrorCodeToName : public IFunction { public: static constexpr auto name = "errorCodeToName"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/evalMLMethod.cpp b/src/Functions/evalMLMethod.cpp index 1ad5d10df92..a69a6f0f550 100644 --- a/src/Functions/evalMLMethod.cpp +++ b/src/Functions/evalMLMethod.cpp @@ -28,11 +28,11 @@ class FunctionEvalMLMethod : public IFunction { public: static constexpr auto name = "evalMLMethod"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionEvalMLMethod(ContextConstPtr context_) : context(context_) + explicit FunctionEvalMLMethod(ContextPtr context_) : context(context_) {} String getName() const override @@ -81,7 +81,7 @@ public: return agg_function->predictValues(arguments, context); } - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index 866a900f6a7..864a788cf18 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -51,7 +51,7 @@ public: static constexpr auto Kind = Impl::Kind; static constexpr auto name = Impl::Name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index 981382828bd..f1d728ee3f6 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -31,7 +31,7 @@ class FunctionExtractGroups : public IFunction { public: static constexpr auto name = "extractGroups"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index aa7c208904e..6a7bdbeaba9 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -296,7 +296,7 @@ class FunctionExtractTextFromHTML : public IFunction public: static constexpr auto name = "extractTextFromHTML"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/filesystem.cpp b/src/Functions/filesystem.cpp index d60b2133813..d264c972656 100644 --- a/src/Functions/filesystem.cpp +++ b/src/Functions/filesystem.cpp @@ -34,7 +34,7 @@ class FilesystemImpl : public IFunction public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared>(std::filesystem::space(context->getConfigRef().getString("path"))); } diff --git a/src/Functions/finalizeAggregation.cpp b/src/Functions/finalizeAggregation.cpp index c8379e6a66f..b3dfbc0aa15 100644 --- a/src/Functions/finalizeAggregation.cpp +++ b/src/Functions/finalizeAggregation.cpp @@ -24,7 +24,7 @@ class FunctionFinalizeAggregation : public IFunction { public: static constexpr auto name = "finalizeAggregation"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index c64c182baf6..d16985b5b34 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -281,7 +281,7 @@ private: public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/formatReadable.h b/src/Functions/formatReadable.h index fc8f5afe764..6cf4fadbf05 100644 --- a/src/Functions/formatReadable.h +++ b/src/Functions/formatReadable.h @@ -28,7 +28,7 @@ class FunctionFormatReadable : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { diff --git a/src/Functions/formatReadableTimeDelta.cpp b/src/Functions/formatReadableTimeDelta.cpp index 1be6fdf36c5..69dbaa71041 100644 --- a/src/Functions/formatReadableTimeDelta.cpp +++ b/src/Functions/formatReadableTimeDelta.cpp @@ -38,7 +38,7 @@ class FunctionFormatReadableTimeDelta : public IFunction { public: static constexpr auto name = "formatReadableTimeDelta"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 60ed6be1351..2d7739a02d6 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -35,7 +35,7 @@ class FunctionFormatRow : public IFunction public: static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; - FunctionFormatRow(const String & format_name_, ContextConstPtr context_) : format_name(format_name_), context(context_) + FunctionFormatRow(const String & format_name_, ContextPtr context_) : format_name(format_name_), context(context_) { if (!FormatFactory::instance().getAllFormats().count(format_name)) throw Exception("Unknown format " + format_name, ErrorCodes::UNKNOWN_FORMAT); @@ -76,7 +76,7 @@ public: private: String format_name; - ContextConstPtr context; + ContextPtr context; }; template @@ -84,8 +84,8 @@ class FormatRowOverloadResolver : public IFunctionOverloadResolver { public: static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; - static FunctionOverloadResolverPtr create(ContextConstPtr context) { return std::make_unique(context); } - explicit FormatRowOverloadResolver(ContextConstPtr context_) : context(context_) { } + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + explicit FormatRowOverloadResolver(ContextPtr context_) : context(context_) { } String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -111,7 +111,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared(); } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/formatString.cpp b/src/Functions/formatString.cpp index 9c222d085c2..4a3b04aaa05 100644 --- a/src/Functions/formatString.cpp +++ b/src/Functions/formatString.cpp @@ -31,7 +31,7 @@ class FormatFunction : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index 277d876e42f..bdd8b9b6bd4 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -151,7 +151,7 @@ namespace DB public: static constexpr auto name = Name::name; - static FunctionOverloadResolverPtr create(ContextConstPtr) + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique>(); } diff --git a/src/Functions/fromUnixTimestamp64Micro.cpp b/src/Functions/fromUnixTimestamp64Micro.cpp index 7eb9e41474b..1fd98e6e673 100644 --- a/src/Functions/fromUnixTimestamp64Micro.cpp +++ b/src/Functions/fromUnixTimestamp64Micro.cpp @@ -7,7 +7,7 @@ namespace DB void registerFromUnixTimestamp64Micro(FunctionFactory & factory) { factory.registerFunction("fromUnixTimestamp64Micro", - [](ContextConstPtr){ return std::make_unique( + [](ContextPtr){ return std::make_unique( std::make_shared(6, "fromUnixTimestamp64Micro")); }); } diff --git a/src/Functions/fromUnixTimestamp64Milli.cpp b/src/Functions/fromUnixTimestamp64Milli.cpp index 68ee1ed4078..90f18699e5d 100644 --- a/src/Functions/fromUnixTimestamp64Milli.cpp +++ b/src/Functions/fromUnixTimestamp64Milli.cpp @@ -7,7 +7,7 @@ namespace DB void registerFromUnixTimestamp64Milli(FunctionFactory & factory) { factory.registerFunction("fromUnixTimestamp64Milli", - [](ContextConstPtr){ return std::make_unique( + [](ContextPtr){ return std::make_unique( std::make_shared(3, "fromUnixTimestamp64Milli")); }); } diff --git a/src/Functions/fromUnixTimestamp64Nano.cpp b/src/Functions/fromUnixTimestamp64Nano.cpp index 19fe8524b78..c489b48fb6e 100644 --- a/src/Functions/fromUnixTimestamp64Nano.cpp +++ b/src/Functions/fromUnixTimestamp64Nano.cpp @@ -7,7 +7,7 @@ namespace DB void registerFromUnixTimestamp64Nano(FunctionFactory & factory) { factory.registerFunction("fromUnixTimestamp64Nano", - [](ContextConstPtr){ return std::make_unique( + [](ContextPtr){ return std::make_unique( std::make_shared(9, "fromUnixTimestamp64Nano")); }); } diff --git a/src/Functions/fuzzBits.cpp b/src/Functions/fuzzBits.cpp index b4110eae32a..0884f586082 100644 --- a/src/Functions/fuzzBits.cpp +++ b/src/Functions/fuzzBits.cpp @@ -53,7 +53,7 @@ class FunctionFuzzBits : public IFunction public: static constexpr auto name = "fuzzBits"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 7d3f80d9096..7ca127857df 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -60,7 +60,7 @@ public: class FunctionGenerateUUIDv4 : public TargetSpecific::Default::FunctionGenerateUUIDv4 { public: - explicit FunctionGenerateUUIDv4(ContextConstPtr context) : selector(context) + explicit FunctionGenerateUUIDv4(ContextPtr context) : selector(context) { selector.registerImplementation(); @@ -76,7 +76,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 45e6e06d114..6030cbeea65 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -33,7 +33,7 @@ class FunctionGeoToH3 : public IFunction public: static constexpr auto name = "geoToH3"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/geohashDecode.cpp b/src/Functions/geohashDecode.cpp index 89e8d40402a..bf774905868 100644 --- a/src/Functions/geohashDecode.cpp +++ b/src/Functions/geohashDecode.cpp @@ -27,7 +27,7 @@ class FunctionGeohashDecode : public IFunction { public: static constexpr auto name = "geohashDecode"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/geohashEncode.cpp b/src/Functions/geohashEncode.cpp index fbee2047ed2..15abe96e3e5 100644 --- a/src/Functions/geohashEncode.cpp +++ b/src/Functions/geohashEncode.cpp @@ -28,7 +28,7 @@ class FunctionGeohashEncode : public IFunction { public: static constexpr auto name = "geohashEncode"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/geohashesInBox.cpp b/src/Functions/geohashesInBox.cpp index fdccc97879f..c0629ab9655 100644 --- a/src/Functions/geohashesInBox.cpp +++ b/src/Functions/geohashesInBox.cpp @@ -29,7 +29,7 @@ class FunctionGeohashesInBox : public IFunction { public: static constexpr auto name = "geohashesInBox"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/getMacro.cpp b/src/Functions/getMacro.cpp index a2fda60623f..c869685af42 100644 --- a/src/Functions/getMacro.cpp +++ b/src/Functions/getMacro.cpp @@ -30,7 +30,7 @@ private: public: static constexpr auto name = "getMacro"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getMacros()); } diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index d74d881473f..a29abd257e7 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -20,16 +20,16 @@ namespace /** Get scalar value of sub queries from query context via IAST::Hash. */ -class FunctionGetScalar : public IFunction, WithConstContext +class FunctionGetScalar : public IFunction, WithContext { public: static constexpr auto name = "__getScalar"; - static FunctionPtr create(ContextConstPtr context_) + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - explicit FunctionGetScalar(ContextConstPtr context_) : WithConstContext(context_) {} + explicit FunctionGetScalar(ContextPtr context_) : WithContext(context_) {} String getName() const override { @@ -46,7 +46,7 @@ public: if (arguments.size() != 1 || !isString(arguments[0].type) || !arguments[0].column || !isColumnConst(*arguments[0].column)) throw Exception("Function " + getName() + " accepts one const string argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto scalar_name = assert_cast(*arguments[0].column).getValue(); - ContextConstPtr query_context = getContext()->hasQueryContext() ? getContext()->getQueryContext() : getContext(); + ContextPtr query_context = getContext()->hasQueryContext() ? getContext()->getQueryContext() : getContext(); scalar = query_context->getScalar(scalar_name).getByPosition(0); return scalar.type; } diff --git a/src/Functions/getSetting.cpp b/src/Functions/getSetting.cpp index 7c6592faf57..0206de33125 100644 --- a/src/Functions/getSetting.cpp +++ b/src/Functions/getSetting.cpp @@ -19,13 +19,13 @@ namespace { /// Get the value of a setting. -class FunctionGetSetting : public IFunction, WithConstContext +class FunctionGetSetting : public IFunction, WithContext { public: static constexpr auto name = "getSetting"; - static FunctionPtr create(ContextConstPtr context_) { return std::make_shared(context_); } - explicit FunctionGetSetting(ContextConstPtr context_) : WithConstContext(context_) {} + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } + explicit FunctionGetSetting(ContextPtr context_) : WithContext(context_) {} String getName() const override { return name; } bool isDeterministic() const override { return false; } diff --git a/src/Functions/getSizeOfEnumType.cpp b/src/Functions/getSizeOfEnumType.cpp index 61f4d7efced..504c088a737 100644 --- a/src/Functions/getSizeOfEnumType.cpp +++ b/src/Functions/getSizeOfEnumType.cpp @@ -20,7 +20,7 @@ class FunctionGetSizeOfEnumType : public IFunction { public: static constexpr auto name = "getSizeOfEnumType"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/globalVariable.cpp b/src/Functions/globalVariable.cpp index be2d4488daa..c141ceb2692 100644 --- a/src/Functions/globalVariable.cpp +++ b/src/Functions/globalVariable.cpp @@ -30,7 +30,7 @@ class FunctionGlobalVariable : public IFunction { public: static constexpr auto name = "globalVariable"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index e160b72740e..756b76859ae 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -287,7 +287,7 @@ template class FunctionGeoDistance : public TargetSpecific::Default::FunctionGeoDistance { public: - explicit FunctionGeoDistance(ContextConstPtr context) : selector(context) + explicit FunctionGeoDistance(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -307,7 +307,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared>(context); } diff --git a/src/Functions/h3EdgeAngle.cpp b/src/Functions/h3EdgeAngle.cpp index 4a0f69ab139..ad8d1ee441f 100644 --- a/src/Functions/h3EdgeAngle.cpp +++ b/src/Functions/h3EdgeAngle.cpp @@ -32,7 +32,7 @@ class FunctionH3EdgeAngle : public IFunction public: static constexpr auto name = "h3EdgeAngle"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3EdgeLengthM.cpp b/src/Functions/h3EdgeLengthM.cpp index 523c4bde061..85d307e1da3 100644 --- a/src/Functions/h3EdgeLengthM.cpp +++ b/src/Functions/h3EdgeLengthM.cpp @@ -37,7 +37,7 @@ class FunctionH3EdgeLengthM : public IFunction public: static constexpr auto name = "h3EdgeLengthM"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3GetBaseCell.cpp b/src/Functions/h3GetBaseCell.cpp index 83d3064a023..fcad82098e6 100644 --- a/src/Functions/h3GetBaseCell.cpp +++ b/src/Functions/h3GetBaseCell.cpp @@ -29,7 +29,7 @@ class FunctionH3GetBaseCell : public IFunction public: static constexpr auto name = "h3GetBaseCell"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3GetResolution.cpp b/src/Functions/h3GetResolution.cpp index 197b0516da9..d1bfd899150 100644 --- a/src/Functions/h3GetResolution.cpp +++ b/src/Functions/h3GetResolution.cpp @@ -29,7 +29,7 @@ class FunctionH3GetResolution : public IFunction public: static constexpr auto name = "h3GetResolution"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3HexAreaM2.cpp b/src/Functions/h3HexAreaM2.cpp index c7eca3396bc..e44bf660333 100644 --- a/src/Functions/h3HexAreaM2.cpp +++ b/src/Functions/h3HexAreaM2.cpp @@ -32,7 +32,7 @@ class FunctionH3HexAreaM2 : public IFunction public: static constexpr auto name = "h3HexAreaM2"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3IndexesAreNeighbors.cpp b/src/Functions/h3IndexesAreNeighbors.cpp index d2873f954e8..56c69ed0a8b 100644 --- a/src/Functions/h3IndexesAreNeighbors.cpp +++ b/src/Functions/h3IndexesAreNeighbors.cpp @@ -29,7 +29,7 @@ class FunctionH3IndexesAreNeighbors : public IFunction public: static constexpr auto name = "h3IndexesAreNeighbors"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3IsValid.cpp b/src/Functions/h3IsValid.cpp index 930b1d3c4a7..1e4840b3537 100644 --- a/src/Functions/h3IsValid.cpp +++ b/src/Functions/h3IsValid.cpp @@ -29,7 +29,7 @@ class FunctionH3IsValid : public IFunction public: static constexpr auto name = "h3IsValid"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3ToChildren.cpp b/src/Functions/h3ToChildren.cpp index 3b09b5a1866..9730aefe3d4 100644 --- a/src/Functions/h3ToChildren.cpp +++ b/src/Functions/h3ToChildren.cpp @@ -38,7 +38,7 @@ class FunctionH3ToChildren : public IFunction public: static constexpr auto name = "h3ToChildren"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3ToParent.cpp b/src/Functions/h3ToParent.cpp index b5c49d4c253..4baf9527301 100644 --- a/src/Functions/h3ToParent.cpp +++ b/src/Functions/h3ToParent.cpp @@ -32,7 +32,7 @@ class FunctionH3ToParent : public IFunction public: static constexpr auto name = "h3ToParent"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3ToString.cpp b/src/Functions/h3ToString.cpp index f2a777580d0..dcd0951f67f 100644 --- a/src/Functions/h3ToString.cpp +++ b/src/Functions/h3ToString.cpp @@ -29,7 +29,7 @@ class FunctionH3ToString : public IFunction public: static constexpr auto name = "h3ToString"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/h3kRing.cpp b/src/Functions/h3kRing.cpp index e616c253c58..40e73f9dea2 100644 --- a/src/Functions/h3kRing.cpp +++ b/src/Functions/h3kRing.cpp @@ -35,7 +35,7 @@ class FunctionH3KRing : public IFunction public: static constexpr auto name = "h3kRing"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 941ffa48e74..0fa0562389b 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -25,16 +25,16 @@ namespace /** Usage: * hasColumnInTable(['hostname'[, 'username'[, 'password']],] 'database', 'table', 'column') */ -class FunctionHasColumnInTable : public IFunction, WithConstContext +class FunctionHasColumnInTable : public IFunction, WithContext { public: static constexpr auto name = "hasColumnInTable"; - static FunctionPtr create(ContextConstPtr context_) + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_->getGlobalContext()); } - explicit FunctionHasColumnInTable(ContextConstPtr global_context_) : WithConstContext(global_context_) + explicit FunctionHasColumnInTable(ContextPtr global_context_) : WithContext(global_context_) { } diff --git a/src/Functions/hasThreadFuzzer.cpp b/src/Functions/hasThreadFuzzer.cpp index fb7014a49b1..e9b48aa70f9 100644 --- a/src/Functions/hasThreadFuzzer.cpp +++ b/src/Functions/hasThreadFuzzer.cpp @@ -16,7 +16,7 @@ class FunctionHasThreadFuzzer : public IFunction { public: static constexpr auto name = "hasThreadFuzzer"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/hostName.cpp b/src/Functions/hostName.cpp index 1fdc039be96..0aba155bb36 100644 --- a/src/Functions/hostName.cpp +++ b/src/Functions/hostName.cpp @@ -15,7 +15,7 @@ class FunctionHostName : public IFunction { public: static constexpr auto name = "hostName"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/identity.cpp b/src/Functions/identity.cpp index 7889a17f05a..d15d9e1f710 100644 --- a/src/Functions/identity.cpp +++ b/src/Functions/identity.cpp @@ -11,7 +11,7 @@ class FunctionIdentity : public IFunction { public: static constexpr auto name = "identity"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index dbe36d4c728..ec3447ffb81 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -173,7 +173,7 @@ class FunctionIf : public FunctionIfBase { public: static constexpr auto name = "if"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: template diff --git a/src/Functions/ifNotFinite.cpp b/src/Functions/ifNotFinite.cpp index be975f28ee1..a5e3131117e 100644 --- a/src/Functions/ifNotFinite.cpp +++ b/src/Functions/ifNotFinite.cpp @@ -17,9 +17,9 @@ class FunctionIfNotFinite : public IFunction public: static constexpr auto name = "ifNotFinite"; - explicit FunctionIfNotFinite(ContextConstPtr context_) : context(context_) {} + explicit FunctionIfNotFinite(ContextPtr context_) : context(context_) {} - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -59,7 +59,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/ifNull.cpp b/src/Functions/ifNull.cpp index 69c7e9346f2..82b63f4dbda 100644 --- a/src/Functions/ifNull.cpp +++ b/src/Functions/ifNull.cpp @@ -21,9 +21,9 @@ class FunctionIfNull : public IFunction public: static constexpr auto name = "ifNull"; - explicit FunctionIfNull(ContextConstPtr context_) : context(context_) {} + explicit FunctionIfNull(ContextPtr context_) : context(context_) {} - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -83,7 +83,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/ignore.cpp b/src/Functions/ignore.cpp index 819cd9a0b52..176bc9afee2 100644 --- a/src/Functions/ignore.cpp +++ b/src/Functions/ignore.cpp @@ -14,7 +14,7 @@ class FunctionIgnore : public IFunction { public: static constexpr auto name = "ignore"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index b1686e2a15a..827e0212396 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -52,7 +52,7 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/indexHint.cpp b/src/Functions/indexHint.cpp index 6fd41a910d3..f3c856c38ce 100644 --- a/src/Functions/indexHint.cpp +++ b/src/Functions/indexHint.cpp @@ -23,7 +23,7 @@ class FunctionIndexHint : public IFunction { public: static constexpr auto name = "indexHint"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/initializeAggregation.cpp b/src/Functions/initializeAggregation.cpp index e3d79c41f5b..aea97a48037 100644 --- a/src/Functions/initializeAggregation.cpp +++ b/src/Functions/initializeAggregation.cpp @@ -29,7 +29,7 @@ class FunctionInitializeAggregation : public IFunction { public: static constexpr auto name = "initializeAggregation"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/isConstant.cpp b/src/Functions/isConstant.cpp index 474fb46eed3..3069ec79ae1 100644 --- a/src/Functions/isConstant.cpp +++ b/src/Functions/isConstant.cpp @@ -15,7 +15,7 @@ class FunctionIsConstant : public IFunction { public: static constexpr auto name = "isConstant"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/isDecimalOverflow.cpp b/src/Functions/isDecimalOverflow.cpp index c82845b7cb8..d409afaf234 100644 --- a/src/Functions/isDecimalOverflow.cpp +++ b/src/Functions/isDecimalOverflow.cpp @@ -28,7 +28,7 @@ class FunctionIsDecimalOverflow : public IFunction public: static constexpr auto name = "isDecimalOverflow"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 439bf0a893e..0886ef55e7b 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -125,7 +125,7 @@ namespace DB public: static constexpr auto name = "isIPAddressInRange"; String getName() const override { return name; } - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* return_type */, size_t input_rows_count) const override { diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index a0119bb48c2..81c870a6303 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -19,7 +19,7 @@ class FunctionIsNotNull : public IFunction public: static constexpr auto name = "isNotNull"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index 36da85dcdd3..de840dab2bf 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -18,7 +18,7 @@ class FunctionIsNull : public IFunction public: static constexpr auto name = "isNull"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/isZeroOrNull.cpp b/src/Functions/isZeroOrNull.cpp index 4b863871629..a2a06af7569 100644 --- a/src/Functions/isZeroOrNull.cpp +++ b/src/Functions/isZeroOrNull.cpp @@ -26,7 +26,7 @@ class FunctionIsZeroOrNull : public IFunction public: static constexpr auto name = "isZeroOrNull"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/logTrace.cpp b/src/Functions/logTrace.cpp index a291a59504f..2ccc4d2ffd2 100644 --- a/src/Functions/logTrace.cpp +++ b/src/Functions/logTrace.cpp @@ -21,7 +21,7 @@ namespace { public: static constexpr auto name = "logTrace"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/lowCardinalityIndices.cpp b/src/Functions/lowCardinalityIndices.cpp index ac7fd45e7b6..580e7381955 100644 --- a/src/Functions/lowCardinalityIndices.cpp +++ b/src/Functions/lowCardinalityIndices.cpp @@ -21,7 +21,7 @@ class FunctionLowCardinalityIndices: public IFunction { public: static constexpr auto name = "lowCardinalityIndices"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/lowCardinalityKeys.cpp b/src/Functions/lowCardinalityKeys.cpp index 2781248d15a..46128267871 100644 --- a/src/Functions/lowCardinalityKeys.cpp +++ b/src/Functions/lowCardinalityKeys.cpp @@ -19,7 +19,7 @@ class FunctionLowCardinalityKeys: public IFunction { public: static constexpr auto name = "lowCardinalityKeys"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 76b57d596cc..f1b08dc75b7 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -34,7 +34,7 @@ class FunctionMap : public IFunction public: static constexpr auto name = "map"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -146,7 +146,7 @@ class FunctionMapContains : public IFunction { public: static constexpr auto name = NameMapContains::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -209,7 +209,7 @@ class FunctionMapKeys : public IFunction { public: static constexpr auto name = "mapKeys"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -256,7 +256,7 @@ class FunctionMapValues : public IFunction { public: static constexpr auto name = "mapValues"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/materialize.h b/src/Functions/materialize.h index 99b556dfff5..4676b263f39 100644 --- a/src/Functions/materialize.h +++ b/src/Functions/materialize.h @@ -11,7 +11,7 @@ class FunctionMaterialize : public IFunction { public: static constexpr auto name = "materialize"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 56399a7f056..92f05df472b 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -35,7 +35,7 @@ class FunctionMultiIf final : public FunctionIfBase { public: static constexpr auto name = "multiIf"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } diff --git a/src/Functions/neighbor.cpp b/src/Functions/neighbor.cpp index 53c47d3972d..785c5817176 100644 --- a/src/Functions/neighbor.cpp +++ b/src/Functions/neighbor.cpp @@ -31,7 +31,7 @@ class FunctionNeighbor : public IFunction { public: static constexpr auto name = "neighbor"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } /// Get the name of the function. String getName() const override { return name; } diff --git a/src/Functions/normalizedQueryHash.cpp b/src/Functions/normalizedQueryHash.cpp index 5cfd5ff4904..e08680518c1 100644 --- a/src/Functions/normalizedQueryHash.cpp +++ b/src/Functions/normalizedQueryHash.cpp @@ -51,7 +51,7 @@ class FunctionNormalizedQueryHash : public IFunction { public: static constexpr auto name = keep_names ? "normalizedQueryHashKeepNames" : "normalizedQueryHash"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index 9d6569fd5df..673484b2ad2 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -83,7 +83,7 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextConstPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index b31a78bf0a7..cccfb9de6b6 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -107,7 +107,7 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextConstPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { diff --git a/src/Functions/nullIf.cpp b/src/Functions/nullIf.cpp index 8f69ef4f550..14f2f72fe61 100644 --- a/src/Functions/nullIf.cpp +++ b/src/Functions/nullIf.cpp @@ -18,16 +18,16 @@ namespace class FunctionNullIf : public IFunction { private: - ContextConstPtr context; + ContextPtr context; public: static constexpr auto name = "nullIf"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionNullIf(ContextConstPtr context_) : context(context_) {} + explicit FunctionNullIf(ContextPtr context_) : context(context_) {} std::string getName() const override { diff --git a/src/Functions/partitionId.cpp b/src/Functions/partitionId.cpp index f33467e6752..cf679452da1 100644 --- a/src/Functions/partitionId.cpp +++ b/src/Functions/partitionId.cpp @@ -23,7 +23,7 @@ class FunctionPartitionId : public IFunction public: static constexpr auto name = "partitionId"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index 06eae802864..f7bfc24559c 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -39,7 +39,7 @@ class FunctionPointInEllipses : public IFunction { public: static constexpr auto name = "pointInEllipses"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index 82e69c468cc..842a5010190 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -60,7 +60,7 @@ public: explicit FunctionPointInPolygon(bool validate_) : validate(validate_) {} - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared>( context->getSettingsRef().validate_polygons); diff --git a/src/Functions/polygonArea.cpp b/src/Functions/polygonArea.cpp index 07e0f922ae0..9a36b71d5b1 100644 --- a/src/Functions/polygonArea.cpp +++ b/src/Functions/polygonArea.cpp @@ -34,7 +34,7 @@ public: explicit FunctionPolygonArea() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonConvexHull.cpp b/src/Functions/polygonConvexHull.cpp index b84e327ac81..5bce3c5bbdd 100644 --- a/src/Functions/polygonConvexHull.cpp +++ b/src/Functions/polygonConvexHull.cpp @@ -33,7 +33,7 @@ public: explicit FunctionPolygonConvexHull() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonPerimeter.cpp b/src/Functions/polygonPerimeter.cpp index 11bb5488fd9..5a6e293630e 100644 --- a/src/Functions/polygonPerimeter.cpp +++ b/src/Functions/polygonPerimeter.cpp @@ -33,7 +33,7 @@ public: explicit FunctionPolygonPerimeter() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonsDistance.cpp b/src/Functions/polygonsDistance.cpp index d92a1931f2b..d6c4ff36e4d 100644 --- a/src/Functions/polygonsDistance.cpp +++ b/src/Functions/polygonsDistance.cpp @@ -35,7 +35,7 @@ public: explicit FunctionPolygonsDistance() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonsEquals.cpp b/src/Functions/polygonsEquals.cpp index c14f9a18980..9e2902d0528 100644 --- a/src/Functions/polygonsEquals.cpp +++ b/src/Functions/polygonsEquals.cpp @@ -34,7 +34,7 @@ public: explicit FunctionPolygonsEquals() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonsIntersection.cpp b/src/Functions/polygonsIntersection.cpp index 5676323a08a..699488abab1 100644 --- a/src/Functions/polygonsIntersection.cpp +++ b/src/Functions/polygonsIntersection.cpp @@ -34,7 +34,7 @@ public: explicit FunctionPolygonsIntersection() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index 991d66a2802..129e553f060 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -34,7 +34,7 @@ public: explicit FunctionPolygonsSymDifference() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 9a9bfc9acfe..e7281c299d0 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -34,7 +34,7 @@ public: explicit FunctionPolygonsUnion() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index 45fc06d4fbe..2fd5b7e2225 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -36,7 +36,7 @@ public: explicit FunctionPolygonsWithin() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 13a07050717..77d8e1f2a33 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -79,7 +79,7 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextConstPtr) + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique>(); } diff --git a/src/Functions/randomFixedString.cpp b/src/Functions/randomFixedString.cpp index 2a60f8dd88d..13996eee677 100644 --- a/src/Functions/randomFixedString.cpp +++ b/src/Functions/randomFixedString.cpp @@ -77,7 +77,7 @@ public: class FunctionRandomFixedString : public FunctionRandomFixedStringImpl { public: - explicit FunctionRandomFixedString(ContextConstPtr context) : selector(context) + explicit FunctionRandomFixedString(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -93,7 +93,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } diff --git a/src/Functions/randomPrintableASCII.cpp b/src/Functions/randomPrintableASCII.cpp index d27772e28fb..ba8c16aa689 100644 --- a/src/Functions/randomPrintableASCII.cpp +++ b/src/Functions/randomPrintableASCII.cpp @@ -27,7 +27,7 @@ class FunctionRandomPrintableASCII : public IFunction { public: static constexpr auto name = "randomPrintableASCII"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/randomString.cpp b/src/Functions/randomString.cpp index 680b01b9d08..10795626326 100644 --- a/src/Functions/randomString.cpp +++ b/src/Functions/randomString.cpp @@ -97,7 +97,7 @@ public: class FunctionRandomString : public FunctionRandomStringImpl { public: - explicit FunctionRandomString(ContextConstPtr context) : selector(context) + explicit FunctionRandomString(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -113,7 +113,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } diff --git a/src/Functions/randomStringUTF8.cpp b/src/Functions/randomStringUTF8.cpp index 48c828d2444..018c591a1fe 100644 --- a/src/Functions/randomStringUTF8.cpp +++ b/src/Functions/randomStringUTF8.cpp @@ -30,7 +30,7 @@ class FunctionRandomStringUTF8 : public IFunction public: static constexpr auto name = "randomStringUTF8"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index 26bb13a2e93..2ce4becb01c 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -68,7 +68,7 @@ public: return true; } - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } diff --git a/src/Functions/regexpQuoteMeta.cpp b/src/Functions/regexpQuoteMeta.cpp index cae2fdc1323..0f1ec476a3b 100644 --- a/src/Functions/regexpQuoteMeta.cpp +++ b/src/Functions/regexpQuoteMeta.cpp @@ -21,7 +21,7 @@ class FunctionRegexpQuoteMeta : public IFunction public: static constexpr auto name = "regexpQuoteMeta"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index d007e479c04..94c9eec5868 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -47,7 +47,7 @@ class FunctionReinterpret : public IFunction public: static constexpr auto name = "reinterpret"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -349,7 +349,7 @@ class FunctionReinterpretAs : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index e87a51b807d..423ed53c53f 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -173,7 +173,7 @@ class FunctionRepeat : public IFunction public: static constexpr auto name = "repeat"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/replicate.h b/src/Functions/replicate.h index 4493d132002..71b39e9e60e 100644 --- a/src/Functions/replicate.h +++ b/src/Functions/replicate.h @@ -13,7 +13,7 @@ class FunctionReplicate : public IFunction public: static constexpr auto name = "replicate"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index 49ce3a214e3..f9701af75da 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -57,7 +57,7 @@ class FunctionReverse : public IFunction { public: static constexpr auto name = "reverse"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -117,9 +117,9 @@ class ReverseOverloadResolver : public IFunctionOverloadResolver { public: static constexpr auto name = "reverse"; - static FunctionOverloadResolverPtr create(ContextConstPtr context) { return std::make_unique(context); } + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } - explicit ReverseOverloadResolver(ContextConstPtr context_) : context(context_) {} + explicit ReverseOverloadResolver(ContextPtr context_) : context(context_) {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -141,7 +141,7 @@ public: } private: - ContextConstPtr context; + ContextPtr context; }; } diff --git a/src/Functions/rowNumberInAllBlocks.cpp b/src/Functions/rowNumberInAllBlocks.cpp index 85211d9e740..9c358aec8f0 100644 --- a/src/Functions/rowNumberInAllBlocks.cpp +++ b/src/Functions/rowNumberInAllBlocks.cpp @@ -18,7 +18,7 @@ private: public: static constexpr auto name = "rowNumberInAllBlocks"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/rowNumberInBlock.cpp b/src/Functions/rowNumberInBlock.cpp index e924207a6d1..48fa472e1dd 100644 --- a/src/Functions/rowNumberInBlock.cpp +++ b/src/Functions/rowNumberInBlock.cpp @@ -13,7 +13,7 @@ class FunctionRowNumberInBlock : public IFunction { public: static constexpr auto name = "rowNumberInBlock"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/runningAccumulate.cpp b/src/Functions/runningAccumulate.cpp index b17faf9c031..1eab573b046 100644 --- a/src/Functions/runningAccumulate.cpp +++ b/src/Functions/runningAccumulate.cpp @@ -33,7 +33,7 @@ class FunctionRunningAccumulate : public IFunction { public: static constexpr auto name = "runningAccumulate"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/runningConcurrency.cpp b/src/Functions/runningConcurrency.cpp index 6f842bcc342..022e2be5f6c 100644 --- a/src/Functions/runningConcurrency.cpp +++ b/src/Functions/runningConcurrency.cpp @@ -154,7 +154,7 @@ namespace DB public: static constexpr auto name = Name::name; - static FunctionOverloadResolverPtr create(ContextConstPtr) + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique>(); } diff --git a/src/Functions/runningDifference.h b/src/Functions/runningDifference.h index 41a418b9b58..f87c57af043 100644 --- a/src/Functions/runningDifference.h +++ b/src/Functions/runningDifference.h @@ -124,7 +124,7 @@ private: public: static constexpr auto name = FunctionRunningDifferenceName::name; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 16b3973cdc2..c0aad0b3820 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -36,7 +36,7 @@ class FunctionSleep : public IFunction { public: static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } diff --git a/src/Functions/stringToH3.cpp b/src/Functions/stringToH3.cpp index 97f9ba3f2ab..9b02711db50 100644 --- a/src/Functions/stringToH3.cpp +++ b/src/Functions/stringToH3.cpp @@ -34,7 +34,7 @@ class FunctionStringToH3 : public IFunction public: static constexpr auto name = "stringToH3"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index 13f0e296602..5ce75035475 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -36,7 +36,7 @@ class FunctionSubstring : public IFunction { public: static constexpr auto name = is_utf8 ? "substringUTF8" : "substring"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/svg.cpp b/src/Functions/svg.cpp index 4d0edaf004b..39473816c8c 100644 --- a/src/Functions/svg.cpp +++ b/src/Functions/svg.cpp @@ -24,7 +24,7 @@ public: explicit FunctionSvg() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/tcpPort.cpp b/src/Functions/tcpPort.cpp index baacdb0a335..484843ced3f 100644 --- a/src/Functions/tcpPort.cpp +++ b/src/Functions/tcpPort.cpp @@ -14,7 +14,7 @@ class FunctionTcpPort : public IFunction public: static constexpr auto name = "tcpPort"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getTCPPort()); } diff --git a/src/Functions/throwIf.cpp b/src/Functions/throwIf.cpp index a3285f64c73..1b56cc0d188 100644 --- a/src/Functions/throwIf.cpp +++ b/src/Functions/throwIf.cpp @@ -26,7 +26,7 @@ class FunctionThrowIf : public IFunction { public: static constexpr auto name = "throwIf"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/tid.cpp b/src/Functions/tid.cpp index 6f2eb3cc595..404eff862b3 100644 --- a/src/Functions/tid.cpp +++ b/src/Functions/tid.cpp @@ -13,7 +13,7 @@ namespace { public: static constexpr auto name = "tid"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index 4f8882a83ad..b64d2687b05 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -109,7 +109,7 @@ class FunctionTimeSlots : public IFunction public: static constexpr auto name = "timeSlots"; static constexpr UInt32 TIME_SLOT_SIZE = 1800; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/timezone.cpp b/src/Functions/timezone.cpp index c0d136ab7d4..67f7462fc95 100644 --- a/src/Functions/timezone.cpp +++ b/src/Functions/timezone.cpp @@ -16,7 +16,7 @@ class FunctionTimezone : public IFunction { public: static constexpr auto name = "timezone"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/timezoneOf.cpp b/src/Functions/timezoneOf.cpp index f768722a284..3dc72424a4f 100644 --- a/src/Functions/timezoneOf.cpp +++ b/src/Functions/timezoneOf.cpp @@ -28,7 +28,7 @@ class FunctionTimezoneOf : public IFunction public: static constexpr auto name = "timezoneOf"; String getName() const override { return name; } - static FunctionPtr create(ContextConstPtr) { return std::make_unique(); } + static FunctionPtr create(ContextPtr) { return std::make_unique(); } size_t getNumberOfArguments() const override { return 1; } diff --git a/src/Functions/toColumnTypeName.cpp b/src/Functions/toColumnTypeName.cpp index 2d3635f4ed1..d64fa12604e 100644 --- a/src/Functions/toColumnTypeName.cpp +++ b/src/Functions/toColumnTypeName.cpp @@ -14,7 +14,7 @@ class FunctionToColumnTypeName : public IFunction { public: static constexpr auto name = "toColumnTypeName"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/toFixedString.h b/src/Functions/toFixedString.h index 303d51f41ad..fe437a24303 100644 --- a/src/Functions/toFixedString.h +++ b/src/Functions/toFixedString.h @@ -32,7 +32,7 @@ class FunctionToFixedString : public IFunction { public: static constexpr auto name = "toFixedString"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static FunctionPtr create() { return std::make_shared(); } String getName() const override diff --git a/src/Functions/toLowCardinality.cpp b/src/Functions/toLowCardinality.cpp index 8f102d5db9c..983e66d1007 100644 --- a/src/Functions/toLowCardinality.cpp +++ b/src/Functions/toLowCardinality.cpp @@ -14,7 +14,7 @@ class FunctionToLowCardinality: public IFunction { public: static constexpr auto name = "toLowCardinality"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/toModifiedJulianDay.cpp b/src/Functions/toModifiedJulianDay.cpp index c76b158baa8..a44979b52ff 100644 --- a/src/Functions/toModifiedJulianDay.cpp +++ b/src/Functions/toModifiedJulianDay.cpp @@ -172,7 +172,7 @@ namespace DB public: static constexpr auto name = Name::name; - static FunctionOverloadResolverPtr create(ContextConstPtr) + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique>(); } diff --git a/src/Functions/toNullable.cpp b/src/Functions/toNullable.cpp index 12d23e59f49..5e383893476 100644 --- a/src/Functions/toNullable.cpp +++ b/src/Functions/toNullable.cpp @@ -16,7 +16,7 @@ class FunctionToNullable : public IFunction public: static constexpr auto name = "toNullable"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 97d2d65b67a..ee592fbb1e3 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -184,7 +184,7 @@ namespace class FunctionToStartOfInterval : public IFunction { public: - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static constexpr auto name = function_name; String getName() const override { return name; } diff --git a/src/Functions/toTimezone.cpp b/src/Functions/toTimezone.cpp index ef80c2f0849..551e07a8354 100644 --- a/src/Functions/toTimezone.cpp +++ b/src/Functions/toTimezone.cpp @@ -25,7 +25,7 @@ class FunctionToTimezone : public IFunction { public: static constexpr auto name = "toTimezone"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/toTypeName.cpp b/src/Functions/toTypeName.cpp index f369ff1ccef..3c733fb3ea7 100644 --- a/src/Functions/toTypeName.cpp +++ b/src/Functions/toTypeName.cpp @@ -18,7 +18,7 @@ public: static constexpr auto name = "toTypeName"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/toUnixTimestamp64Micro.cpp b/src/Functions/toUnixTimestamp64Micro.cpp index bd13f753802..c5b841a1a81 100644 --- a/src/Functions/toUnixTimestamp64Micro.cpp +++ b/src/Functions/toUnixTimestamp64Micro.cpp @@ -7,7 +7,7 @@ namespace DB void registerToUnixTimestamp64Micro(FunctionFactory & factory) { factory.registerFunction("toUnixTimestamp64Micro", - [](ContextConstPtr){ return std::make_unique( + [](ContextPtr){ return std::make_unique( std::make_shared(6, "toUnixTimestamp64Micro")); }); } diff --git a/src/Functions/toUnixTimestamp64Milli.cpp b/src/Functions/toUnixTimestamp64Milli.cpp index 088cd2011c5..bfceb3708d3 100644 --- a/src/Functions/toUnixTimestamp64Milli.cpp +++ b/src/Functions/toUnixTimestamp64Milli.cpp @@ -7,7 +7,7 @@ namespace DB void registerToUnixTimestamp64Milli(FunctionFactory & factory) { factory.registerFunction("toUnixTimestamp64Milli", - [](ContextConstPtr){ return std::make_unique( + [](ContextPtr){ return std::make_unique( std::make_shared(3, "toUnixTimestamp64Milli")); }); } diff --git a/src/Functions/toUnixTimestamp64Nano.cpp b/src/Functions/toUnixTimestamp64Nano.cpp index 3961b9630ba..2256dc369b9 100644 --- a/src/Functions/toUnixTimestamp64Nano.cpp +++ b/src/Functions/toUnixTimestamp64Nano.cpp @@ -7,7 +7,7 @@ namespace DB void registerToUnixTimestamp64Nano(FunctionFactory & factory) { factory.registerFunction("toUnixTimestamp64Nano", - [](ContextConstPtr){ return std::make_unique( + [](ContextPtr){ return std::make_unique( std::make_shared(9, "toUnixTimestamp64Nano")); }); } diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index 9d04bd3a84d..43be4c4582a 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -71,7 +71,7 @@ public: size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextConstPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared(); } diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index e9305304ac0..1debc2cb6a0 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -58,7 +58,7 @@ class FunctionTransform : public IFunction { public: static constexpr auto name = "transform"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index c7f935bac3e..f946a7fabfb 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -32,16 +32,16 @@ namespace ErrorCodes class FunctionTrap : public IFunction { private: - ContextConstPtr context; + ContextPtr context; public: static constexpr auto name = "trap"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - FunctionTrap(ContextConstPtr context_) : context(context_) {} + FunctionTrap(ContextPtr context_) : context(context_) {} String getName() const override { diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 8c1ad90070e..dda034ee911 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -24,7 +24,7 @@ class FunctionTuple : public IFunction public: static constexpr auto name = "tuple"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 7f71e9d93e9..97e2f70fccf 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -30,7 +30,7 @@ class FunctionTupleElement : public IFunction { public: static constexpr auto name = "tupleElement"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index a4ad7e9a5ba..9d660e388cb 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -19,14 +19,14 @@ namespace ErrorCodes class FunctionTupleHammingDistance : public IFunction { private: - ContextConstPtr context; + ContextPtr context; public: static constexpr auto name = "tupleHammingDistance"; using ResultType = UInt8; - explicit FunctionTupleHammingDistance(ContextConstPtr context_) : context(context_) {} - static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context); } + explicit FunctionTupleHammingDistance(ContextPtr context_) : context(context_) {} + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } String getName() const override { return name; } diff --git a/src/Functions/uptime.cpp b/src/Functions/uptime.cpp index 7e2165318a3..02454df4de5 100644 --- a/src/Functions/uptime.cpp +++ b/src/Functions/uptime.cpp @@ -13,7 +13,7 @@ class FunctionUptime : public IFunction { public: static constexpr auto name = "uptime"; - static FunctionPtr create(ContextConstPtr context) + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getUptimeSeconds()); } diff --git a/src/Functions/validateNestedArraySizes.cpp b/src/Functions/validateNestedArraySizes.cpp index f3327479844..1d96f988690 100644 --- a/src/Functions/validateNestedArraySizes.cpp +++ b/src/Functions/validateNestedArraySizes.cpp @@ -24,7 +24,7 @@ class FunctionValidateNestedArraySizes : public IFunction { public: static constexpr auto name = "validateNestedArraySizes"; - static FunctionPtr create(ContextConstPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } diff --git a/src/Functions/version.cpp b/src/Functions/version.cpp index 6181999e2f5..4e0ddf60975 100644 --- a/src/Functions/version.cpp +++ b/src/Functions/version.cpp @@ -16,7 +16,7 @@ class FunctionVersion : public IFunction { public: static constexpr auto name = "version"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/visibleWidth.cpp b/src/Functions/visibleWidth.cpp index f6a79a82df6..6e96a4844ce 100644 --- a/src/Functions/visibleWidth.cpp +++ b/src/Functions/visibleWidth.cpp @@ -19,7 +19,7 @@ class FunctionVisibleWidth : public IFunction { public: static constexpr auto name = "visibleWidth"; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/wkt.cpp b/src/Functions/wkt.cpp index 888f6a4ea5f..b5567a19e89 100644 --- a/src/Functions/wkt.cpp +++ b/src/Functions/wkt.cpp @@ -16,7 +16,7 @@ public: explicit FunctionWkt() = default; - static FunctionPtr create(ContextConstPtr) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/yesterday.cpp b/src/Functions/yesterday.cpp index 0b3ac6d2a8e..737552e192e 100644 --- a/src/Functions/yesterday.cpp +++ b/src/Functions/yesterday.cpp @@ -69,7 +69,7 @@ public: size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextConstPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared(); } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 160566c0f8d..42b800f15d0 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -111,7 +111,7 @@ struct ConnectionTimeouts static ConnectionTimeouts getTCPTimeoutsWithoutFailover(const Settings & settings); /// Timeouts for the case when we will try many addresses in a loop. static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); - static ConnectionTimeouts getHTTPTimeouts(ContextConstPtr context); + static ConnectionTimeouts getHTTPTimeouts(ContextPtr context); }; } diff --git a/src/IO/ConnectionTimeoutsContext.h b/src/IO/ConnectionTimeoutsContext.h index 909ccff8ef0..8d6f96b82a3 100644 --- a/src/IO/ConnectionTimeoutsContext.h +++ b/src/IO/ConnectionTimeoutsContext.h @@ -27,7 +27,7 @@ inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const S settings.receive_data_timeout_ms); } -inline ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(ContextConstPtr context) +inline ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(ContextPtr context) { const auto & settings = context->getSettingsRef(); const auto & config = context->getConfigRef(); diff --git a/src/Interpreters/ColumnAliasesVisitor.h b/src/Interpreters/ColumnAliasesVisitor.h index 6b89c1a086f..a1f8e79f64c 100644 --- a/src/Interpreters/ColumnAliasesVisitor.h +++ b/src/Interpreters/ColumnAliasesVisitor.h @@ -56,12 +56,12 @@ public: /// Do not analyze joined columns. /// They may have aliases and come to description as is. const NameSet & forbidden_columns; - ContextConstPtr context; + ContextPtr context; /// private_aliases are from lambda, so these are local names. NameSet private_aliases; - Data(const ColumnsDescription & columns_, const NameSet & forbidden_columns_, ContextConstPtr context_) + Data(const ColumnsDescription & columns_, const NameSet & forbidden_columns_, ContextPtr context_) : columns(columns_) , forbidden_columns(forbidden_columns_) , context(context_) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a920a26e4c4..06ab801b85d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -538,7 +538,7 @@ ContextMutablePtr Context::createCopy(const ContextPtr & other) return std::shared_ptr(new Context(*other)); } -ContextMutablePtr Context::createCopy(const ContextWeakConstPtr & other) +ContextMutablePtr Context::createCopy(const ContextWeakPtr & other) { auto ptr = other.lock(); if (!ptr) throw Exception("Can't copy an expired context", ErrorCodes::LOGICAL_ERROR); @@ -2583,7 +2583,7 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL); auto resolved_id = StorageID::createEmpty(); - auto try_resolve = [&](ContextConstPtr context) -> bool + auto try_resolve = [&](ContextPtr context) -> bool { const auto & tables = context->external_tables_mapping; auto it = tables.find(storage_id.getTableName()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 97cb1b980f1..7d86cee6e84 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -294,7 +294,7 @@ private: public: /// Create initial Context with ContextShared and etc. static ContextMutablePtr createGlobal(ContextSharedPart * shared); - static ContextMutablePtr createCopy(const ContextWeakConstPtr & other); + static ContextMutablePtr createCopy(const ContextWeakPtr & other); static ContextMutablePtr createCopy(const ContextMutablePtr & other); static ContextMutablePtr createCopy(const ContextPtr & other); static SharedContextHolder createShared(); diff --git a/src/Interpreters/Context_fwd.h b/src/Interpreters/Context_fwd.h index af18e5a7fdd..03f5ea149c6 100644 --- a/src/Interpreters/Context_fwd.h +++ b/src/Interpreters/Context_fwd.h @@ -23,10 +23,8 @@ class Context; /// Most used types have shorter names /// TODO: in the first part of refactoring all the context pointers are non-const. using ContextPtr = std::shared_ptr; -using ContextConstPtr = std::shared_ptr; using ContextMutablePtr = std::shared_ptr; using ContextWeakPtr = std::weak_ptr; -using ContextWeakConstPtr = std::weak_ptr; using ContextWeakMutablePtr = std::weak_ptr; template @@ -51,7 +49,6 @@ protected: }; using WithContext = WithContextImpl<>; -using WithConstContext = WithContextImpl; using WithMutableContext = WithContextImpl; } diff --git a/src/Interpreters/DuplicateOrderByVisitor.h b/src/Interpreters/DuplicateOrderByVisitor.h index bd8652ff298..4231b2600af 100644 --- a/src/Interpreters/DuplicateOrderByVisitor.h +++ b/src/Interpreters/DuplicateOrderByVisitor.h @@ -21,7 +21,7 @@ class ASTFunctionStatefulData public: using TypeToVisit = ASTFunction; - ContextConstPtr context; + ContextPtr context; bool & is_stateful; void visit(ASTFunction & ast_function, ASTPtr &) { @@ -80,7 +80,7 @@ class DuplicateOrderByData public: using TypeToVisit = ASTSelectQuery; - ContextConstPtr context; + ContextPtr context; void visit(ASTSelectQuery & select_query, ASTPtr &) { diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index aa404f2c622..c230f346779 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -33,7 +33,7 @@ class ExecuteScalarSubqueriesMatcher public: using Visitor = InDepthNodeVisitor; - struct Data : public WithConstContext + struct Data : public WithContext { size_t subquery_depth; Scalars & scalars; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index e6414474817..2010d0eff65 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -23,9 +23,9 @@ namespace ErrorCodes } /// Must not acquire Context lock in constructor to avoid possibility of deadlocks. -ExternalDictionariesLoader::ExternalDictionariesLoader(ContextConstPtr global_context_) +ExternalDictionariesLoader::ExternalDictionariesLoader(ContextPtr global_context_) : ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader")) - , WithConstContext(global_context_) + , WithContext(global_context_) { setConfigSettings({"dictionary", "name", "database", "uuid"}); enableAsyncLoading(true); @@ -42,26 +42,26 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), created_from_ddl); } -ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextConstPtr local_context) const +ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); return std::static_pointer_cast(load(resolved_dictionary_name)); } -ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextConstPtr local_context) const +ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); return std::static_pointer_cast(tryLoad(resolved_dictionary_name)); } -void ExternalDictionariesLoader::reloadDictionary(const std::string & dictionary_name, ContextConstPtr local_context) const +void ExternalDictionariesLoader::reloadDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); loadOrReload(resolved_dictionary_name); } -DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std::string & dictionary_name, ContextConstPtr query_context) const +DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std::string & dictionary_name, ContextPtr query_context) const { std::string resolved_name = resolveDictionaryName(dictionary_name, query_context->getCurrentDatabase()); diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 1182e96573c..06f64ef30c5 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -13,21 +13,21 @@ namespace DB class IExternalLoaderConfigRepository; /// Manages user-defined dictionaries. -class ExternalDictionariesLoader : public ExternalLoader, WithConstContext +class ExternalDictionariesLoader : public ExternalLoader, WithContext { public: using DictPtr = std::shared_ptr; /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. - explicit ExternalDictionariesLoader(ContextConstPtr global_context_); + explicit ExternalDictionariesLoader(ContextPtr global_context_); - DictPtr getDictionary(const std::string & dictionary_name, ContextConstPtr context) const; + DictPtr getDictionary(const std::string & dictionary_name, ContextPtr context) const; - DictPtr tryGetDictionary(const std::string & dictionary_name, ContextConstPtr context) const; + DictPtr tryGetDictionary(const std::string & dictionary_name, ContextPtr context) const; - void reloadDictionary(const std::string & dictionary_name, ContextConstPtr context) const; + void reloadDictionary(const std::string & dictionary_name, ContextPtr context) const; - DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextConstPtr context) const; + DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const; static DictionaryStructure getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config = "dictionary"); diff --git a/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 34632a96bda..2d46fe08e95 100644 --- a/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -82,12 +82,12 @@ bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) return !node->as(); } -bool hasNonRewritableFunction(const ASTPtr & node, ContextConstPtr context) +bool hasNonRewritableFunction(const ASTPtr & node, ContextPtr context) { for (const auto & select_expression : node->children) { TablesWithColumns tables; - ExpressionInfoVisitor::Data expression_info{WithConstContext{context}, tables}; + ExpressionInfoVisitor::Data expression_info{WithContext{context}, tables}; ExpressionInfoVisitor(expression_info).visit(select_expression); if (expression_info.is_stateful_function diff --git a/src/Interpreters/ExtractExpressionInfoVisitor.h b/src/Interpreters/ExtractExpressionInfoVisitor.h index bfb414e8601..c84e243ce2e 100644 --- a/src/Interpreters/ExtractExpressionInfoVisitor.h +++ b/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -13,7 +13,7 @@ namespace DB struct ExpressionInfoMatcher { - struct Data : public WithConstContext + struct Data : public WithContext { const TablesWithColumns & tables; @@ -36,6 +36,6 @@ struct ExpressionInfoMatcher using ExpressionInfoVisitor = ConstInDepthNodeVisitor; -bool hasNonRewritableFunction(const ASTPtr & node, ContextConstPtr context); +bool hasNonRewritableFunction(const ASTPtr & node, ContextPtr context); } diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index f0698b27426..350318047c7 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -26,7 +26,7 @@ public: struct Data { const TablesWithColumns & tables; - ContextConstPtr context; + ContextPtr context; const std::unordered_set & group_by_function_hashes; Monotonicity monotonicity{true, true, true}; ASTIdentifier * identifier = nullptr; diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index b3ef2e42e4a..f2e55441fb6 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -19,8 +19,8 @@ namespace ErrorCodes } PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( - ContextConstPtr context_, const TablesWithColumns & tables_with_columns_, const Settings & settings) - : WithConstContext(context_) + ContextPtr context_, const TablesWithColumns & tables_with_columns_, const Settings & settings) + : WithContext(context_) , enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression) , enable_optimize_predicate_expression_to_final_subquery(settings.enable_optimize_predicate_expression_to_final_subquery) , allow_push_predicate_when_subquery_contains_with(settings.allow_push_predicate_when_subquery_contains_with) @@ -87,7 +87,7 @@ std::vector PredicateExpressionsOptimizer::extractTablesPredicates(const A for (const auto & predicate_expression : splitConjunctionPredicate({where, prewhere})) { - ExpressionInfoVisitor::Data expression_info{WithConstContext{getContext()}, tables_with_columns}; + ExpressionInfoVisitor::Data expression_info{WithContext{getContext()}, tables_with_columns}; ExpressionInfoVisitor(expression_info).visit(predicate_expression); if (expression_info.is_stateful_function @@ -188,7 +188,7 @@ bool PredicateExpressionsOptimizer::tryMovePredicatesFromHavingToWhere(ASTSelect for (const auto & moving_predicate: splitConjunctionPredicate({select_query.having()})) { TablesWithColumns tables; - ExpressionInfoVisitor::Data expression_info{WithConstContext{getContext()}, tables}; + ExpressionInfoVisitor::Data expression_info{WithContext{getContext()}, tables}; ExpressionInfoVisitor(expression_info).visit(moving_predicate); /// TODO: If there is no group by, where, and prewhere expression, we can push down the stateful function diff --git a/src/Interpreters/PredicateExpressionsOptimizer.h b/src/Interpreters/PredicateExpressionsOptimizer.h index db580032f73..a31b9907da6 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/src/Interpreters/PredicateExpressionsOptimizer.h @@ -15,10 +15,10 @@ struct Settings; * - Move predicates from having to where * - Push the predicate down from the current query to the having of the subquery */ -class PredicateExpressionsOptimizer : WithConstContext +class PredicateExpressionsOptimizer : WithContext { public: - PredicateExpressionsOptimizer(ContextConstPtr context_, const TablesWithColumns & tables_with_columns_, const Settings & settings_); + PredicateExpressionsOptimizer(ContextPtr context_, const TablesWithColumns & tables_with_columns_, const Settings & settings_); bool optimize(ASTSelectQuery & select_query); diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index 3dda066c4ac..0f2a11e6ff1 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -17,12 +17,12 @@ namespace DB { PredicateRewriteVisitorData::PredicateRewriteVisitorData( - ContextConstPtr context_, + ContextPtr context_, const ASTs & predicates_, const TableWithColumnNamesAndTypes & table_columns_, bool optimize_final_, bool optimize_with_) - : WithConstContext(context_) + : WithContext(context_) , predicates(predicates_) , table_columns(table_columns_) , optimize_final(optimize_final_) diff --git a/src/Interpreters/PredicateRewriteVisitor.h b/src/Interpreters/PredicateRewriteVisitor.h index b4ff063c954..fc076464925 100644 --- a/src/Interpreters/PredicateRewriteVisitor.h +++ b/src/Interpreters/PredicateRewriteVisitor.h @@ -10,7 +10,7 @@ namespace DB { -class PredicateRewriteVisitorData : WithConstContext +class PredicateRewriteVisitorData : WithContext { public: bool is_rewrite = false; @@ -24,7 +24,7 @@ public: } PredicateRewriteVisitorData( - ContextConstPtr context_, + ContextPtr context_, const ASTs & predicates_, const TableWithColumnNamesAndTypes & table_columns_, bool optimize_final_, diff --git a/src/Interpreters/RedundantFunctionsInOrderByVisitor.h b/src/Interpreters/RedundantFunctionsInOrderByVisitor.h index 04ac1607b60..f807849fb86 100644 --- a/src/Interpreters/RedundantFunctionsInOrderByVisitor.h +++ b/src/Interpreters/RedundantFunctionsInOrderByVisitor.h @@ -16,7 +16,7 @@ public: struct Data { std::unordered_set & keys; - ContextConstPtr context; + ContextPtr context; bool redundant = true; bool done = false; diff --git a/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp b/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp index e242eab919a..8d030379909 100644 --- a/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp +++ b/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp @@ -17,7 +17,7 @@ static bool isUniq(const ASTFunction & func) } /// Remove injective functions of one argument: replace with a child -static bool removeInjectiveFunction(ASTPtr & ast, ContextConstPtr context, const FunctionFactory & function_factory) +static bool removeInjectiveFunction(ASTPtr & ast, ContextPtr context, const FunctionFactory & function_factory) { const ASTFunction * func = ast->as(); if (!func) diff --git a/src/Interpreters/RemoveInjectiveFunctionsVisitor.h b/src/Interpreters/RemoveInjectiveFunctionsVisitor.h index 29fd9bb0af5..a3bbd562407 100644 --- a/src/Interpreters/RemoveInjectiveFunctionsVisitor.h +++ b/src/Interpreters/RemoveInjectiveFunctionsVisitor.h @@ -13,9 +13,9 @@ class ASTFunction; class RemoveInjectiveFunctionsMatcher { public: - struct Data : public WithConstContext + struct Data : public WithContext { - explicit Data(ContextConstPtr context_) : WithConstContext(context_) {} + explicit Data(ContextPtr context_) : WithContext(context_) {} }; static void visit(ASTPtr & ast, const Data & data); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index a2725f2506e..5b06c00435a 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -81,7 +81,7 @@ void appendUnusedGroupByColumn(ASTSelectQuery * select_query, const NameSet & so } /// Eliminates injective function calls and constant expressions from group by statement. -void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, ContextConstPtr context) +void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, ContextPtr context) { const FunctionFactory & function_factory = FunctionFactory::instance(); @@ -270,7 +270,7 @@ void optimizeDuplicatesInOrderBy(const ASTSelectQuery * select_query) } /// Optimize duplicate ORDER BY -void optimizeDuplicateOrderBy(ASTPtr & query, ContextConstPtr context) +void optimizeDuplicateOrderBy(ASTPtr & query, ContextPtr context) { DuplicateOrderByVisitor::Data order_by_data{context}; DuplicateOrderByVisitor(order_by_data).visit(query); @@ -396,7 +396,7 @@ void optimizeDuplicateDistinct(ASTSelectQuery & select) /// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression, /// has a single argument and not an aggregate functions. -void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, ContextConstPtr context, +void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, ContextPtr context, const TablesWithColumns & tables_with_columns, const Names & sorting_key_columns) { @@ -448,7 +448,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context /// Optimize ORDER BY x, y, f(x), g(x, y), f(h(x)), t(f(x), g(x)) into ORDER BY x, y /// in case if f(), g(), h(), t() are deterministic (in scope of query). /// Don't optimize ORDER BY f(x), g(x), x even if f(x) is bijection for x or g(x). -void optimizeRedundantFunctionsInOrderBy(const ASTSelectQuery * select_query, ContextConstPtr context) +void optimizeRedundantFunctionsInOrderBy(const ASTSelectQuery * select_query, ContextPtr context) { const auto & order_by = select_query->orderBy(); if (!order_by) @@ -561,7 +561,7 @@ void optimizeCountConstantAndSumOne(ASTPtr & query) } -void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextConstPtr context) +void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) { RemoveInjectiveFunctionsVisitor::Data data(context); RemoveInjectiveFunctionsVisitor(data).visit(query); @@ -592,7 +592,7 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, - ContextConstPtr context, const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, const StorageMetadataPtr & metadata_snapshot, bool & rewrite_subqueries) { const auto & settings = context->getSettingsRef(); diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index 706f030e620..b268b230f4e 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -21,7 +21,7 @@ public: Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, - ContextConstPtr context, + ContextPtr context, const StorageMetadataPtr & metadata_snapshot, bool & rewrite_subqueries); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 5b4a869d44b..92cfba1bcb1 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -413,10 +413,10 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, } /// Replacing scalar subqueries with constant values. -void executeScalarSubqueries(ASTPtr & query, ContextConstPtr context, size_t subquery_depth, Scalars & scalars, bool only_analyze) +void executeScalarSubqueries(ASTPtr & query, ContextPtr context, size_t subquery_depth, Scalars & scalars, bool only_analyze) { LogAST log; - ExecuteScalarSubqueriesVisitor::Data visitor_data{WithConstContext{context}, subquery_depth, scalars, only_analyze}; + ExecuteScalarSubqueriesVisitor::Data visitor_data{WithContext{context}, subquery_depth, scalars, only_analyze}; ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query); } diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 1f535325666..32826bcc61d 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -92,10 +92,10 @@ using TreeRewriterResultPtr = std::shared_ptr; /// * scalar subqueries are executed replaced with constants /// * unneeded columns are removed from SELECT clause /// * duplicated columns are removed from ORDER BY, LIMIT BY, USING(...). -class TreeRewriter : WithConstContext +class TreeRewriter : WithContext { public: - explicit TreeRewriter(ContextConstPtr context_) : WithConstContext(context_) {} + explicit TreeRewriter(ContextPtr context_) : WithContext(context_) {} /// Analyze and rewrite not select query TreeRewriterResultPtr analyze( diff --git a/src/Interpreters/addTypeConversionToAST.cpp b/src/Interpreters/addTypeConversionToAST.cpp index 86fd7926e78..295ac858e28 100644 --- a/src/Interpreters/addTypeConversionToAST.cpp +++ b/src/Interpreters/addTypeConversionToAST.cpp @@ -32,7 +32,7 @@ ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name) return func; } -ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name, const NamesAndTypesList & all_columns, ContextConstPtr context) +ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name, const NamesAndTypesList & all_columns, ContextPtr context) { auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, all_columns); const auto actions = ExpressionAnalyzer(ast, diff --git a/src/Interpreters/addTypeConversionToAST.h b/src/Interpreters/addTypeConversionToAST.h index 7a4d879dc61..eb391b2c749 100644 --- a/src/Interpreters/addTypeConversionToAST.h +++ b/src/Interpreters/addTypeConversionToAST.h @@ -14,6 +14,6 @@ class NamesAndTypesList; ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name); // If same type, then ignore the wrapper of CAST function -ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name, const NamesAndTypesList & all_columns, ContextConstPtr context); +ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name, const NamesAndTypesList & all_columns, ContextPtr context); } diff --git a/src/Interpreters/replaceAliasColumnsInQuery.cpp b/src/Interpreters/replaceAliasColumnsInQuery.cpp index d85c1b23c2f..0bc8828c878 100644 --- a/src/Interpreters/replaceAliasColumnsInQuery.cpp +++ b/src/Interpreters/replaceAliasColumnsInQuery.cpp @@ -6,7 +6,7 @@ namespace DB { -void replaceAliasColumnsInQuery(ASTPtr & ast, const ColumnsDescription & columns, const NameSet & forbidden_columns, ContextConstPtr context) +void replaceAliasColumnsInQuery(ASTPtr & ast, const ColumnsDescription & columns, const NameSet & forbidden_columns, ContextPtr context) { ColumnAliasesVisitor::Data aliases_column_data(columns, forbidden_columns, context); ColumnAliasesVisitor aliases_column_visitor(aliases_column_data); diff --git a/src/Interpreters/replaceAliasColumnsInQuery.h b/src/Interpreters/replaceAliasColumnsInQuery.h index 90963ea167b..92d2686b45b 100644 --- a/src/Interpreters/replaceAliasColumnsInQuery.h +++ b/src/Interpreters/replaceAliasColumnsInQuery.h @@ -10,6 +10,6 @@ namespace DB class ColumnsDescription; -void replaceAliasColumnsInQuery(ASTPtr & ast, const ColumnsDescription & columns, const NameSet & forbidden_columns, ContextConstPtr context); +void replaceAliasColumnsInQuery(ASTPtr & ast, const ColumnsDescription & columns, const NameSet & forbidden_columns, ContextPtr context); } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.h b/src/Processors/Formats/Impl/MySQLOutputFormat.h index 01a892410df..7d67df3015e 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -18,14 +18,14 @@ class WriteBuffer; /** A stream for outputting data in a binary line-by-line format. */ -class MySQLOutputFormat final : public IOutputFormat, WithConstContext +class MySQLOutputFormat final : public IOutputFormat, WithContext { public: MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_); String getName() const override { return "MySQLOutputFormat"; } - void setContext(ContextConstPtr context_) + void setContext(ContextPtr context_) { context = context_; packet_endpoint = std::make_unique(out, const_cast(getContext()->mysql.sequence_id)); /// TODO: fix it diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index ea5ab9239e0..b1e44f5d104 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -34,7 +34,7 @@ public: void resetParser() override; /// TODO: remove context somehow. - void setContext(ContextConstPtr context_) { context = Context::createCopy(context_); } + void setContext(ContextPtr context_) { context = Context::createCopy(context_); } const BlockMissingValues & getMissingValues() const override { return block_missing_values; } From 84af235a6037f7d87b0139a94cf1be91ec1ff407 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Jun 2021 15:21:41 +0300 Subject: [PATCH 309/652] fix test --- tests/queries/0_stateless/01821_join_table_race_long.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01821_join_table_race_long.sh b/tests/queries/0_stateless/01821_join_table_race_long.sh index 9602da1e12a..7c56bf77bfd 100755 --- a/tests/queries/0_stateless/01821_join_table_race_long.sh +++ b/tests/queries/0_stateless/01821_join_table_race_long.sh @@ -8,13 +8,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS join_table_race" $CLICKHOUSE_CLIENT -q "CREATE TABLE join_table_race(id Int32, name String) ENGINE = Join(ANY, LEFT, id)" -for i in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & -for i in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & +for _ in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & -for i in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & -for i in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & wait From 596989161174096c4117915c027219752d90922c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Jun 2021 16:25:23 +0300 Subject: [PATCH 310/652] do not crash on intersecting parts --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 19 +++-- src/Storages/MergeTree/ActiveDataPartSet.h | 6 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 25 ++++-- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 3 + .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 83 ++++++++++--------- .../MergeTree/ReplicatedMergeTreeQueue.h | 11 +-- src/Storages/StorageReplicatedMergeTree.cpp | 6 -- 9 files changed, 93 insertions(+), 68 deletions(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index bbf05afa42c..990ca5d249d 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -1,6 +1,8 @@ #include #include +#include #include +#include namespace DB @@ -18,8 +20,8 @@ ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, add(name); } - -bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) +/// FIXME replace warnings with logical errors +bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts, Poco::Logger * log) { /// TODO make it exception safe (out_replaced_parts->push_back(...) may throw) auto part_info = MergeTreePartInfo::fromPartName(name, format_version); @@ -40,7 +42,10 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) if (!part_info.contains(it->first)) { if (!part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects previous part {}. It is a bug.", name, it->first.getPartName()); + { + LOG_ERROR(log, "Part {} intersects previous part {}. It is a bug.", name, it->first.getPartName()); + assert(false); + } ++it; break; } @@ -56,15 +61,17 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) /// Let's go to the right. while (it != part_info_to_name.end() && part_info.contains(it->first)) { - if (part_info == it->first) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected duplicate part {}. It is a bug.", name); + assert(part_info != it->first); if (out_replaced_parts) out_replaced_parts->push_back(it->second); part_info_to_name.erase(it++); } if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects next part {}. It is a bug.", name, it->first.getPartName()); + { + LOG_ERROR(log, "Part {} intersects next part {}. It is a bug.", name, it->first.getPartName()); + assert(false); + } part_info_to_name.emplace(part_info, name); return true; diff --git a/src/Storages/MergeTree/ActiveDataPartSet.h b/src/Storages/MergeTree/ActiveDataPartSet.h index 323ef3c4830..188bba91795 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.h +++ b/src/Storages/MergeTree/ActiveDataPartSet.h @@ -5,6 +5,10 @@ #include #include +namespace Poco +{ +class Logger; +} namespace DB { @@ -46,7 +50,7 @@ public: /// Returns true if the part was actually added. If out_replaced_parts != nullptr, it will contain /// parts that were replaced from the set by the newly added part. - bool add(const String & name, Strings * out_replaced_parts = nullptr); + bool add(const String & name, Strings * out_replaced_parts = nullptr, Poco::Logger * log = nullptr); bool remove(const MergeTreePartInfo & part_info) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 188f6366f8b..271d8a4c926 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4897,7 +4897,11 @@ void MergeTreeData::removeQueryId(const String & query_id) const { std::lock_guard lock(query_id_set_mutex); if (query_id_set.find(query_id) == query_id_set.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "We have query_id removed but it's not recorded. This is a bug"); + { + /// Do not throw exception, because this method is used in destructor. + LOG_WARNING(log, "We have query_id removed but it's not recorded. This is a bug"); + assert(false); + } else query_id_set.erase(query_id); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index dbd55bc4ff3..0f296fd1571 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -413,6 +413,24 @@ ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String return res; } +std::optional ReplicatedMergeTreeLogEntryData::getDropRange(MergeTreeDataFormatVersion format_version) const +{ + if (type == DROP_RANGE) + return new_part_name; + + if (type == REPLACE_RANGE) + { + auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version); + if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info)) + { + /// It's REPLACE, not MOVE or ATTACH, so drop range is real + return replace_range_entry->drop_range_part_name; + } + } + + return {}; +} + Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormatVersion format_version) const { /// Doesn't produce any part @@ -431,11 +449,8 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat { Strings res = replace_range_entry->new_part_names; auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version); - if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info)) - { - /// It's REPLACE, not MOVE or ATTACH, so drop range is real - res.emplace_back(replace_range_entry->drop_range_part_name); - } + if (auto drop_range = getDropRange(format_version)) + res.emplace_back(*drop_range); return res; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 12f1c78fb5d..1e7e1475ac0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -152,6 +152,9 @@ struct ReplicatedMergeTreeLogEntryData return res; } + /// Returns fake part for drop range (for DROP_RANGE and REPLACE_RANGE) + std::optional getDropRange(MergeTreeDataFormatVersion format_version) const; + /// Access under queue_mutex, see ReplicatedMergeTreeQueue. bool currently_executing = false; /// Whether the action is executing now. bool removed_by_other_entry = false; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 35c42a7f325..d170ba835cb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -191,7 +191,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPartAndFetchIfPossible( if (missing_part_search_result == MissingPartSearchResult::LostForever) { /// Is it in the replication queue? If there is - delete, because the task can not be processed. - if (!storage.queue.markPartAsLostForever(zookeeper, part_name)) + if (!storage.queue.remove(zookeeper, part_name)) { /// The part was not in our queue. LOG_WARNING(log, "Missing part {} is not in our queue, this can happen rarely.", part_name); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 30569e53f64..323eb18b203 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -34,24 +34,29 @@ ReplicatedMergeTreeQueue::ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & } +void ReplicatedMergeTreeQueue::clear() +{ + auto locks = lockQueue(); + assert(future_parts.empty()); + current_parts.clear(); + virtual_parts.clear(); + queue.clear(); + inserts_by_time.clear(); + mutations_by_znode.clear(); + mutations_by_partition.clear(); + mutation_pointer.clear(); +} + void ReplicatedMergeTreeQueue::initialize(const MergeTreeData::DataParts & parts) -{ - addVirtualParts(parts); -} - - -void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts & parts) { std::lock_guard lock(state_mutex); - for (const auto & part : parts) { - current_parts.add(part->name); - virtual_parts.add(part->name); + current_parts.add(part->name, nullptr, log); + virtual_parts.add(part->name, nullptr, log); } } - bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const { std::lock_guard lock(state_mutex); @@ -74,9 +79,6 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) /// Reset batch size on initialization to recover from possible errors of too large batch size. current_multi_batch_size = 1; - String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); - log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); - std::unordered_set already_loaded_paths; { std::lock_guard lock(state_mutex); @@ -134,7 +136,7 @@ void ReplicatedMergeTreeQueue::insertUnlocked( { for (const String & virtual_part_name : entry->getVirtualPartNames(format_version)) { - virtual_parts.add(virtual_part_name); + virtual_parts.add(virtual_part_name, nullptr, log); addPartToMutations(virtual_part_name); } @@ -221,23 +223,17 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( for (const String & virtual_part_name : entry->getVirtualPartNames(format_version)) { - current_parts.add(virtual_part_name); + current_parts.add(virtual_part_name, nullptr, log); /// These parts are already covered by newer part, we don't have to /// mutate it. removeCoveredPartsFromMutations(virtual_part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } - String drop_range_part_name; - if (entry->type == LogEntry::DROP_RANGE) - drop_range_part_name = entry->new_part_name; - else if (entry->type == LogEntry::REPLACE_RANGE) - drop_range_part_name = entry->replace_range_entry->drop_range_part_name; - - if (!drop_range_part_name.empty()) + if (auto drop_range_part_name = entry->getDropRange(format_version)) { - current_parts.remove(drop_range_part_name); - virtual_parts.remove(drop_range_part_name); + current_parts.remove(*drop_range_part_name); + virtual_parts.remove(*drop_range_part_name); } if (entry->type == LogEntry::ALTER_METADATA) @@ -302,9 +298,7 @@ void ReplicatedMergeTreeQueue::addPartToMutations(const String & part_name) auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); /// Do not add special virtual parts to parts_to_do - auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION - auto another_max_level = std::numeric_limits::max(); /// REPLACE/MOVE PARTITION - if (part_info.level == max_level || part_info.level == another_max_level) + if (part_info.isFakeDropRangePart()) return; auto in_partition = mutations_by_partition.find(part_info.partition_id); @@ -344,7 +338,9 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper( auto code = zookeeper->tryMulti(ops, responses); if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Couldn't set value of nodes for insert times ({}/min_unprocessed_insert_time, max_processed_insert_time): {}. This shouldn't happen often.", replica_path, Coordination::errorMessage(code)); + LOG_ERROR(log, "Couldn't set value of nodes for insert times " + "({}/min_unprocessed_insert_time, max_processed_insert_time): {}. " + "This shouldn't happen often.", replica_path, Coordination::errorMessage(code)); } } @@ -392,7 +388,8 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep } if (!found && need_remove_from_zk) - throw Exception("Can't find " + entry->znode_name + " in the memory queue. It is a bug", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't find {} in the memory queue. It is a bug. Entry: {}", + entry->znode_name, entry->toString()); notifySubscribers(queue_size); @@ -434,7 +431,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri { auto part_in_current_parts = current_parts.getContainingPart(source_part); if (part_in_current_parts == source_part) - virtual_parts.add(source_part); + virtual_parts.add(source_part, nullptr, log); } } @@ -462,8 +459,9 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri } -bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & part_info) +bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & part_info) { + assert(part_info.level == 0); std::lock_guard lock(state_mutex); return virtual_parts.remove(part_info); } @@ -587,8 +585,6 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper { std::lock_guard state_lock(state_mutex); - log_pointer = last_entry_index + 1; - for (size_t copied_entry_idx = 0, num_copied_entries = copied_entries.size(); copied_entry_idx < num_copied_entries; ++copied_entry_idx) { String path_created = dynamic_cast(*responses[copied_entry_idx]).path_created; @@ -758,9 +754,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C /// Such parts do not exist and will never appear, so we should not add virtual parts to parts_to_do list. /// Fortunately, it's easy to distinguish virtual parts from normal parts by part level. /// See StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(...) - auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION - auto another_max_level = std::numeric_limits::max(); /// REPLACE/MOVE PARTITION - if (part_info.level == max_level || part_info.level == another_max_level) + if (part_info.isFakeDropRangePart()) continue; auto it = entry->block_numbers.find(part_info.partition_id); @@ -941,9 +935,6 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( if ((*it)->currently_executing) to_wait.push_back(*it); auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); - /// FIXME it's probably unsafe to remove entries non-atomically - /// when this method called directly from alter query (not from replication queue task), - /// because entries will be lost if ALTER fails. if (code != Coordination::Error::ZOK) LOG_INFO(log, "Couldn't remove {}: {}", replica_path + "/queue/" + (*it)->znode_name, Coordination::errorMessage(code)); @@ -1259,7 +1250,9 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(const Replicate for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version)) { if (!queue.future_parts.emplace(new_part_name, entry).second) - throw Exception("Tagging already tagged future part " + new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged future part {}. This is a bug. " + "It happened on attempt to execute {}: {}", + new_part_name, entry->znode_name, entry->toString()); } } @@ -1277,7 +1270,9 @@ void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName(ReplicatedM return; if (!queue.future_parts.emplace(entry.actual_new_part_name, entry.shared_from_this()).second) - throw Exception("Attaching already existing future part " + entry.actual_new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attaching already existing future part {}. This is a bug. " + "It happened on attempt to execute {}: {}", + entry.actual_new_part_name, entry.znode_name, entry.toString()); } @@ -1296,13 +1291,19 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version)) { if (!queue.future_parts.erase(new_part_name)) + { LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name); + assert(false); + } } if (!entry->actual_new_part_name.empty()) { if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name)) + { LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name); + assert(false); + } entry->actual_new_part_name.clear(); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 496f277d132..078795472bf 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -92,9 +92,6 @@ private: using FuturePartsSet = std::map; FuturePartsSet future_parts; - /// Index of the first log entry that we didn't see yet. - Int64 log_pointer = 0; - /// Avoid parallel execution of queue enties, which may remove other entries from the queue. bool currently_executing_drop_or_replace_range = false; @@ -183,9 +180,6 @@ private: /// Ensures that only one thread is simultaneously updating mutations. std::mutex update_mutations_mutex; - /// Put a set of (already existing) parts in virtual_parts. - void addVirtualParts(const MergeTreeData::DataParts & parts); - /// Insert new entry from log into queue void insertUnlocked( const LogEntryPtr & entry, std::optional & min_unprocessed_insert_time_changed, @@ -275,7 +269,10 @@ public: ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeMergeStrategyPicker & merge_strategy_picker_); ~ReplicatedMergeTreeQueue(); + /// Clears queue state + void clear(); + /// Put a set of (already existing) parts in virtual_parts. void initialize(const MergeTreeData::DataParts & parts); /** Inserts an action to the end of the queue. @@ -295,7 +292,7 @@ public: */ bool load(zkutil::ZooKeeperPtr zookeeper); - bool removeFromVirtualParts(const MergeTreePartInfo & part_info); + bool removeFailedQuorumPart(const MergeTreePartInfo & part_info); /** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value. * If watch_callback is not empty, will call it when new entries appear in the log. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 23abb39eb3e..3bb4b16edfc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1465,12 +1465,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } - if (entry.type == LogEntry::PART_IS_LOST) - { - queue.executePartIsLost(getZooKeeper(), entry); - return true; - } - const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; if (is_get_or_attach || entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) From 275a616bb6182a02288c632e9c6bd51d095d220a Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Jun 2021 16:27:09 +0300 Subject: [PATCH 311/652] Update ActiveDataPartSet.cpp --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 990ca5d249d..8961b3b452f 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -8,10 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, const Strings & names) : format_version(format_version_) From 1d539a8695256811926b4373a4b57b99788e9221 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Jun 2021 16:29:45 +0300 Subject: [PATCH 312/652] Fix gitignore --- utils/simple-backport/.gitignore | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/utils/simple-backport/.gitignore b/utils/simple-backport/.gitignore index 72e8ffc0db8..9ab24b6c8b8 100644 --- a/utils/simple-backport/.gitignore +++ b/utils/simple-backport/.gitignore @@ -1 +1,3 @@ -* +*.md +*.txt +*.json From 27a9c1a0cb1eb386e53325552f5aaefac745bf44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Jun 2021 16:29:57 +0300 Subject: [PATCH 313/652] Remove obsolete script and russian readme --- utils/simple-backport/README.md | 107 ---------------------------- utils/simple-backport/backport.sh | 112 ------------------------------ 2 files changed, 219 deletions(-) delete mode 100644 utils/simple-backport/README.md delete mode 100755 utils/simple-backport/backport.sh diff --git a/utils/simple-backport/README.md b/utils/simple-backport/README.md deleted file mode 100644 index c5a625ca0d1..00000000000 --- a/utils/simple-backport/README.md +++ /dev/null @@ -1,107 +0,0 @@ -# Упрощённый скрипт для бекпортирования - -Это упрощённый скрипт для бекпортирования. Он определяет, какие пулреквесты ещё не бекпортировали из мастера в указанную ветку. Запускать скрипт нужно из папки, где он лежит, указав ему название ветки. Он предполагает, что ваш апстримный remote называется origin. -``` -cd my-clickhouse-repo/utils/simple-backport -git fetch origin -time GITHUB_TOKEN= ./backport.sh 20.1 -``` - -Скрипт выведет примитивный отчёт: -``` -$ time GITHUB_TOKEN= ~/backport.sh 20.3 -144 PRs differ between 20.3 and master. -backport https://github.com/ClickHouse/ClickHouse/pull/10135 -backport https://github.com/ClickHouse/ClickHouse/pull/10121 -... -backport https://github.com/ClickHouse/ClickHouse/pull/9808 -backport https://github.com/ClickHouse/ClickHouse/pull/9410 - -real 0m1.213s -user 0m1.065s -sys 0m0.311s -``` - -Также в рабочей папке сгенерируется отчёт `<ваша-ветка>-report.tsv`: - -``` -$ cat 20.3-report.tsv -skip 10153 https://github.com/ClickHouse/ClickHouse/pull/10153 pr10153.json -skip 10147 https://github.com/ClickHouse/ClickHouse/pull/10147 pr10147.json -no-backport 10138 https://github.com/ClickHouse/ClickHouse/pull/10138 pr10138.json -backport 10135 https://github.com/ClickHouse/ClickHouse/pull/10135 pr10135.json -skip 10134 https://github.com/ClickHouse/ClickHouse/pull/10134 pr10134.json -... -``` - -Можно кликать по ссылкам прям из консоли, а можно ещё проще: - -``` -$ cat <ветка>-report.tsv | grep ^backport | cut -f3 -$ cat <ветка>-report.tsv | grep ^backport | cut -f3 | xargs -n1 xdg-open -``` - -Такая команда откроет в браузере все пулреквесты, которые надо бекпортировать. Есть и другие статусы, посмотрите какие: - -``` -$ cat 20.1-report.tsv | cut -f1 | sort | uniq -c | sort -rn - 446 skip - 38 done - 25 conflict - 18 backport - 10 no-backport -``` - -### Как разметить пулреквест? -По умолчанию бекпортируются все пулреквесты, у которых в описании указана -категория чейнжлога Bug fix. Если этого недостаточно, используйте теги: -* v20.1-no-backport -- в ветку 20.1 бекпортировать не нужно. -* pr-no-backport -- ни в какие ветки бекпортировать не нужно. -* v20.1-conflicts -- при бекпорте в 20.1 произошёл конфликт. Такие пулреквесты - скрипт пропускает, к ним можно потом вернуться. -* pr-must-backport -- нужно бекпортировать в поддерживаемые ветки. -* v20.1-must-backport -- нужно бекпортировать в 20.1. - -### Я бекпортировал, почему скрипт не видит? -* Сообщение коммита должно содержать текст backport/cherry-pick #12345, или - иметь вид стандартного гитхабовского мерж-коммита для ПР #12345. -* Коммит должен быть достижим по `git log --first-parent my-branch`. Возможно, - в ветке сделали pull с merge, от чего некоторые коммиты из ветки становятся -недоступны по `--first-parent`. - -В качестве обхода, добавьте в ветку пустой коммит с текстом вроде "backport -#12345 -- real backport commit is ". - -### Я поправил пулреквест, почему скрипт не видит? -В процессе работы скрипт кеширует данные о пулреквестах в текущей папке, чтобы -экономить квоту гитхаба. Удалите закешированные файлы, например, для всех -реквестов, которые не помечены как пропущенные: -``` -$ cat <ваша-ветка>-report.tsv | grep -v "^skip" | cut -f4 -$ cat <ваша-ветка>-report.tsv | grep -v "^skip" | cut -f4 | xargs rm -``` - -## Как сформировать change log -В этой же папке запустите: -``` -$ time GITHUB_TOKEN=... ./changelog.sh v20.3.4.10-stable v20.3.5.21-stable -9 PRs added between v20.3.4.10-stable and v20.3.5.21-stable. -### ClickHouse release v20.3.5.21-stable FIXME as compared to v20.3.4.10-stable - -#### Bug Fix - -* Fix 'Different expressions with the same alias' error when query has PREWHERE - and WHERE on distributed table and `SET distributed_product_mode = 'local'`. -[#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem -Zuikov](https://github.com/4ertus2)). -... -``` - -Скрипт выведет changelog на экран, а также сохранит его в `./changelog.md`. -Скопируйте этот текст в большой changelog, проверьте и поправьте версию и дату -релиза, вычитайте сообщения. Если сообщения неправильные, обязательно исправьте -их на гитхабе -- это поможет при последующей генерации changelog для других -версий, содержащих этот пулреквест. Чтобы скрипт подтянул изменения с гитхаба, -удалите соответствующие файлы `./pr12345.json`. Если вы часто видите -неправильно оформленные пулреквесты, это повод подумать об улучшении проверки -Description check в CI. diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh deleted file mode 100755 index 64f8e6004bf..00000000000 --- a/utils/simple-backport/backport.sh +++ /dev/null @@ -1,112 +0,0 @@ -#!/bin/bash -set -e - -branch="$1" -merge_base=$(git merge-base origin/master "origin/$branch") -master_git_cmd=(git log "$merge_base..origin/master" --first-parent) -# The history in back branches shouldn't be too crazy, and sometimes we have a PR -# that merges several backport commits there (3f2cba6824fddf31c30bde8c6f4f860572f4f580), -# so don't use --first-parent -branch_git_cmd=(git log "$merge_base..origin/$branch") - -# Make lists of PRs that were merged into each branch. Use first parent here, or else -# we'll get weird things like seeing older master that was merged into a PR branch -# that was then merged into master. -"${master_git_cmd[@]}" > master-log.txt -"${branch_git_cmd[@]}" > "$branch-log.txt" - -# Check for diamond merges. -diamonds_in_master=$("${master_git_cmd[@]}" --oneline --grep "Merge branch '") -diamonds_in_branch=$("${branch_git_cmd[@]}" --oneline --grep "Merge branch '") - -if [ "$diamonds_in_master" != "" ] || [ "$diamonds_in_branch" != "" ] -then - echo "$diamonds_in_master" - echo "$diamonds_in_branch" - # DO NOT ADD automated handling of diamond merges to this script. - # It is an unsustainable way to work with git, and it MUST be visible. - echo Warning: suspected diamond merges above. - echo Some commits will be missed, review these manually. -fi - -# NOTE keep in sync with ./backport.sh. -# Search for PR numbers in commit messages. First variant is normal merge, and second -# variant is squashed. Next are some backport message variants. -find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*#\([[:digit:]]\+\))$/\1/p; - s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") - -# awk is to filter out small task numbers from different task tracker, which are -# referenced by documentation commits like '* DOCSUP-824: query log (#115)'. -"${find_prs[@]}" master-log.txt | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > master-prs.txt -"${find_prs[@]}" "$branch-log.txt" | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > "$branch-prs.txt" - -# Find all master PRs that are not in branch by calculating differences of two PR lists. -grep -f "$branch-prs.txt" -F -x -v master-prs.txt > "$branch-diff-prs.txt" - -echo "$(wc -l < "$branch-diff-prs".txt) PRs differ between $branch and master." - -function github_download() -{ - local url=${1} - local file=${2} - if ! [ -f "$file" ] - then - if ! curl -H "Authorization: token $GITHUB_TOKEN" \ - -sSf "$url" \ - > "$file" - then - >&2 echo "Failed to download '$url' to '$file'. Contents: '$(cat "$file")'." - rm "$file" - return 1 - fi - sleep 0.1 - fi -} - -rm "$branch-report.tsv" &> /dev/null ||: -for pr in $(cat "$branch-diff-prs.txt") -do - # Download PR info from github. - file="pr$pr.json" - github_download "https://api.github.com/repos/ClickHouse/ClickHouse/pulls/$pr" "$file" || continue - - if ! [ "$pr" == "$(jq -r .number "$file")" ] - then - >&2 echo "Got wrong data for PR #$pr (please check and remove '$file')." - continue - fi - - action="skip" - - # First, check the changelog category. We port all bugfixes. - if jq -r .body "$file" | grep -i "^- bug[ -]*fix" > /dev/null - then - action="backport" - fi - - # Next, check the tag. They might override the decision. Checks are ordered by priority. - labels="$(jq -r .labels[].name "$file")" - if echo "$labels" | grep -x "pr-must-backport\|v$branch-must-backport" > /dev/null; then action="backport"; fi - if echo "$labels" | grep -x "v$branch-conflicts" > /dev/null; then action="conflict"; fi - if echo "$labels" | grep -x "pr-no-backport\|v$branch-no-backport" > /dev/null; then action="no-backport"; fi - # FIXME Ignore "backported" labels for now. If we can't find the backport commit, - # this means that the changelog script also won't be able to. An alternative - # way to mark PR as backported is to add an empty commit with text like - # "backported #12345", so that it can be found between tags and put in proper - # place in changelog. - #if echo "$labels" | grep -x "v$branch\|v$branch-backported" > /dev/null; then action="done"; fi - - # Find merge commit SHA for convenience - merge_sha="$(jq -r .merge_commit_sha "$file")" - - url="https://github.com/ClickHouse/ClickHouse/pull/$pr" - printf "%s\t%s\t%s\t%s\t%s\n" "$action" "$pr" "$url" "$file" "$merge_sha" >> "$branch-report.tsv" - if [ "$action" == "backport" ] - then - printf "%s\t%s\t%s\n" "$action" "$url" "$merge_sha" - fi -done - -echo "Done." From 0dc4bea16be4afcbf69b28f8b6ca77f9a1d89bac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Jun 2021 16:38:47 +0300 Subject: [PATCH 314/652] Update Context_fwd.h --- src/Interpreters/Context_fwd.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/Context_fwd.h b/src/Interpreters/Context_fwd.h index 03f5ea149c6..99c7d29f084 100644 --- a/src/Interpreters/Context_fwd.h +++ b/src/Interpreters/Context_fwd.h @@ -23,8 +23,10 @@ class Context; /// Most used types have shorter names /// TODO: in the first part of refactoring all the context pointers are non-const. using ContextPtr = std::shared_ptr; +using ContextConstPtr = ContextPtr; /// For compatibility. Use ContextPtr. using ContextMutablePtr = std::shared_ptr; using ContextWeakPtr = std::weak_ptr; +using ContextWeakConstPtr = ContextWeakPtr; /// For compatibility. Use ContextWeakPtr. using ContextWeakMutablePtr = std::weak_ptr; template @@ -49,6 +51,7 @@ protected: }; using WithContext = WithContextImpl<>; +using WithConstContext = WithContext; /// For compatibility. Use WithContext. using WithMutableContext = WithContextImpl; } From ff7f7f62008930dbfda83f122e5ad79f96205e04 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Jun 2021 17:18:35 +0300 Subject: [PATCH 315/652] try --- tests/integration/helpers/cluster.py | 30 ++++++++++++++++------- tests/integration/test_storage_s3/test.py | 2 +- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d4981b6f5de..fe753f7dad1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -57,13 +57,19 @@ def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=su return res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell, timeout=timeout) + out = res.stdout.decode('utf-8') + err = res.stderr.decode('utf-8') if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually - logging.debug(f"Stderr:\n{res.stderr.decode('utf-8')}\n") - logging.debug(f"Stdout:\n{res.stdout.decode('utf-8')}\n") - logging.debug(f"Env:\n{env}\n") + logging.debug(f"Stderr:{err}") + logging.debug(f"Stdout:{out}") + logging.debug(f"Env: {env}") if not nothrow: raise Exception(f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}") + else: + logging.debug(f"Stderr: {err}") + logging.debug(f"Stdout: {out}") + return out # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 def get_free_port(): @@ -97,7 +103,7 @@ def retry_exception(num, delay, func, exception=Exception, *args, **kwargs): def subprocess_check_call(args, detach=False, nothrow=False): # Uncomment for debugging #logging.info('run:' + ' '.join(args)) - run_and_check(args, detach=detach, nothrow=nothrow) + return run_and_check(args, detach=detach, nothrow=nothrow) def subprocess_call(args): @@ -872,9 +878,10 @@ class ClickHouseCluster: return list(handle.attrs['NetworkSettings']['Networks'].values())[0]['IPAddress'] def get_container_id(self, instance_name): - docker_id = self.get_instance_docker_id(instance_name) - handle = self.docker_client.containers.get(docker_id) - return handle.attrs['Id'] + return self.get_instance_docker_id(instance_name) + # docker_id = self.get_instance_docker_id(instance_name) + # handle = self.docker_client.containers.get(docker_id) + # return handle.attrs['Id'] def get_container_logs(self, instance_name): container_id = self.get_container_id(instance_name) @@ -882,6 +889,7 @@ class ClickHouseCluster: def exec_in_container(self, container_id, cmd, detach=False, nothrow=False, use_cli=True, **kwargs): if use_cli: + logging.debug(f"run container_id:{container_id} detach:{detach} nothrow:{nothrow} cmd: {cmd}") result = subprocess_check_call(["docker", "exec", container_id] + cmd, detach=detach, nothrow=nothrow) return result else: @@ -1054,6 +1062,7 @@ class ClickHouseCluster: for instance in ['zoo1', 'zoo2', 'zoo3']: conn = self.get_kazoo_client(instance) conn.get_children('/') + conn.stop() logging.debug("All instances of ZooKeeper Secure started") return except Exception as ex: @@ -1070,6 +1079,7 @@ class ClickHouseCluster: for instance in ['zoo1', 'zoo2', 'zoo3']: conn = self.get_kazoo_client(instance) conn.get_children('/') + conn.stop() logging.debug("All instances of ZooKeeper started") return except Exception as ex: @@ -1519,15 +1529,17 @@ class ClickHouseCluster: return zk def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1): + zk = self.get_kazoo_client(zoo_instance_name) logging.debug(f"run_kazoo_commands_with_retries: {zoo_instance_name}, {kazoo_callback}") for i in range(repeats - 1): try: - kazoo_callback(self.get_kazoo_client(zoo_instance_name)) + kazoo_callback(zk) return except KazooException as e: logging.debug(repr(e)) time.sleep(sleep_for) - kazoo_callback(self.get_kazoo_client(zoo_instance_name)) + kazoo_callback(zk) + zk.stop() def add_zookeeper_startup_command(self, command): self.pre_zookeeper_commands.append(command) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 957852fed00..acd28baa5c9 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -442,7 +442,7 @@ def run_s3_mocks(started_cluster): for mock_filename, container, port in mocks: for attempt in range(10): ping_response = started_cluster.exec_in_container(started_cluster.get_container_id(container), - ["curl", "-s", f"http://localhost:{port}/"], nothrow=True) + ["curl", "-s", f"http://localhost:{port}/"], nothrow=True) if ping_response != 'OK': if attempt == 9: assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) From 0055a924f6b5670a11aa3e7307ffe6e6c68d11ff Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 1 Jun 2021 14:23:46 +0000 Subject: [PATCH 316/652] better --- .../engines/table-engines/integrations/s3.md | 12 +-- .../mergetree-family/mergetree.md | 56 +++++++------- .../mergetree-family/mergetree.md | 74 +++++++++---------- src/Disks/S3/DiskS3.cpp | 4 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 2 +- 6 files changed, 73 insertions(+), 77 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 28ba40e281c..4e8ca628c38 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -10,7 +10,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec ## Create Table {#creating-a-table} ``` sql -CREATE TABLE s3_engine_table (name String, value UInt32) +CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression]) ``` @@ -130,7 +130,7 @@ The following settings can be set before query execution or placed into configur - `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`. - `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`. - `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`. -- `s3_single_read_retry_attempts` — The maximum number of attempts during single read. Default value is `4`. +- `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. @@ -180,21 +180,21 @@ Suppose we have several files in TSV format with the following URIs on HDFS: 1. There are several ways to make a table consisting of all six files: ``` sql -CREATE TABLE table_with_range (name String, value UInt32) +CREATE TABLE table_with_range (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}', 'CSV'); ``` 2. Another way: ``` sql -CREATE TABLE table_with_question_mark (name String, value UInt32) +CREATE TABLE table_with_question_mark (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_?', 'CSV'); ``` 3. Table consists of all the files in both directories (all files should satisfy format and schema described in query): ``` sql -CREATE TABLE table_with_asterisk (name String, value UInt32) +CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/*', 'CSV'); ``` @@ -204,7 +204,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p 4. Create table with files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: ``` sql -CREATE TABLE big_table (name String, value UInt32) +CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV'); ``` diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1c8bb60dc5e..eb87dbc6580 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -45,10 +45,10 @@ ORDER BY expr [PARTITION BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] -[TTL expr +[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ] - [WHERE conditions] - [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] [SETTINGS name=value, ...] ``` @@ -105,7 +105,7 @@ For a description of parameters, see the [CREATE query description](../../../sql - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - `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](../../../operations/settings/merge-tree-settings.md#max-partitions-to-read) in the global setting. - + **Example of Sections Setting** ``` sql @@ -158,7 +158,7 @@ When data is inserted in a table, separate data parts are created and each of th Data belonging to different partitions are separated into different parts. In the background, ClickHouse merges data parts for more efficient storage. Parts belonging to different partitions are not merged. The merge mechanism does not guarantee that all rows with the same primary key will be in the same data part. -Data parts can be stored in `Wide` or `Compact` format. In `Wide` format each column is stored in a separate file in a filesystem, in `Compact` format all columns are stored in one file. `Compact` format can be used to increase performance of small and frequent inserts. +Data parts can be stored in `Wide` or `Compact` format. In `Wide` format each column is stored in a separate file in a filesystem, in `Compact` format all columns are stored in one file. `Compact` format can be used to increase performance of small and frequent inserts. Data storing format is controlled by the `min_bytes_for_wide_part` and `min_rows_for_wide_part` settings of the table engine. If the number of bytes or rows in a data part is less then the corresponding setting's value, the part is stored in `Compact` format. Otherwise it is stored in `Wide` format. If none of these settings is set, data parts are stored in `Wide` format. @@ -457,10 +457,10 @@ ALTER TABLE example_table Table can have an expression for removal of expired rows, and multiple expressions for automatic move of parts between [disks or volumes](#table_engine-mergetree-multiple-volumes). When rows in the table expire, ClickHouse deletes all corresponding rows. For parts moving feature, all rows of a part must satisfy the movement expression criteria. ``` sql -TTL expr +TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ... - [WHERE conditions] - [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ``` Type of TTL rule may follow each TTL expression. It affects an action which is to be done once the expression is satisfied (reaches current time): @@ -472,7 +472,7 @@ Type of TTL rule may follow each TTL expression. It affects an action which is t With `WHERE` clause you may specify which of the expired rows to delete or aggregate (it cannot be applied to moves). -`GROUP BY` expression must be a prefix of the table primary key. +`GROUP BY` expression must be a prefix of the table primary key. If a column is not part of the `GROUP BY` expression and is not set explicitely in the `SET` clause, in result row it contains an occasional value from the grouped rows (as if aggregate function `any` is applied to it). @@ -506,7 +506,7 @@ Creating a table, where the rows are expired after one month. The expired rows w ``` sql CREATE TABLE table_with_where ( - d DateTime, + d DateTime, a Int ) ENGINE = MergeTree @@ -520,10 +520,10 @@ Creating a table, where expired rows are aggregated. In result rows `x` contains ``` sql CREATE TABLE table_for_aggregation ( - d DateTime, - k1 Int, - k2 Int, - x Int, + d DateTime, + k1 Int, + k2 Int, + x Int, y Int ) ENGINE = MergeTree @@ -657,7 +657,7 @@ Cofiguration examples: 0.2 - +
@@ -748,7 +748,7 @@ Configuration markup: 10000 5000 10 - 4 + 4 1000 /var/lib/clickhouse/disks/s3/ true @@ -762,22 +762,22 @@ Configuration markup: Required parameters: - `endpoint` — S3 endpoint url in `path` or `virtual hosted` [styles](https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). Endpoint url should contain bucket and root path to store data. -- `access_key_id` — S3 access key id. +- `access_key_id` — S3 access key id. - `secret_access_key` — S3 secret access key. -Optional parameters: +Optional parameters: - `region` — S3 region name. - `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`. - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`. -- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL. -- `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`. -- `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`. -- `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`. -- `single_read_retry_attempts` — Number of retry attempts in case of connection drop during read. Default value is `4`. -- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`. -- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks//`. -- `cache_enabled` — Allows to cache mark and index files on local FS. Default value is `true`. -- `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks//cache/`. +- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL. +- `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`. +- `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`. +- `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`. +- `single_read_retries` — Number of retry attempts in case of connection drop during read. Default value is `4`. +- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`. +- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks//`. +- `cache_enabled` — Allows to cache mark and index files on local FS. Default value is `true`. +- `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks//cache/`. - `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. @@ -818,6 +818,6 @@ S3 disk can be configured as `main` or `cold` storage: ``` -In case of `cold` option a data can be moved to S3 if local disk free size will be smaller than `move_factor * disk_size` or by TTL move rule. +In case of `cold` option a data can be moved to S3 if local disk free size will be smaller than `move_factor * disk_size` or by TTL move rule. [Original article](https://clickhouse.tech/docs/ru/operations/table_engines/mergetree/) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 8530af66352..b8f06c0fde0 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -37,10 +37,10 @@ ORDER BY expr [PARTITION BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] -[TTL expr +[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ] - [WHERE conditions] - [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] [SETTINGS name=value, ...] ``` @@ -51,31 +51,31 @@ ORDER BY expr - `ENGINE` — имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров. - `ORDER BY` — ключ сортировки. - + Кортеж столбцов или произвольных выражений. Пример: `ORDER BY (CounterID, EventDate)`. ClickHouse использует ключ сортировки в качестве первичного ключа, если первичный ключ не задан в секции `PRIMARY KEY`. - Чтобы отключить сортировку, используйте синтаксис `ORDER BY tuple()`. Смотрите [выбор первичного ключа](#primary-keys-and-indexes-in-queries). + Чтобы отключить сортировку, используйте синтаксис `ORDER BY tuple()`. Смотрите [выбор первичного ключа](#primary-keys-and-indexes-in-queries). - `PARTITION BY` — [ключ партиционирования](custom-partitioning-key.md). Необязательный параметр. Для партиционирования по месяцам используйте выражение `toYYYYMM(date_column)`, где `date_column` — столбец с датой типа [Date](../../../engines/table-engines/mergetree-family/mergetree.md). В этом случае имена партиций имеют формат `"YYYYMM"`. - `PRIMARY KEY` — первичный ключ, если он [отличается от ключа сортировки](#choosing-a-primary-key-that-differs-from-the-sorting-key). Необязательный параметр. - + По умолчанию первичный ключ совпадает с ключом сортировки (который задаётся секцией `ORDER BY`.) Поэтому в большинстве случаев секцию `PRIMARY KEY` отдельно указывать не нужно. - `SAMPLE BY` — выражение для сэмплирования. Необязательный параметр. - + Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. - `TTL` — список правил, определяющих длительности хранения строк, а также задающих правила перемещения частей на определённые тома или диски. Необязательный параметр. - - Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`. + + Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`. Тип правила `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` указывает действие, которое будет выполнено с частью: удаление строк (прореживание), перемещение (при выполнении условия для всех строк части) на определённый диск (`TO DISK 'xxx'`) или том (`TO VOLUME 'xxx'`), или агрегирование данных в устаревших строках. Поведение по умолчанию соответствует удалению строк (`DELETE`). В списке правил может быть указано только одно выражение с поведением `DELETE`. - + Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl) - `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree` (необязательные): @@ -149,7 +149,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID) Данные, относящиеся к разным партициям, разбиваются на разные куски. В фоновом режиме ClickHouse выполняет слияния (merge) кусков данных для более эффективного хранения. Куски, относящиеся к разным партициям не объединяются. Механизм слияния не гарантирует, что все строки с одинаковым первичным ключом окажутся в одном куске. -Куски данных могут храниться в формате `Wide` или `Compact`. В формате `Wide` каждый столбец хранится в отдельном файле, а в формате `Compact` все столбцы хранятся в одном файле. Формат `Compact` может быть полезен для повышения производительности при частом добавлении небольших объемов данных. +Куски данных могут храниться в формате `Wide` или `Compact`. В формате `Wide` каждый столбец хранится в отдельном файле, а в формате `Compact` все столбцы хранятся в одном файле. Формат `Compact` может быть полезен для повышения производительности при частом добавлении небольших объемов данных. Формат хранения определяется настройками движка `min_bytes_for_wide_part` и `min_rows_for_wide_part`. Если число байт или строк в куске данных меньше значения, указанного в соответствующей настройке, тогда этот кусок данных хранится в формате `Compact`. В противном случае кусок данных хранится в формате `Wide`. Если ни одна из настроек не задана, куски данных хранятся в формате `Wide`. @@ -211,7 +211,7 @@ ClickHouse не требует уникального первичного кл Вы можете создать таблицу без первичного ключа, используя синтаксис `ORDER BY tuple()`. В этом случае ClickHouse хранит данные в порядке вставки. Если вы хотите сохранить порядок данных при вставке данных с помощью запросов `INSERT ... SELECT`, установите [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). -Чтобы выбрать данные в первоначальном порядке, используйте +Чтобы выбрать данные в первоначальном порядке, используйте [однопоточные](../../../operations/settings/settings.md#settings-max_threads) запросы `SELECT. @@ -323,7 +323,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 Необязательный параметр `false_positive` — это вероятность получения ложноположительного срабатывания. Возможные значения: (0, 1). Значение по умолчанию: 0.025. Поддержанные типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`. - + Фильтром могут пользоваться функции: [equals](../../../engines/table-engines/mergetree-family/mergetree.md), [notEquals](../../../engines/table-engines/mergetree-family/mergetree.md), [in](../../../engines/table-engines/mergetree-family/mergetree.md), [notIn](../../../engines/table-engines/mergetree-family/mergetree.md). **Примеры** @@ -447,10 +447,10 @@ ALTER TABLE example_table Для таблицы можно задать одно выражение для устаревания данных, а также несколько выражений, по срабатывании которых данные переместятся на [некоторый диск или том](#table_engine-mergetree-multiple-volumes). Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки. ``` sql -TTL expr +TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ... - [WHERE conditions] - [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ``` За каждым TTL выражением может следовать тип действия, которое выполняется после достижения времени, соответствующего результату TTL выражения: @@ -462,13 +462,13 @@ TTL expr В секции `WHERE` можно задать условие удаления или агрегирования устаревших строк (для перемещения условие `WHERE` не применимо). -Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы. +Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы. Если колонка не является частью выражения `GROUP BY` и не задается напрямую в секции `SET`, в результирующих строках она будет содержать случайное значение, взятое из одной из сгруппированных строк (как будто к ней применяется агрегирующая функция `any`). **Примеры** -Создание таблицы с TTL: +Создание таблицы с TTL: ``` sql CREATE TABLE example_table @@ -496,7 +496,7 @@ ALTER TABLE example_table ``` sql CREATE TABLE table_with_where ( - d DateTime, + d DateTime, a Int ) ENGINE = MergeTree @@ -510,10 +510,10 @@ TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1; ``` sql CREATE TABLE table_for_aggregation ( - d DateTime, - k1 Int, - k2 Int, - x Int, + d DateTime, + k1 Int, + k2 Int, + x Int, y Int ) ENGINE = MergeTree @@ -735,7 +735,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 10000 5000 10 - 4 + 4 1000 /var/lib/clickhouse/disks/s3/ true @@ -750,23 +750,23 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' Обязательные параметры: - `endpoint` — URL точки приема запроса на стороне S3 в [форматах](https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html) `path` или `virtual hosted`. URL точки должен содержать бакет и путь к корневой директории на сервере, где хранятся данные. -- `access_key_id` — id ключа доступа к S3. +- `access_key_id` — id ключа доступа к S3. - `secret_access_key` — секретный ключ доступа к S3. -Необязательные параметры: +Необязательные параметры: - `region` — название региона S3. - `use_environment_credentials` — признак, нужно ли считывать учетные данные AWS из сетевого окружения, а также из переменных окружения `AWS_ACCESS_KEY_ID`, `AWS_SECRET_ACCESS_KEY` и `AWS_SESSION_TOKEN`, если они есть. Значение по умолчанию: `false`. - `use_insecure_imds_request` — признак, нужно ли использовать менее безопасное соединение при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию: `false`. -- `proxy` — конфигурация прокси-сервера для конечной точки S3. Каждый элемент `uri` внутри блока `proxy` должен содержать URL прокси-сервера. -- `connect_timeout_ms` — таймаут подключения к сокету в миллисекундах. Значение по умолчанию: 10 секунд. -- `request_timeout_ms` — таймаут выполнения запроса в миллисекундах. Значение по умолчанию: 5 секунд. -- `retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки. Значение по умолчанию: `10`. -- `single_read_retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки в процессе чтения. Значение по умолчанию: `4`. -- `min_bytes_for_seek` — минимальное количество байтов, которые используются для операций поиска вместо последовательного чтения. Значение по умолчанию: 1 МБайт. -- `metadata_path` — путь к локальному файловому хранилищу для хранения файлов с метаданными для S3. Значение по умолчанию: `/var/lib/clickhouse/disks//`. -- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`. -- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks//cache/`. +- `proxy` — конфигурация прокси-сервера для конечной точки S3. Каждый элемент `uri` внутри блока `proxy` должен содержать URL прокси-сервера. +- `connect_timeout_ms` — таймаут подключения к сокету в миллисекундах. Значение по умолчанию: 10 секунд. +- `request_timeout_ms` — таймаут выполнения запроса в миллисекундах. Значение по умолчанию: 5 секунд. +- `retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки. Значение по умолчанию: `10`. +- `single_read_retries` — число попыток выполнения запроса в случае возникновения ошибки в процессе чтения. Значение по умолчанию: `4`. +- `min_bytes_for_seek` — минимальное количество байтов, которые используются для операций поиска вместо последовательного чтения. Значение по умолчанию: 1 МБайт. +- `metadata_path` — путь к локальному файловому хранилищу для хранения файлов с метаданными для S3. Значение по умолчанию: `/var/lib/clickhouse/disks//`. +- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`. +- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks//cache/`. - `skip_access_check` — признак, выполнять ли проверку доступов при запуске диска. Если установлено значение `true`, то проверка не выполняется. Значение по умолчанию: `false`. @@ -807,6 +807,4 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' ``` -Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`. - - +Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`. diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index bbaf6e0d266..d36e519353d 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -126,7 +126,6 @@ public: : ReadIndirectBufferFromRemoteFS(metadata_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) - , metadata(std::move(metadata_)) , max_single_read_retries(max_single_read_retries_) , buf_size(buf_size_) { @@ -134,13 +133,12 @@ public: std::unique_ptr createReadBuffer(const String & path) override { - return std::make_unique(client_ptr, bucket, metadata.remote_fs_root_path + path, s3_max_single_read_retries, buf_size); + return std::make_unique(client_ptr, bucket, metadata.remote_fs_root_path + path, max_single_read_retries, buf_size); } private: std::shared_ptr client_ptr; const String & bucket; - DiskS3::Metadata metadata; UInt64 max_single_read_retries; size_t buf_size; }; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a10752f69ee..290a585128e 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -170,7 +170,7 @@ StorageS3Source::StorageS3Source( const ColumnsDescription & columns_, UInt64 max_block_size_, UInt64 max_single_read_retries_, - const String & compression_hint_, + const String compression_hint_, const std::shared_ptr & client_, const String & bucket_, std::shared_ptr file_iterator_) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 177e31975c6..6498e7f0a05 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -56,7 +56,7 @@ public: const ColumnsDescription & columns_, UInt64 max_block_size_, UInt64 max_single_read_retries_, - const String & compression_hint_, + const String compression_hint_, const std::shared_ptr & client_, const String & bucket, std::shared_ptr file_iterator_); From c00d55f705c7c834736674853e7a868214cac85f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 1 Jun 2021 14:28:34 +0000 Subject: [PATCH 317/652] better --- .../engines/table-engines/integrations/s3.md | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index bf9c51c6c04..709c6fb63c4 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -10,7 +10,7 @@ toc_title: S3 ## Создание таблицы {#creating-a-table} ``` sql -CREATE TABLE s3_engine_table (name String, value UInt32) +CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression]) ``` @@ -19,12 +19,12 @@ ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compressi - `path` — URL-адрес бакета с указанием пути к файлу. Поддерживает следующие подстановочные знаки в режиме "только чтение": `*`, `?`, `{abc,def}` и `{N..M}` где `N`, `M` — числа, `'abc'`, `'def'` — строки. Подробнее смотри [ниже](#wildcards-in-path). - `format` — [формат](../../../interfaces/formats.md#formats) файла. - `aws_access_key_id`, `aws_secret_access_key` - данные пользователя учетной записи [AWS](https://aws.amazon.com/ru/). Вы можете использовать их для аутентификации ваших запросов. Необязательный параметр. Если параметры учетной записи не указаны, то используются данные из конфигурационного файла. Смотрите подробнее [Использование сервиса S3 для хранения данных](../mergetree-family/mergetree.md#table_engine-mergetree-s3). -- `compression` — тип сжатия. Возможные значения: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Необязательный параметр. Если не указано, то тип сжатия определяется автоматически по расширению файла. +- `compression` — тип сжатия. Возможные значения: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Необязательный параметр. Если не указано, то тип сжатия определяется автоматически по расширению файла. **Пример** ``` sql -CREATE TABLE s3_engine_table (name String, value UInt32) +CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'gzip'); INSERT INTO s3_engine_table VALUES ('one', 1), ('two', 2), ('three', 3); SELECT * FROM s3_engine_table LIMIT 2; @@ -65,12 +65,12 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Настройки движка S3 {#s3-settings} -Перед выполнением запроса или в конфигурационном файле могут быть установлены следующие настройки: +Перед выполнением запроса или в конфигурационном файле могут быть установлены следующие настройки: -- `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`. +- `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`. - `s3_min_upload_part_size` — минимальный размер объекта для загрузки при многокомпонентной загрузке в [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Значение по умолчанию — `512 Mб`. -- `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`. -- `s3_single_read_retry_attempts` — максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. +- `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`. +- `s3_single_read_retries` — максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. Соображение безопасности: если злонамеренный пользователь попробует указать произвольные URL-адреса S3, параметр `s3_max_redirects` должен быть установлен в ноль, чтобы избежать атак [SSRF] (https://en.wikipedia.org/wiki/Server-side_request_forgery). Как альтернатива, в конфигурации сервера должен быть указан `remote_host_filter`. @@ -79,7 +79,7 @@ SELECT * FROM s3_engine_table LIMIT 2; Для точки приема запроса (которая соответствует точному префиксу URL-адреса) в конфигурационном файле могут быть заданы следующие настройки: Обязательная настройка: -- `endpoint` — указывает префикс точки приема запроса. +- `endpoint` — указывает префикс точки приема запроса. Необязательные настройки: - `access_key_id` и `secret_access_key` — указывают учетные данные для использования с данной точкой приема запроса. @@ -87,7 +87,7 @@ SELECT * FROM s3_engine_table LIMIT 2; - `use_insecure_imds_request` — признак использования менее безопасного соединения при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`. - `region` — название региона S3. - `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз. -- `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. +- `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. - `single_read_retry_attempts` — Максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. **Пример** @@ -122,21 +122,21 @@ SELECT * FROM s3_engine_table LIMIT 2; 1. Существует несколько способов создать таблицу, включающую в себя все шесть файлов: ``` sql -CREATE TABLE table_with_range (name String, value UInt32) +CREATE TABLE table_with_range (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}', 'CSV'); ``` 2. Другой способ: ``` sql -CREATE TABLE table_with_question_mark (name String, value UInt32) +CREATE TABLE table_with_question_mark (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_?', 'CSV'); ``` 3. Таблица содержит все файлы в обоих каталогах (все файлы должны соответствовать формату и схеме, описанным в запросе): ``` sql -CREATE TABLE table_with_asterisk (name String, value UInt32) +CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/*', 'CSV'); ``` @@ -145,7 +145,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p 4. Создание таблицы из файлов с именами `file-000.csv`, `file-001.csv`, … , `file-999.csv`: ``` sql -CREATE TABLE big_table (name String, value UInt32) +CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV'); ``` From ccf54556244e2517c324765d4598f21db176feda Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Jun 2021 13:45:32 +0000 Subject: [PATCH 318/652] Allow null values in postgresql protocol --- docker/test/fasttest/run.sh | 3 +++ src/Core/PostgreSQLProtocol.h | 3 ++- .../01889_postgresql_protocol_null_fields.reference | 5 +++++ .../0_stateless/01889_postgresql_protocol_null_fields.sh | 7 +++++++ 4 files changed, 17 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference create mode 100755 tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index fc73a0df0ee..cc6aeff357f 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -378,6 +378,9 @@ function run_tests 01852_jit_if 01865_jit_comparison_constant_result 01871_merge_tree_compile_expressions + + # needs psql + 01889_postgresql_protocol_null_fields ) time clickhouse-test --hung-check -j 8 --order=random --use-skip-list \ diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 7c5800f5a8f..114abc0101f 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -724,8 +724,9 @@ public: Int32 size() const override { Int32 sz = 4 + 2; // size of message + number of fields + /// If values is NULL, field size is -1 and data not added. for (const std::shared_ptr & field : row) - sz += 4 + field->size(); + sz += 4 + (field->size() > 0 ? field->size() : 0); return sz; } diff --git a/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference new file mode 100644 index 00000000000..17ee5ae55a2 --- /dev/null +++ b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference @@ -0,0 +1,5 @@ + NULL +------ + +(1 row) + diff --git a/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh new file mode 100755 index 00000000000..a1e559ca731 --- /dev/null +++ b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +psql --host localhost --port ${CLICKHOUSE_PORT_POSTGRESQL} default -c "SELECT NULL;" From 375dd1b6c4ade021bf1948584734053497aecc93 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Jun 2021 19:26:54 +0300 Subject: [PATCH 319/652] Update 01154_move_partition_long.sh --- tests/queries/0_stateless/01154_move_partition_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index f666cc929cc..66ebbacee42 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) declare -A engines engines[0]="MergeTree" -engines[1]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src', toString(randConstant()))" -engines[2]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src_' || toString(randConstant()), 'single_replica')" +engines[1]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}/src', '{replica}_' || toString(randConstant()))" +engines[2]="ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}/src_' || toString(randConstant()), '{replica}')" for ((i=0; i<16; i++)) do $CLICKHOUSE_CLIENT -q "CREATE TABLE dst_$i (p UInt64, k UInt64, v UInt64) From a4ea5783d5ca3203c8282e7b46fa7bf0c7e449cc Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Jun 2021 20:03:52 +0300 Subject: [PATCH 320/652] Update ActiveDataPartSet.cpp --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 8961b3b452f..28a4f9e2068 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -39,7 +39,8 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts, P { if (!part_info.isDisjoint(it->first)) { - LOG_ERROR(log, "Part {} intersects previous part {}. It is a bug.", name, it->first.getPartName()); + if (log) + LOG_ERROR(log, "Part {} intersects previous part {}. It is a bug.", name, it->first.getPartName()); assert(false); } ++it; @@ -65,7 +66,8 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts, P if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first)) { - LOG_ERROR(log, "Part {} intersects next part {}. It is a bug.", name, it->first.getPartName()); + if (log) + LOG_ERROR(log, "Part {} intersects next part {}. It is a bug.", name, it->first.getPartName()); assert(false); } From 332b5168883c32c368c1d6186221bd5d5e5239d5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Jun 2021 20:13:53 +0300 Subject: [PATCH 321/652] remove accidental changes --- src/Dictionaries/registerDictionaries.cpp | 2 +- src/IO/tests/gtest_perf.cpp | 32 ----------------------- 2 files changed, 1 insertion(+), 33 deletions(-) delete mode 100644 src/IO/tests/gtest_perf.cpp diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 75ecc77068f..8d24a6ea979 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -59,7 +59,7 @@ void registerDictionaries() registerDictionaryRangeHashed(factory); registerDictionaryTrie(factory); registerDictionaryFlat(factory); - // registerDictionaryHashed(factory); + registerDictionaryHashed(factory); registerDictionaryCache(factory); registerDictionaryPolygon(factory); registerDictionaryDirect(factory); diff --git a/src/IO/tests/gtest_perf.cpp b/src/IO/tests/gtest_perf.cpp deleted file mode 100644 index c870ce16a11..00000000000 --- a/src/IO/tests/gtest_perf.cpp +++ /dev/null @@ -1,32 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - -using namespace DB; - -TEST(TestPerf, qqqq) -{ - constexpr auto N = 10000000; - WriteBufferFromOwnString ss; - for (size_t i = 0; i < N; ++i) - ss << rand() << "\n"; - - auto str = ss.str(); - ReadBufferFromMemory buf(str.data(), str.size()); - - auto start = clock(); - - for (size_t i = 0; i < N; ++i) - { - UInt64 x; - readIntTextUnsafe(x, buf); - assertChar('\n', buf); - } - - std::cerr << "time: " << static_cast(clock() - start) / CLOCKS_PER_SEC << "\n"; -} From dad4d9d0c5964db6e30b44d39dd3a2a7a77383e4 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 1 Jun 2021 20:44:04 +0300 Subject: [PATCH 322/652] Create intervalLengthSum.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал функцию intervalLengthSum. --- .../reference/intervalLengthSum.md | 143 ++++++++++++++++++ 1 file changed, 143 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/intervalLengthSum.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/intervalLengthSum.md b/docs/en/sql-reference/aggregate-functions/reference/intervalLengthSum.md new file mode 100644 index 00000000000..e010afc3981 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/intervalLengthSum.md @@ -0,0 +1,143 @@ +--- +toc_priority: 146 +toc_title: intervalLengthSum +--- + +# intervalLengthSum {#agg_function-intervallengthsum} + +Calculates the sum of the length of all ranges (segments on numeric axis) without counting intersection twice. + +**Syntax** + +``` sql +intervalLengthSum(start, end) +``` + +**Arguments** + +- `start` — Starting interval value. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) or [Date](../../../sql-reference/data-types/date.md#data_type-date). +- `end` — Ending interval value. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) or [Date](../../../sql-reference/data-types/date.md#data_type-date). + +!!! info "Note" + Arguments must be of the same data type. Otherwise, an exception will be thrown. + +**Returned value** + +- Sum of the length of all ranges (segments on numeric axis) without counting intersection twice. If the arguments are of integer type, the function returns a value of the [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64) type. If the arguments are of the floating-point type, the function returns a value of the [Float64](../../../sql-reference/data-types/float.md#float32-float64) type. + +**Examples** + +1. Input table: + +``` text +┌─id─┬─start─┬─end─┐ +│ a │ 1 │ 3 │ +│ a │ 5 │ 9 │ +└────┴───────┴─────┘ +``` + +In this example, the non-intersecting intervals are summed up: + +``` sql +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM interval GROUP BY id ORDER BY id; +``` + +Result: + +``` text +┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐ +│ a │ 6 │ UInt64 │ +└────┴───────────────────────────────┴───────────────────────────────────────────┘ +``` + +2. Input table: + +``` text +┌─id─┬─start─┬─end─┐ +│ a │ 1 │ 3 │ +│ a │ 2 │ 4 │ +└────┴───────┴─────┘ +``` + +In this example, the intersecting intervals are summed up. In this case, calculates the sum of the length of ranges without counting intersection twice: + +``` sql +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM interval GROUP BY id ORDER BY id; +``` + +Result: + +``` text +┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐ +│ a │ 3 │ UInt64 │ +└────┴───────────────────────────────┴───────────────────────────────────────────┘ +``` + +3. Input table: + +``` text +┌─id─┬─start─┬─end─┐ +│ a │ 1.1 │ 3.2 │ +│ a │ 4 │ 5 │ +└────┴───────┴─────┘ +``` + +In this example, the arguments of the `Float32` type are used. In this case, the function returns a value of the `Float64` type: + +``` sql +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM fl_interval GROUP BY id ORDER BY id; +``` + +Result: + +``` text +┌─id─┬─segmentLengthSum(start, end)─┬─toTypeName(segmentLengthSum(start, end))─┐ +│ a │ 3.1 │ Float64 │ +└────┴──────────────────────────────┴──────────────────────────────────────────┘ +``` + +4. Input table: + +``` text +┌─id─┬───────────────start─┬─────────────────end─┐ +│ a │ 2020-01-01 01:12:30 │ 2020-01-01 02:50:31 │ +│ a │ 2020-01-01 03:11:22 │ 2020-01-01 03:23:31 │ +└────┴─────────────────────┴─────────────────────┘ +``` + +In this example, the arguments of the `DateTime` type are used. In this case, the function returns a value in seconds: + +``` sql +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM dt_interval GROUP BY id ORDER BY id; +``` + +Result: + +``` text +┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐ +│ a │ 6610 │ UInt64 │ +└────┴───────────────────────────────┴───────────────────────────────────────────┘ +``` + +5. Input table: + +``` text +┌─id─┬──────start─┬────────end─┐ +│ a │ 2020-01-01 │ 2020-01-04 │ +│ a │ 2020-01-12 │ 2020-01-18 │ +└────┴────────────┴────────────┘ +``` + +In this example, the arguments of the `Date` type are used. In this case, the function returns a value in days: + +``` sql +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM date_interval GROUP BY id ORDER BY id; +``` + +Result: + +``` text +┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐ +│ a │ 9 │ UInt64 │ +└────┴───────────────────────────────┴───────────────────────────────────────────┘ +``` From c1f79fb0467062b788fa419beeff9e9c815daf8e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Jun 2021 17:45:03 +0000 Subject: [PATCH 323/652] Add test to parallel skip list --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 6a278316387..d770860cde5 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -729,6 +729,7 @@ "01821_table_comment", "01710_projection_fetch", "01870_modulo_partition_key", - "01870_buffer_flush" // creates database + "01870_buffer_flush", // creates database + "01889_postgresql_protocol_null_fields" ] } From 0ac21f14a54652733e5b46b222cad6727a4bebfe Mon Sep 17 00:00:00 2001 From: MyroTk Date: Tue, 1 Jun 2021 21:07:42 +0200 Subject: [PATCH 324/652] Adding snapshots for output comparison. --- .../snapshots/common.py.tests.snapshot | 6282 +++++++++++++++++ 1 file changed, 6282 insertions(+) create mode 100644 tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot diff --git a/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot new file mode 100644 index 00000000000..6e3848b9e68 --- /dev/null +++ b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot @@ -0,0 +1,6282 @@ +I_check_plus_with_Int128_max_and_min_value = r""" +plus(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) plus(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +-170141183460469231731687303715884105728 -170141183460469231731687303715884105727 +""" + +I_check_plus_with_Int256_max_and_min_value = r""" +plus(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) plus(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 -57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_plus_with_UInt128_max_and_min_value = r""" +plus(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) plus(toUInt128(\'0\'), toUInt128(1)) +0 1 +""" + +I_check_plus_with_UInt256_max_and_min_value = r""" +plus(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) plus(toUInt256(\'0\'), toUInt256(1)) +0 1 +""" + +I_check_minus_with_Int128_max_and_min_value = r""" +minus(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) minus(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105726 170141183460469231731687303715884105727 +""" + +I_check_minus_with_Int256_max_and_min_value = r""" +minus(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) minus(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819966 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_minus_with_UInt128_max_and_min_value = r""" +minus(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) minus(toUInt128(\'0\'), toUInt128(1)) +-2 -1 +""" + +I_check_minus_with_UInt256_max_and_min_value = r""" +minus(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) minus(toUInt256(\'0\'), toUInt256(1)) +-2 -1 +""" + +I_check_multiply_with_Int128_max_and_min_value = r""" +multiply(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) multiply(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_multiply_with_Int256_max_and_min_value = r""" +multiply(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) multiply(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_multiply_with_UInt128_max_and_min_value = r""" +multiply(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) multiply(toUInt128(\'0\'), toUInt128(1)) +340282366920938463463374607431768211455 0 +""" + +I_check_multiply_with_UInt256_max_and_min_value = r""" +multiply(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) multiply(toUInt256(\'0\'), toUInt256(1)) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_divide_with_Int128_max_and_min_value = r""" +divide(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) divide(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +1.7014118346046923e38 -1.7014118346046923e38 +""" + +I_check_divide_with_Int256_max_and_min_value = r""" +divide(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) divide(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +5.78960446186581e76 -5.78960446186581e76 +""" + +I_check_divide_with_UInt128_max_and_min_value = r""" +divide(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) divide(toUInt128(\'0\'), toUInt128(1)) +3.402823669209385e38 0 +""" + +I_check_divide_with_UInt256_max_and_min_value = r""" +divide(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) divide(toUInt256(\'0\'), toUInt256(1)) +1.157920892373162e77 0 +""" + +I_check_intDiv_with_Int128_max_and_min_value = r""" +intDiv(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) intDiv(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_intDiv_with_Int256_max_and_min_value = r""" +intDiv(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) intDiv(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_intDiv_with_UInt128_max_and_min_value = r""" +intDiv(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) intDiv(toUInt128(\'0\'), toUInt128(1)) +340282366920938463463374607431768211455 0 +""" + +I_check_intDiv_with_UInt256_max_and_min_value = r""" +intDiv(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) intDiv(toUInt256(\'0\'), toUInt256(1)) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_intDivOrZero_with_Int128_max_and_min_value = r""" +intDivOrZero(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) intDivOrZero(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_intDivOrZero_with_Int256_max_and_min_value = r""" +intDivOrZero(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) intDivOrZero(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_intDivOrZero_with_UInt128_max_and_min_value = r""" +intDivOrZero(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) intDivOrZero(toUInt128(\'0\'), toUInt128(1)) +340282366920938463463374607431768211455 0 +""" + +I_check_intDivOrZero_with_UInt256_max_and_min_value = r""" +intDivOrZero(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) intDivOrZero(toUInt256(\'0\'), toUInt256(1)) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_modulo_with_Int128_max_and_min_value = r""" +modulo(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) modulo(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +0 0 +""" + +I_check_modulo_with_Int256_max_and_min_value = r""" +modulo(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) modulo(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +0 0 +""" + +I_check_modulo_with_UInt128_max_and_min_value = r""" +modulo(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) modulo(toUInt128(\'0\'), toUInt128(1)) +0 0 +""" + +I_check_modulo_with_UInt256_max_and_min_value = r""" +modulo(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) modulo(toUInt256(\'0\'), toUInt256(1)) +0 0 +""" + +I_check_moduloOrZero_with_Int128_max_and_min_value = r""" +moduloOrZero(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) moduloOrZero(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +0 0 +""" + +I_check_moduloOrZero_with_Int256_max_and_min_value = r""" +moduloOrZero(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) moduloOrZero(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +0 0 +""" + +I_check_moduloOrZero_with_UInt128_max_and_min_value = r""" +moduloOrZero(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) moduloOrZero(toUInt128(\'0\'), toUInt128(1)) +0 0 +""" + +I_check_moduloOrZero_with_UInt256_max_and_min_value = r""" +moduloOrZero(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) moduloOrZero(toUInt256(\'0\'), toUInt256(1)) +0 0 +""" + +I_check_negate_with_Int128_max_and_min_value = r""" +negate(toInt128(\'170141183460469231731687303715884105727\')) negate(toInt128(\'-170141183460469231731687303715884105728\')) +-170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_negate_with_Int256_max_and_min_value = r""" +negate(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) negate(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_negate_with_UInt128_max_and_min_value = r""" +negate(toUInt128(\'340282366920938463463374607431768211455\')) negate(toUInt128(\'0\')) +1 0 +""" + +I_check_negate_with_UInt256_max_and_min_value = r""" +negate(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) negate(toUInt256(\'0\')) +1 0 +""" + +I_check_abs_with_Int128_max_and_min_value = r""" +abs(toInt128(\'170141183460469231731687303715884105727\')) abs(toInt128(\'-170141183460469231731687303715884105728\')) +170141183460469231731687303715884105727 170141183460469231731687303715884105728 +""" + +I_check_abs_with_Int256_max_and_min_value = r""" +abs(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) abs(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +57896044618658097711785492504343953926634992332820282019728792003956564819967 57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_abs_with_UInt128_max_and_min_value = r""" +abs(toUInt128(\'340282366920938463463374607431768211455\')) abs(toUInt128(\'0\')) +340282366920938463463374607431768211455 0 +""" + +I_check_abs_with_UInt256_max_and_min_value = r""" +abs(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) abs(toUInt256(\'0\')) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_the_table_output_of_plus_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +2 +""" + +I_check_the_table_output_of_plus_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +2 +""" + +I_check_the_table_output_of_plus_with_UInt128 = r""" +a +0 +1 +2 +""" + +I_check_the_table_output_of_plus_with_UInt256 = r""" +a +0 +1 +2 +""" + +I_check_the_table_output_of_minus_with_Int128 = r""" +a +0 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_minus_with_Int256 = r""" +a +0 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_minus_with_UInt128 = r""" +a +0 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_minus_with_UInt256 = r""" +a +0 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_multiply_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_multiply_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_multiply_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_multiply_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_divide_with_Int128 = r""" +a +-170141183460469231722463931679029329921 +1 +170141183460469231722463931679029329921 +""" + +I_check_the_table_output_of_divide_with_Int256 = r""" +a +-57896044618658097702369839901263932781391731748390190090761097376371310592000 +1 +57896044618658097702369839901263932781391731748390190090761097376371310592000 +""" + +I_check_the_table_output_of_divide_with_UInt128 = r""" +a +0 +1 +340282366920938463426481119284349108225 +""" + +I_check_the_table_output_of_divide_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_output_of_intDiv_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_intDiv_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_intDiv_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_intDiv_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_intDivOrZero_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_intDivOrZero_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_intDivOrZero_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_intDivOrZero_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_modulo_with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_modulo_with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_modulo_with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_modulo_with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_negate_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +-1 +""" + +I_check_the_table_output_of_negate_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +-1 +""" + +I_check_the_table_output_of_negate_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_negate_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_abs_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_abs_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_abs_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_abs_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_gcd_with_Int128 = r""" +a +1 +""" + +I_check_the_table_output_of_gcd_with_Int256 = r""" +a +1 +""" + +I_check_the_table_output_of_gcd_with_UInt128 = r""" +a +1 +1 +""" + +I_check_the_table_output_of_gcd_with_UInt256 = r""" +a +1 +1 +""" + +I_check_the_table_output_of_lcm_with_Int128 = r""" +a +1 +""" + +I_check_the_table_output_of_lcm_with_Int256 = r""" +a +1 +""" + +I_check_the_table_output_of_lcm_with_UInt128 = r""" +a +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_lcm_with_UInt256 = r""" +a +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_for_output_of_negate_with_Decimal256 = r""" +a +-1 +""" + +I_check_the_table_for_output_of_abs_with_Decimal256 = r""" +a +1 +""" + +Inline___Int128___arrayPopBack_ = r""" +arrayPopBack(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2] +""" + +Table___Int128___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___Int128___arrayPopFront_ = r""" +arrayPopFront(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[2,1] +""" + +Table___Int128___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___Int128___arraySort_ = r""" +arraySort(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayReverseSort_ = r""" +arrayReverseSort(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayDistinct_ = r""" +arrayDistinct(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayEnumerate_ = r""" +arrayEnumerate(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,1,1] +""" + +Table___Int128___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___Int128___arrayReverse_ = r""" +arrayReverse(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___Int128___reverse_ = r""" +reverse(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___reverse_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayFlatten_ = r""" +arrayFlatten(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayCompact_ = r""" +arrayCompact(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[6] +""" + +Table___Int128___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___Int128___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[5,4,3] +""" + +Table___Int128___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___Int128___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,3,3] +""" + +Table___Int128___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___Int128___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,1,1] +""" + +Table___Int128___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___Int128___arrayConcat__toInt128__3____toInt128__2____toInt128__1____ = r""" +arrayConcat(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___Int128___arrayConcat__toInt128__3____toInt128__2____toInt128__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___Int128___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1] +""" + +Table___Int128___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___Int128___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[[0,0,0]] +""" + +Table___Int128___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___Int128___arrayZip__toInt128__1____ = r""" +arrayZip(array(toInt128(\'1\')), array(toInt128(\'3\'))) +[(1,3)] +""" + +Table___Int128___arrayZip__toInt128__1____ = r""" +a +[(1,1)] +""" + +Inline___Int128___empty_ = r""" +empty(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +0 +""" + +Table___Int128___empty_ = r""" +a +0 +""" + +Inline___Int128___notEmpty_ = r""" +notEmpty(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___notEmpty_ = r""" +a +1 +""" + +Inline___Int128___length_ = r""" +length(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___length_ = r""" +a +3 +""" + +Inline___Int128___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___Int128___arrayUniq_ = r""" +arrayUniq(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayUniq_ = r""" +a +3 +""" + +Inline___Int128___arrayJoin_ = r""" +arrayJoin(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +2 +1 +""" + +Table___Int128___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___Int128___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___Int128___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +0 +""" + +Table___Int128___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___Int128___arrayMin_ = r""" +arrayMin(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayMin_ = r""" +a +1 +""" + +Inline___Int128___arrayMax_ = r""" +arrayMax(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayMax_ = r""" +a +3 +""" + +Inline___Int128___arraySum_ = r""" +arraySum(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +6 +""" + +Table___Int128___arraySum_ = r""" +a +6 +""" + +Inline___Int128___arrayAvg_ = r""" +arrayAvg(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +2 +""" + +Table___Int128___arrayAvg_ = r""" +a +2 +""" + +Inline___Int128___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayReduce__max___ = r""" +a +3 +""" + +Inline___Int128___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___Int128___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___Int128___hasAll__toInt128__3____toInt128__2____toInt128__1_____ = r""" +hasAll(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___hasAll__toInt128__3____toInt128__2____toInt128__1_____ = r""" +a +1 +""" + +Inline___Int128___hasAny__toInt128__2____toInt128__1_____ = r""" +hasAny(array(toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___hasAny__toInt128__2____toInt128__1_____ = r""" +a +1 +""" + +Inline___Int128___hasSubstr__toInt128__2____toInt128__1_____ = r""" +hasSubstr(array(toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +0 +""" + +Table___Int128___hasSubstr__toInt128__2____toInt128__1_____ = r""" +a +0 +""" + +Table___Int128___arrayDifference_ = r""" +a +""" + +Table___Int128___arrayCumSum_ = r""" +a +""" + +Table___Int128___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___Int128___arrayElement = r""" +arrayElement(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), 1) +3 +""" + +Table___Int128___arrayElement = r""" +a +3 +""" + +Inline___Int128___arrayPushBack = r""" +arrayPushBack(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), toInt128(\'1\')) +[3,2,1,1] +""" + +Table___Int128___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___Int128___arrayPushFront = r""" +arrayPushFront(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), toInt128(\'1\')) +[1,3,2,1] +""" + +Table___Int128___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___Int128___arrayResize = r""" +arrayResize(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), 1) +[3] +""" + +Table___Int128___arrayResize = r""" +a +[3] +""" + +Inline___Int128___arraySlice = r""" +arraySlice(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), 1) +[3,2,1] +""" + +Table___Int128___arraySlice = r""" +a +[3,2,1] +""" + +Inline___Int128___has = r""" +has(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), NULL) +0 +""" + +Table___Int128___has = r""" +a +0 +""" + +Inline___Int128___indexOf = r""" +indexOf(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), NULL) +0 +""" + +Table___Int128___indexOf = r""" +a +0 +""" + +Inline___Int128___countEqual = r""" +countEqual(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), NULL) +0 +""" + +Table___Int128___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_Int128_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_Int128_on_a_table = r""" +a +1 +""" + +untuple_with_Int128_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_Int128_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_Int128_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_Int128_on_a_table = r""" +a +""" + +mapSubtract_with_Int128_on_a_table = r""" +a +""" + +mapPopulateSeries_with_Int128_on_a_table = r""" +a +""" + +mapContains_with_Int128_on_a_table = r""" +a +1 +""" + +mapKeys_with_Int128_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_Int128_on_a_table = r""" +a +[1,2] +""" + +Inline___Int256___arrayPopBack_ = r""" +arrayPopBack(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2] +""" + +Table___Int256___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___Int256___arrayPopFront_ = r""" +arrayPopFront(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[2,1] +""" + +Table___Int256___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___Int256___arraySort_ = r""" +arraySort(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayReverseSort_ = r""" +arrayReverseSort(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayDistinct_ = r""" +arrayDistinct(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayEnumerate_ = r""" +arrayEnumerate(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,1,1] +""" + +Table___Int256___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___Int256___arrayReverse_ = r""" +arrayReverse(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___Int256___reverse_ = r""" +reverse(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___reverse_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayFlatten_ = r""" +arrayFlatten(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayCompact_ = r""" +arrayCompact(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[6] +""" + +Table___Int256___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___Int256___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[5,4,3] +""" + +Table___Int256___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___Int256___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,3,3] +""" + +Table___Int256___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___Int256___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,1,1] +""" + +Table___Int256___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___Int256___arrayConcat__toInt256__3____toInt256__2____toInt256__1____ = r""" +arrayConcat(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___Int256___arrayConcat__toInt256__3____toInt256__2____toInt256__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___Int256___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1] +""" + +Table___Int256___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___Int256___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[[0,0,0]] +""" + +Table___Int256___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___Int256___arrayZip__toInt256__1____ = r""" +arrayZip(array(toInt256(\'1\')), array(toInt256(\'3\'))) +[(1,3)] +""" + +Table___Int256___arrayZip__toInt256__1____ = r""" +a +[(1,1)] +""" + +Inline___Int256___empty_ = r""" +empty(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +0 +""" + +Table___Int256___empty_ = r""" +a +0 +""" + +Inline___Int256___notEmpty_ = r""" +notEmpty(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___notEmpty_ = r""" +a +1 +""" + +Inline___Int256___length_ = r""" +length(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___length_ = r""" +a +3 +""" + +Inline___Int256___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___Int256___arrayUniq_ = r""" +arrayUniq(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayUniq_ = r""" +a +3 +""" + +Inline___Int256___arrayJoin_ = r""" +arrayJoin(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +2 +1 +""" + +Table___Int256___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___Int256___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___Int256___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +0 +""" + +Table___Int256___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___Int256___arrayMin_ = r""" +arrayMin(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayMin_ = r""" +a +1 +""" + +Inline___Int256___arrayMax_ = r""" +arrayMax(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayMax_ = r""" +a +3 +""" + +Inline___Int256___arraySum_ = r""" +arraySum(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +6 +""" + +Table___Int256___arraySum_ = r""" +a +6 +""" + +Inline___Int256___arrayAvg_ = r""" +arrayAvg(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +2 +""" + +Table___Int256___arrayAvg_ = r""" +a +2 +""" + +Inline___Int256___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayReduce__max___ = r""" +a +3 +""" + +Inline___Int256___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___Int256___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___Int256___hasAll__toInt256__3____toInt256__2____toInt256__1_____ = r""" +hasAll(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___hasAll__toInt256__3____toInt256__2____toInt256__1_____ = r""" +a +1 +""" + +Inline___Int256___hasAny__toInt256__2____toInt256__1_____ = r""" +hasAny(array(toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___hasAny__toInt256__2____toInt256__1_____ = r""" +a +1 +""" + +Inline___Int256___hasSubstr__toInt256__2____toInt256__1_____ = r""" +hasSubstr(array(toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +0 +""" + +Table___Int256___hasSubstr__toInt256__2____toInt256__1_____ = r""" +a +0 +""" + +Table___Int256___arrayDifference_ = r""" +a +""" + +Table___Int256___arrayCumSum_ = r""" +a +""" + +Table___Int256___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___Int256___arrayElement = r""" +arrayElement(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), 1) +3 +""" + +Table___Int256___arrayElement = r""" +a +3 +""" + +Inline___Int256___arrayPushBack = r""" +arrayPushBack(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), toInt256(\'1\')) +[3,2,1,1] +""" + +Table___Int256___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___Int256___arrayPushFront = r""" +arrayPushFront(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), toInt256(\'1\')) +[1,3,2,1] +""" + +Table___Int256___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___Int256___arrayResize = r""" +arrayResize(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), 1) +[3] +""" + +Table___Int256___arrayResize = r""" +a +[3] +""" + +Inline___Int256___arraySlice = r""" +arraySlice(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), 1) +[3,2,1] +""" + +Table___Int256___arraySlice = r""" +a +[3,2,1] +""" + +Inline___Int256___has = r""" +has(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), NULL) +0 +""" + +Table___Int256___has = r""" +a +0 +""" + +Inline___Int256___indexOf = r""" +indexOf(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), NULL) +0 +""" + +Table___Int256___indexOf = r""" +a +0 +""" + +Inline___Int256___countEqual = r""" +countEqual(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), NULL) +0 +""" + +Table___Int256___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_Int256_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_Int256_on_a_table = r""" +a +1 +""" + +untuple_with_Int256_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_Int256_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_Int256_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_Int256_on_a_table = r""" +a +""" + +mapSubtract_with_Int256_on_a_table = r""" +a +""" + +mapPopulateSeries_with_Int256_on_a_table = r""" +a +""" + +mapContains_with_Int256_on_a_table = r""" +a +1 +""" + +mapKeys_with_Int256_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_Int256_on_a_table = r""" +a +[1,2] +""" + +Inline___UInt128___arrayPopBack_ = r""" +arrayPopBack(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2] +""" + +Table___UInt128___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___UInt128___arrayPopFront_ = r""" +arrayPopFront(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[2,1] +""" + +Table___UInt128___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___UInt128___arraySort_ = r""" +arraySort(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayReverseSort_ = r""" +arrayReverseSort(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayDistinct_ = r""" +arrayDistinct(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayEnumerate_ = r""" +arrayEnumerate(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,1,1] +""" + +Table___UInt128___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___UInt128___arrayReverse_ = r""" +arrayReverse(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___reverse_ = r""" +reverse(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___reverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayFlatten_ = r""" +arrayFlatten(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayCompact_ = r""" +arrayCompact(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[6] +""" + +Table___UInt128___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___UInt128___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[5,4,3] +""" + +Table___UInt128___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___UInt128___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,3,3] +""" + +Table___UInt128___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___UInt128___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,1,1] +""" + +Table___UInt128___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___UInt128___arrayConcat__toUInt128__3____toUInt128__2____toUInt128__1____ = r""" +arrayConcat(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___UInt128___arrayConcat__toUInt128__3____toUInt128__2____toUInt128__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___UInt128___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1] +""" + +Table___UInt128___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___UInt128___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[[0,0,0]] +""" + +Table___UInt128___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___UInt128___arrayZip__toUInt128__1____ = r""" +arrayZip(array(toUInt128(\'1\')), array(toUInt128(\'3\'))) +[(1,3)] +""" + +Table___UInt128___arrayZip__toUInt128__1____ = r""" +a +[(1,1)] +""" + +Inline___UInt128___empty_ = r""" +empty(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +0 +""" + +Table___UInt128___empty_ = r""" +a +0 +""" + +Inline___UInt128___notEmpty_ = r""" +notEmpty(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___notEmpty_ = r""" +a +1 +""" + +Inline___UInt128___length_ = r""" +length(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___length_ = r""" +a +3 +""" + +Inline___UInt128___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___UInt128___arrayUniq_ = r""" +arrayUniq(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayUniq_ = r""" +a +3 +""" + +Inline___UInt128___arrayJoin_ = r""" +arrayJoin(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +2 +1 +""" + +Table___UInt128___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___UInt128___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___UInt128___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +0 +""" + +Table___UInt128___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___UInt128___arrayMin_ = r""" +arrayMin(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayMin_ = r""" +a +1 +""" + +Inline___UInt128___arrayMax_ = r""" +arrayMax(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayMax_ = r""" +a +3 +""" + +Inline___UInt128___arraySum_ = r""" +arraySum(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +6 +""" + +Table___UInt128___arraySum_ = r""" +a +6 +""" + +Inline___UInt128___arrayAvg_ = r""" +arrayAvg(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +2 +""" + +Table___UInt128___arrayAvg_ = r""" +a +2 +""" + +Inline___UInt128___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayReduce__max___ = r""" +a +3 +""" + +Inline___UInt128___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___UInt128___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___UInt128___hasAll__toUInt128__3____toUInt128__2____toUInt128__1_____ = r""" +hasAll(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___hasAll__toUInt128__3____toUInt128__2____toUInt128__1_____ = r""" +a +1 +""" + +Inline___UInt128___hasAny__toUInt128__2____toUInt128__1_____ = r""" +hasAny(array(toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___hasAny__toUInt128__2____toUInt128__1_____ = r""" +a +1 +""" + +Inline___UInt128___hasSubstr__toUInt128__2____toUInt128__1_____ = r""" +hasSubstr(array(toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +0 +""" + +Table___UInt128___hasSubstr__toUInt128__2____toUInt128__1_____ = r""" +a +0 +""" + +Table___UInt128___arrayDifference_ = r""" +a +""" + +Table___UInt128___arrayCumSum_ = r""" +a +""" + +Table___UInt128___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___UInt128___arrayElement = r""" +arrayElement(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), 1) +3 +""" + +Table___UInt128___arrayElement = r""" +a +3 +""" + +Inline___UInt128___arrayPushBack = r""" +arrayPushBack(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), toUInt128(\'1\')) +[3,2,1,1] +""" + +Table___UInt128___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___UInt128___arrayPushFront = r""" +arrayPushFront(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), toUInt128(\'1\')) +[1,3,2,1] +""" + +Table___UInt128___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___UInt128___arrayResize = r""" +arrayResize(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), 1) +[3] +""" + +Table___UInt128___arrayResize = r""" +a +[3] +""" + +Inline___UInt128___arraySlice = r""" +arraySlice(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), 1) +[3,2,1] +""" + +Table___UInt128___arraySlice = r""" +a +[3,2,1] +""" + +Inline___UInt128___has = r""" +has(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), NULL) +0 +""" + +Table___UInt128___has = r""" +a +0 +""" + +Inline___UInt128___indexOf = r""" +indexOf(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), NULL) +0 +""" + +Table___UInt128___indexOf = r""" +a +0 +""" + +Inline___UInt128___countEqual = r""" +countEqual(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), NULL) +0 +""" + +Table___UInt128___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_UInt128_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_UInt128_on_a_table = r""" +a +1 +""" + +untuple_with_UInt128_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_UInt128_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_UInt128_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_UInt128_on_a_table = r""" +a +""" + +mapSubtract_with_UInt128_on_a_table = r""" +a +""" + +mapPopulateSeries_with_UInt128_on_a_table = r""" +a +""" + +mapContains_with_UInt128_on_a_table = r""" +a +1 +""" + +mapKeys_with_UInt128_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_UInt128_on_a_table = r""" +a +[1,2] +""" + +Inline___UInt256___arrayPopBack_ = r""" +arrayPopBack(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2] +""" + +Table___UInt256___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___UInt256___arrayPopFront_ = r""" +arrayPopFront(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[2,1] +""" + +Table___UInt256___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___UInt256___arraySort_ = r""" +arraySort(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayReverseSort_ = r""" +arrayReverseSort(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayDistinct_ = r""" +arrayDistinct(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayEnumerate_ = r""" +arrayEnumerate(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,1,1] +""" + +Table___UInt256___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___UInt256___arrayReverse_ = r""" +arrayReverse(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___reverse_ = r""" +reverse(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___reverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayFlatten_ = r""" +arrayFlatten(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayCompact_ = r""" +arrayCompact(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[6] +""" + +Table___UInt256___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___UInt256___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[5,4,3] +""" + +Table___UInt256___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___UInt256___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,3,3] +""" + +Table___UInt256___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___UInt256___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,1,1] +""" + +Table___UInt256___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___UInt256___arrayConcat__toUInt256__3____toUInt256__2____toUInt256__1____ = r""" +arrayConcat(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___UInt256___arrayConcat__toUInt256__3____toUInt256__2____toUInt256__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___UInt256___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1] +""" + +Table___UInt256___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___UInt256___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[[0,0,0]] +""" + +Table___UInt256___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___UInt256___arrayZip__toUInt256__1____ = r""" +arrayZip(array(toUInt256(\'1\')), array(toUInt256(\'3\'))) +[(1,3)] +""" + +Table___UInt256___arrayZip__toUInt256__1____ = r""" +a +[(1,1)] +""" + +Inline___UInt256___empty_ = r""" +empty(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +0 +""" + +Table___UInt256___empty_ = r""" +a +0 +""" + +Inline___UInt256___notEmpty_ = r""" +notEmpty(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___notEmpty_ = r""" +a +1 +""" + +Inline___UInt256___length_ = r""" +length(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___length_ = r""" +a +3 +""" + +Inline___UInt256___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___UInt256___arrayUniq_ = r""" +arrayUniq(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayUniq_ = r""" +a +3 +""" + +Inline___UInt256___arrayJoin_ = r""" +arrayJoin(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +2 +1 +""" + +Table___UInt256___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___UInt256___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___UInt256___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +0 +""" + +Table___UInt256___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___UInt256___arrayMin_ = r""" +arrayMin(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayMin_ = r""" +a +1 +""" + +Inline___UInt256___arrayMax_ = r""" +arrayMax(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayMax_ = r""" +a +3 +""" + +Inline___UInt256___arraySum_ = r""" +arraySum(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +6 +""" + +Table___UInt256___arraySum_ = r""" +a +6 +""" + +Inline___UInt256___arrayAvg_ = r""" +arrayAvg(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +2 +""" + +Table___UInt256___arrayAvg_ = r""" +a +2 +""" + +Inline___UInt256___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayReduce__max___ = r""" +a +3 +""" + +Inline___UInt256___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___UInt256___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___UInt256___hasAll__toUInt256__3____toUInt256__2____toUInt256__1_____ = r""" +hasAll(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___hasAll__toUInt256__3____toUInt256__2____toUInt256__1_____ = r""" +a +1 +""" + +Inline___UInt256___hasAny__toUInt256__2____toUInt256__1_____ = r""" +hasAny(array(toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___hasAny__toUInt256__2____toUInt256__1_____ = r""" +a +1 +""" + +Inline___UInt256___hasSubstr__toUInt256__2____toUInt256__1_____ = r""" +hasSubstr(array(toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +0 +""" + +Table___UInt256___hasSubstr__toUInt256__2____toUInt256__1_____ = r""" +a +0 +""" + +Table___UInt256___arrayDifference_ = r""" +a +""" + +Table___UInt256___arrayCumSum_ = r""" +a +""" + +Table___UInt256___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___UInt256___arrayElement = r""" +arrayElement(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), 1) +3 +""" + +Table___UInt256___arrayElement = r""" +a +3 +""" + +Inline___UInt256___arrayPushBack = r""" +arrayPushBack(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), toUInt256(\'1\')) +[3,2,1,1] +""" + +Table___UInt256___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___UInt256___arrayPushFront = r""" +arrayPushFront(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), toUInt256(\'1\')) +[1,3,2,1] +""" + +Table___UInt256___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___UInt256___arrayResize = r""" +arrayResize(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), 1) +[3] +""" + +Table___UInt256___arrayResize = r""" +a +[3] +""" + +Inline___UInt256___arraySlice = r""" +arraySlice(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), 1) +[3,2,1] +""" + +Table___UInt256___arraySlice = r""" +a +[3,2,1] +""" + +Inline___UInt256___has = r""" +has(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), NULL) +0 +""" + +Table___UInt256___has = r""" +a +0 +""" + +Inline___UInt256___indexOf = r""" +indexOf(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), NULL) +0 +""" + +Table___UInt256___indexOf = r""" +a +0 +""" + +Inline___UInt256___countEqual = r""" +countEqual(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), NULL) +0 +""" + +Table___UInt256___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_UInt256_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_UInt256_on_a_table = r""" +a +1 +""" + +untuple_with_UInt256_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_UInt256_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_UInt256_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_UInt256_on_a_table = r""" +a +""" + +mapSubtract_with_UInt256_on_a_table = r""" +a +""" + +mapPopulateSeries_with_UInt256_on_a_table = r""" +a +""" + +mapContains_with_UInt256_on_a_table = r""" +a +1 +""" + +mapKeys_with_UInt256_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_UInt256_on_a_table = r""" +a +[1,2] +""" + +Inline___Decimal256_0____arrayPopBack_ = r""" +arrayPopBack(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2] +""" + +Table___Decimal256_0____arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___Decimal256_0____arrayPopFront_ = r""" +arrayPopFront(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[2,1] +""" + +Table___Decimal256_0____arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___Decimal256_0____arraySort_ = r""" +arraySort(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arraySort_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayReverseSort_ = r""" +arrayReverseSort(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayDistinct_ = r""" +arrayDistinct(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayEnumerate_ = r""" +arrayEnumerate(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,1,1] +""" + +Table___Decimal256_0____arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___Decimal256_0____arrayReverse_ = r""" +arrayReverse(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____reverse_ = r""" +reverse(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____reverse_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayFlatten_ = r""" +arrayFlatten(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayCompact_ = r""" +arrayCompact(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[6] +""" + +Table___Decimal256_0____arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___Decimal256_0____arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[5,4,3] +""" + +Table___Decimal256_0____arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___Decimal256_0____arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,3,3] +""" + +Table___Decimal256_0____arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___Decimal256_0____arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,1,1] +""" + +Table___Decimal256_0____arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___Decimal256_0____arrayConcat__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0___ = r""" +arrayConcat(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1,3,2,1] +""" + +Table___Decimal256_0____arrayConcat__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0___ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___Decimal256_0____arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1] +""" + +Table___Decimal256_0____arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___Decimal256_0____arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[[0,0,0]] +""" + +Table___Decimal256_0____arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___Decimal256_0____arrayZip__toDecimal256__1__0___ = r""" +arrayZip(array(toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0))) +[(1,3)] +""" + +Table___Decimal256_0____arrayZip__toDecimal256__1__0___ = r""" +a +[(1,1)] +""" + +Inline___Decimal256_0____empty_ = r""" +empty(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +0 +""" + +Table___Decimal256_0____empty_ = r""" +a +0 +""" + +Inline___Decimal256_0____notEmpty_ = r""" +notEmpty(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____notEmpty_ = r""" +a +1 +""" + +Inline___Decimal256_0____length_ = r""" +length(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____length_ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___Decimal256_0____arrayUniq_ = r""" +arrayUniq(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____arrayUniq_ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayJoin_ = r""" +arrayJoin(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +2 +1 +""" + +Table___Decimal256_0____arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___Decimal256_0____arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___Decimal256_0____arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +0 +""" + +Table___Decimal256_0____arrayAll_x____x__1_ = r""" +a +0 +""" + +Table___Decimal256_0____arrayMin_ = r""" +a +""" + +Table___Decimal256_0____arrayMax_ = r""" +a +""" + +Table___Decimal256_0____arraySum_ = r""" +a +""" + +Table___Decimal256_0____arrayAvg_ = r""" +a +""" + +Inline___Decimal256_0____arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____arrayReduce__max___ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___Decimal256_0____hasAll__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0____ = r""" +hasAll(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____hasAll__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0____ = r""" +a +1 +""" + +Inline___Decimal256_0____hasAny__toDecimal256__2__0___toDecimal256__1__0____ = r""" +hasAny(array(toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____hasAny__toDecimal256__2__0___toDecimal256__1__0____ = r""" +a +1 +""" + +Inline___Decimal256_0____hasSubstr__toDecimal256__2__0___toDecimal256__1__0____ = r""" +hasSubstr(array(toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +0 +""" + +Table___Decimal256_0____hasSubstr__toDecimal256__2__0___toDecimal256__1__0____ = r""" +a +0 +""" + +Table___Decimal256_0____arrayDifference_ = r""" +a +""" + +Table___Decimal256_0____arrayCumSum_ = r""" +a +""" + +Table___Decimal256_0____arrayCumSumNonNegative_ = r""" +a +""" + +Inline___Decimal256_0____arrayElement = r""" +arrayElement(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), 1) +3 +""" + +Table___Decimal256_0____arrayElement = r""" +a +3 +""" + +Inline___Decimal256_0____arrayPushBack = r""" +arrayPushBack(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), toDecimal256(\'1\', 0)) +[3,2,1,1] +""" + +Table___Decimal256_0____arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___Decimal256_0____arrayPushFront = r""" +arrayPushFront(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), toDecimal256(\'1\', 0)) +[1,3,2,1] +""" + +Table___Decimal256_0____arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___Decimal256_0____arrayResize = r""" +arrayResize(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), 1) +[3] +""" + +Table___Decimal256_0____arrayResize = r""" +a +[3] +""" + +Inline___Decimal256_0____arraySlice = r""" +arraySlice(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), 1) +[3,2,1] +""" + +Table___Decimal256_0____arraySlice = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____has = r""" +has(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), NULL) +0 +""" + +Table___Decimal256_0____has = r""" +a +0 +""" + +Inline___Decimal256_0____indexOf = r""" +indexOf(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), NULL) +0 +""" + +Table___Decimal256_0____indexOf = r""" +a +0 +""" + +Inline___Decimal256_0____countEqual = r""" +countEqual(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), NULL) +0 +""" + +Table___Decimal256_0____countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_Decimal256_0__on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_Decimal256_0__on_a_table = r""" +a +1 +""" + +untuple_with_Decimal256_0__on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_Decimal256_0__on_a_table = r""" +a +2 +""" + +Creating_a_map_with_Decimal256_0__on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_Decimal256_0__on_a_table = r""" +a +""" + +mapSubtract_with_Decimal256_0__on_a_table = r""" +a +""" + +mapPopulateSeries_with_Decimal256_0__on_a_table = r""" +a +""" + +mapContains_with_Decimal256_0__on_a_table = r""" +a +1 +""" + +mapKeys_with_Decimal256_0__on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_Decimal256_0__on_a_table = r""" +a +[1,2] +""" + +I_check_equals_with_Int128 = r""" +equals(toInt128(1), toInt128(1)) equals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +1 0 +""" + +I_check_equals_with_Int256 = r""" +equals(toInt256(1), toInt256(1)) equals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 0 +""" + +I_check_equals_with_UInt128 = r""" +equals(toUInt128(1), toUInt128(1)) equals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +1 0 +""" + +I_check_equals_with_UInt256 = r""" +equals(toUInt256(1), toUInt256(1)) equals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +1 0 +""" + +I_check_notEquals_with_Int128 = r""" +notEquals(toInt128(1), toInt128(1)) notEquals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +0 1 +""" + +I_check_notEquals_with_Int256 = r""" +notEquals(toInt256(1), toInt256(1)) notEquals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 1 +""" + +I_check_notEquals_with_UInt128 = r""" +notEquals(toUInt128(1), toUInt128(1)) notEquals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +0 1 +""" + +I_check_notEquals_with_UInt256 = r""" +notEquals(toUInt256(1), toUInt256(1)) notEquals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +0 1 +""" + +I_check_less_with_Int128 = r""" +less(toInt128(1), toInt128(1)) less(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +0 0 +""" + +I_check_less_with_Int256 = r""" +less(toInt256(1), toInt256(1)) less(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 0 +""" + +I_check_less_with_UInt128 = r""" +less(toUInt128(1), toUInt128(1)) less(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +0 0 +""" + +I_check_less_with_UInt256 = r""" +less(toUInt256(1), toUInt256(1)) less(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +0 0 +""" + +I_check_greater_with_Int128 = r""" +greater(toInt128(1), toInt128(1)) greater(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +0 1 +""" + +I_check_greater_with_Int256 = r""" +greater(toInt256(1), toInt256(1)) greater(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 1 +""" + +I_check_greater_with_UInt128 = r""" +greater(toUInt128(1), toUInt128(1)) greater(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +0 1 +""" + +I_check_greater_with_UInt256 = r""" +greater(toUInt256(1), toUInt256(1)) greater(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +0 1 +""" + +I_check_lessOrEquals_with_Int128 = r""" +lessOrEquals(toInt128(1), toInt128(1)) lessOrEquals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +1 0 +""" + +I_check_lessOrEquals_with_Int256 = r""" +lessOrEquals(toInt256(1), toInt256(1)) lessOrEquals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 0 +""" + +I_check_lessOrEquals_with_UInt128 = r""" +lessOrEquals(toUInt128(1), toUInt128(1)) lessOrEquals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +1 0 +""" + +I_check_lessOrEquals_with_UInt256 = r""" +lessOrEquals(toUInt256(1), toUInt256(1)) lessOrEquals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +1 0 +""" + +I_check_greaterOrEquals_with_Int128 = r""" +greaterOrEquals(toInt128(1), toInt128(1)) greaterOrEquals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +1 1 +""" + +I_check_greaterOrEquals_with_Int256 = r""" +greaterOrEquals(toInt256(1), toInt256(1)) greaterOrEquals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 1 +""" + +I_check_greaterOrEquals_with_UInt128 = r""" +greaterOrEquals(toUInt128(1), toUInt128(1)) greaterOrEquals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +1 1 +""" + +I_check_greaterOrEquals_with_UInt256 = r""" +greaterOrEquals(toUInt256(1), toUInt256(1)) greaterOrEquals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +1 1 +""" + +I_check_the_table_for_the_output_of_equals_with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_equals_with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_equals_with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_equals_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_less_with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_less_with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_less_with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_less_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_equals_with_Decimal256 = r""" +equals(toDecimal256(1, 0), toDecimal256(1, 0)) equals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 0 +""" + +I_check_notEquals_with_Decimal256 = r""" +notEquals(toDecimal256(1, 0), toDecimal256(1, 0)) notEquals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 1 +""" + +I_check_less_with_Decimal256 = r""" +less(toDecimal256(1, 0), toDecimal256(1, 0)) less(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 0 +""" + +I_check_greater_with_Decimal256 = r""" +greater(toDecimal256(1, 0), toDecimal256(1, 0)) greater(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 1 +""" + +I_check_lessOrEquals_with_Decimal256 = r""" +lessOrEquals(toDecimal256(1, 0), toDecimal256(1, 0)) lessOrEquals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 0 +""" + +I_check_greaterOrEquals_with_Decimal256 = r""" +greaterOrEquals(toDecimal256(1, 0), toDecimal256(1, 0)) greaterOrEquals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_the_table_for_the_output_of_equals_with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_Decimal256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_less_with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_Decimal256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_Decimal256 = r""" +a +0 +1 +1 +""" + +I_check_exp__with_Int128_using_max_and_min = r""" +exp(toInt128(\'170141183460469231731687303715884105727\')) exp(toInt128(\'-170141183460469231731687303715884105728\')) +inf 0 +""" + +I_check_exp__with_Int256_using_max_and_min = r""" +exp(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) exp(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf 0 +""" + +I_check_exp__with_UInt128_using_max_and_min = r""" +exp(toUInt128(\'340282366920938463463374607431768211455\')) exp(toUInt128(\'0\')) +inf 1 +""" + +I_check_exp__with_UInt256_using_max_and_min = r""" +exp(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) exp(toUInt256(\'0\')) +inf 1 +""" + +I_check_log__with_Int128_using_max_and_min = r""" +log(toInt128(\'170141183460469231731687303715884105727\')) log(toInt128(\'-170141183460469231731687303715884105728\')) +88.02969193111305 nan +""" + +I_check_log__with_Int256_using_max_and_min = r""" +log(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +176.75253104278605 nan +""" + +I_check_log__with_UInt128_using_max_and_min = r""" +log(toUInt128(\'340282366920938463463374607431768211455\')) log(toUInt128(\'0\')) +88.722839111673 -inf +""" + +I_check_log__with_UInt256_using_max_and_min = r""" +log(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log(toUInt256(\'0\')) +177.445678223346 -inf +""" + +I_check_ln__with_Int128_using_max_and_min = r""" +log(toInt128(\'170141183460469231731687303715884105727\')) log(toInt128(\'-170141183460469231731687303715884105728\')) +88.02969193111305 nan +""" + +I_check_ln__with_Int256_using_max_and_min = r""" +log(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +176.75253104278605 nan +""" + +I_check_ln__with_UInt128_using_max_and_min = r""" +log(toUInt128(\'340282366920938463463374607431768211455\')) log(toUInt128(\'0\')) +88.722839111673 -inf +""" + +I_check_ln__with_UInt256_using_max_and_min = r""" +log(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log(toUInt256(\'0\')) +177.445678223346 -inf +""" + +I_check_exp2__with_Int128_using_max_and_min = r""" +exp2(toInt128(\'170141183460469231731687303715884105727\')) exp2(toInt128(\'-170141183460469231731687303715884105728\')) +inf 0 +""" + +I_check_exp2__with_Int256_using_max_and_min = r""" +exp2(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) exp2(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf 0 +""" + +I_check_exp2__with_UInt128_using_max_and_min = r""" +exp2(toUInt128(\'340282366920938463463374607431768211455\')) exp2(toUInt128(\'0\')) +inf 1 +""" + +I_check_exp2__with_UInt256_using_max_and_min = r""" +exp2(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) exp2(toUInt256(\'0\')) +inf 1 +""" + +I_check_log2__with_Int128_using_max_and_min = r""" +log2(toInt128(\'170141183460469231731687303715884105727\')) log2(toInt128(\'-170141183460469231731687303715884105728\')) +127 nan +""" + +I_check_log2__with_Int256_using_max_and_min = r""" +log2(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log2(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +255 nan +""" + +I_check_log2__with_UInt128_using_max_and_min = r""" +log2(toUInt128(\'340282366920938463463374607431768211455\')) log2(toUInt128(\'0\')) +128 -inf +""" + +I_check_log2__with_UInt256_using_max_and_min = r""" +log2(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log2(toUInt256(\'0\')) +256 -inf +""" + +I_check_exp10__with_Int128_using_max_and_min = r""" +exp10(toInt128(\'170141183460469231731687303715884105727\')) exp10(toInt128(\'-170141183460469231731687303715884105728\')) +inf 0 +""" + +I_check_exp10__with_Int256_using_max_and_min = r""" +exp10(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) exp10(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf 0 +""" + +I_check_exp10__with_UInt128_using_max_and_min = r""" +exp10(toUInt128(\'340282366920938463463374607431768211455\')) exp10(toUInt128(\'0\')) +inf 1 +""" + +I_check_exp10__with_UInt256_using_max_and_min = r""" +exp10(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) exp10(toUInt256(\'0\')) +inf 1 +""" + +I_check_log10__with_Int128_using_max_and_min = r""" +log10(toInt128(\'170141183460469231731687303715884105727\')) log10(toInt128(\'-170141183460469231731687303715884105728\')) +38.23080944932561 nan +""" + +I_check_log10__with_Int256_using_max_and_min = r""" +log10(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log10(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +76.7626488943152 nan +""" + +I_check_log10__with_UInt128_using_max_and_min = r""" +log10(toUInt128(\'340282366920938463463374607431768211455\')) log10(toUInt128(\'0\')) +38.53183944498959 -inf +""" + +I_check_log10__with_UInt256_using_max_and_min = r""" +log10(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log10(toUInt256(\'0\')) +77.06367888997919 -inf +""" + +I_check_sqrt__with_Int128_using_max_and_min = r""" +sqrt(toInt128(\'170141183460469231731687303715884105727\')) sqrt(toInt128(\'-170141183460469231731687303715884105728\')) +13043817825332783000 nan +""" + +I_check_sqrt__with_Int256_using_max_and_min = r""" +sqrt(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sqrt(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +2.4061596916800453e38 nan +""" + +I_check_sqrt__with_UInt128_using_max_and_min = r""" +sqrt(toUInt128(\'340282366920938463463374607431768211455\')) sqrt(toUInt128(\'0\')) +18446744073709552000 0 +""" + +I_check_sqrt__with_UInt256_using_max_and_min = r""" +sqrt(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sqrt(toUInt256(\'0\')) +3.402823669209385e38 0 +""" + +I_check_cbrt__with_Int128_using_max_and_min = r""" +cbrt(toInt128(\'170141183460469231731687303715884105727\')) cbrt(toInt128(\'-170141183460469231731687303715884105728\')) +5541191377756.637 -5541191377756.637 +""" + +I_check_cbrt__with_Int256_using_max_and_min = r""" +cbrt(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) cbrt(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +3.8685626227668134e25 -3.8685626227668134e25 +""" + +I_check_cbrt__with_UInt128_using_max_and_min = r""" +cbrt(toUInt128(\'340282366920938463463374607431768211455\')) cbrt(toUInt128(\'0\')) +6981463658331.56 0 +""" + +I_check_cbrt__with_UInt256_using_max_and_min = r""" +cbrt(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) cbrt(toUInt256(\'0\')) +4.874083481260429e25 0 +""" + +I_check_erf__with_Int128_using_max_and_min = r""" +erf(toInt128(\'170141183460469231731687303715884105727\')) erf(toInt128(\'-170141183460469231731687303715884105728\')) +1 -1 +""" + +I_check_erf__with_Int256_using_max_and_min = r""" +erf(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) erf(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 -1 +""" + +I_check_erf__with_UInt128_using_max_and_min = r""" +erf(toUInt128(\'340282366920938463463374607431768211455\')) erf(toUInt128(\'0\')) +1 0 +""" + +I_check_erf__with_UInt256_using_max_and_min = r""" +erf(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) erf(toUInt256(\'0\')) +1 0 +""" + +I_check_erfc__with_Int128_using_max_and_min = r""" +erfc(toInt128(\'170141183460469231731687303715884105727\')) erfc(toInt128(\'-170141183460469231731687303715884105728\')) +0 2 +""" + +I_check_erfc__with_Int256_using_max_and_min = r""" +erfc(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) erfc(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 2 +""" + +I_check_erfc__with_UInt128_using_max_and_min = r""" +erfc(toUInt128(\'340282366920938463463374607431768211455\')) erfc(toUInt128(\'0\')) +0 1 +""" + +I_check_erfc__with_UInt256_using_max_and_min = r""" +erfc(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) erfc(toUInt256(\'0\')) +0 1 +""" + +I_check_lgamma__with_Int128_using_max_and_min = r""" +lgamma(toInt128(\'170141183460469231731687303715884105727\')) lgamma(toInt128(\'-170141183460469231731687303715884105728\')) +1.4807334781359624e40 -1.4807334781359624e40 +""" + +I_check_lgamma__with_Int256_using_max_and_min = r""" +lgamma(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) lgamma(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1.0175376379095233e79 -1.0175376379095233e79 +""" + +I_check_lgamma__with_UInt128_using_max_and_min = r""" +lgamma(toUInt128(\'340282366920938463463374607431768211455\')) lgamma(toUInt128(\'0\')) +2.985053532594476e40 inf +""" + +I_check_lgamma__with_UInt256_using_max_and_min = r""" +lgamma(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) lgamma(toUInt256(\'0\')) +2.0431013718376458e79 inf +""" + +I_check_tgamma__with_Int128_using_max_and_min = r""" +tgamma(toInt128(\'170141183460469231731687303715884105727\')) tgamma(toInt128(\'-170141183460469231731687303715884105728\')) +inf nan +""" + +I_check_tgamma__with_Int256_using_max_and_min = r""" +tgamma(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) tgamma(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf nan +""" + +I_check_tgamma__with_UInt128_using_max_and_min = r""" +tgamma(toUInt128(\'340282366920938463463374607431768211455\')) tgamma(toUInt128(\'0\')) +inf inf +""" + +I_check_tgamma__with_UInt256_using_max_and_min = r""" +tgamma(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) tgamma(toUInt256(\'0\')) +inf inf +""" + +I_check_sin__with_Int128_using_max_and_min = r""" +sin(toInt128(\'170141183460469231731687303715884105727\')) sin(toInt128(\'-170141183460469231731687303715884105728\')) +0.6233855129558702 -0.6233855129558702 +""" + +I_check_sin__with_Int256_using_max_and_min = r""" +sin(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sin(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0.9751222164851924 -0.9751222164851924 +""" + +I_check_sin__with_UInt128_using_max_and_min = r""" +sin(toUInt128(\'340282366920938463463374607431768211455\')) sin(toUInt128(\'0\')) +0.9748685162860586 0 +""" + +I_check_sin__with_UInt256_using_max_and_min = r""" +sin(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sin(toUInt256(\'0\')) +0.4323066100553458 0 +""" + +I_check_cos__with_Int128_using_max_and_min = r""" +cos(toInt128(\'170141183460469231731687303715884105727\')) cos(toInt128(\'-170141183460469231731687303715884105728\')) +0.78191463871496 0.78191463871496 +""" + +I_check_cos__with_Int256_using_max_and_min = r""" +cos(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) cos(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0.22166791133812228 0.22166791133812228 +""" + +I_check_cos__with_UInt128_using_max_and_min = r""" +cos(toUInt128(\'340282366920938463463374607431768211455\')) cos(toUInt128(\'0\')) +0.22278100447349308 1 +""" + +I_check_cos__with_UInt256_using_max_and_min = r""" +cos(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) cos(toUInt256(\'0\')) +-0.9017266741659887 1 +""" + +I_check_tan__with_Int128_using_max_and_min = r""" +tan(toInt128(\'170141183460469231731687303715884105727\')) tan(toInt128(\'-170141183460469231731687303715884105728\')) +0.7972552016424389 -0.7972552016424389 +""" + +I_check_tan__with_Int256_using_max_and_min = r""" +tan(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) tan(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +4.399022892392326 -4.399022892392326 +""" + +I_check_tan__with_UInt128_using_max_and_min = r""" +tan(toUInt128(\'340282366920938463463374607431768211455\')) tan(toUInt128(\'0\')) +4.375905022019283 0 +""" + +I_check_tan__with_UInt256_using_max_and_min = r""" +tan(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) tan(toUInt256(\'0\')) +-0.4794208959773628 0 +""" + +I_check_asin__with_Int128_using_max_and_min = r""" +asin(toInt128(\'170141183460469231731687303715884105727\')) asin(toInt128(\'-170141183460469231731687303715884105728\')) +nan nan +""" + +I_check_asin__with_Int256_using_max_and_min = r""" +asin(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) asin(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +nan nan +""" + +I_check_asin__with_UInt128_using_max_and_min = r""" +asin(toUInt128(\'340282366920938463463374607431768211455\')) asin(toUInt128(\'0\')) +nan 0 +""" + +I_check_asin__with_UInt256_using_max_and_min = r""" +asin(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) asin(toUInt256(\'0\')) +nan 0 +""" + +I_check_acos__with_Int128_using_max_and_min = r""" +acos(toInt128(\'170141183460469231731687303715884105727\')) acos(toInt128(\'-170141183460469231731687303715884105728\')) +nan nan +""" + +I_check_acos__with_Int256_using_max_and_min = r""" +acos(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) acos(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +nan nan +""" + +I_check_acos__with_UInt128_using_max_and_min = r""" +acos(toUInt128(\'340282366920938463463374607431768211455\')) acos(toUInt128(\'0\')) +nan 1.5707963267948966 +""" + +I_check_acos__with_UInt256_using_max_and_min = r""" +acos(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) acos(toUInt256(\'0\')) +nan 1.5707963267948966 +""" + +I_check_atan__with_Int128_using_max_and_min = r""" +atan(toInt128(\'170141183460469231731687303715884105727\')) atan(toInt128(\'-170141183460469231731687303715884105728\')) +1.5707963267948966 -1.5707963267948966 +""" + +I_check_atan__with_Int256_using_max_and_min = r""" +atan(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) atan(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1.5707963267948966 -1.5707963267948966 +""" + +I_check_atan__with_UInt128_using_max_and_min = r""" +atan(toUInt128(\'340282366920938463463374607431768211455\')) atan(toUInt128(\'0\')) +1.5707963267948966 0 +""" + +I_check_atan__with_UInt256_using_max_and_min = r""" +atan(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) atan(toUInt256(\'0\')) +1.5707963267948966 0 +""" + +I_check_cosh__with_Int128_using_max_and_min = r""" +cosh(toInt128(\'170141183460469231731687303715884105727\')) cosh(toInt128(\'-170141183460469231731687303715884105728\')) +inf inf +""" + +I_check_cosh__with_Int256_using_max_and_min = r""" +cosh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) cosh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf inf +""" + +I_check_cosh__with_UInt128_using_max_and_min = r""" +cosh(toUInt128(\'340282366920938463463374607431768211455\')) cosh(toUInt128(\'0\')) +inf 1 +""" + +I_check_cosh__with_UInt256_using_max_and_min = r""" +cosh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) cosh(toUInt256(\'0\')) +inf 1 +""" + +I_check_acosh__with_Int128_using_max_and_min = r""" +acosh(toInt128(\'170141183460469231731687303715884105727\')) acosh(toInt128(\'-170141183460469231731687303715884105728\')) +88.722839111673 nan +""" + +I_check_acosh__with_Int256_using_max_and_min = r""" +acosh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) acosh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +177.445678223346 nan +""" + +I_check_acosh__with_UInt128_using_max_and_min = r""" +acosh(toUInt128(\'340282366920938463463374607431768211455\')) acosh(toUInt128(\'0\')) +89.41598629223294 nan +""" + +I_check_acosh__with_UInt256_using_max_and_min = r""" +acosh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) acosh(toUInt256(\'0\')) +178.13882540390594 nan +""" + +I_check_sinh__with_Int128_using_max_and_min = r""" +sinh(toInt128(\'170141183460469231731687303715884105727\')) sinh(toInt128(\'-170141183460469231731687303715884105728\')) +inf -inf +""" + +I_check_sinh__with_Int256_using_max_and_min = r""" +sinh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sinh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf -inf +""" + +I_check_sinh__with_UInt128_using_max_and_min = r""" +sinh(toUInt128(\'340282366920938463463374607431768211455\')) sinh(toUInt128(\'0\')) +inf 0 +""" + +I_check_sinh__with_UInt256_using_max_and_min = r""" +sinh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sinh(toUInt256(\'0\')) +inf 0 +""" + +I_check_asinh__with_Int128_using_max_and_min = r""" +asinh(toInt128(\'170141183460469231731687303715884105727\')) asinh(toInt128(\'-170141183460469231731687303715884105728\')) +88.722839111673 -88.722839111673 +""" + +I_check_asinh__with_Int256_using_max_and_min = r""" +asinh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) asinh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +177.445678223346 -177.445678223346 +""" + +I_check_asinh__with_UInt128_using_max_and_min = r""" +asinh(toUInt128(\'340282366920938463463374607431768211455\')) asinh(toUInt128(\'0\')) +89.41598629223294 0 +""" + +I_check_asinh__with_UInt256_using_max_and_min = r""" +asinh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) asinh(toUInt256(\'0\')) +178.13882540390594 0 +""" + +I_check_tanh__with_Int128_using_max_and_min = r""" +tanh(toInt128(\'170141183460469231731687303715884105727\')) tanh(toInt128(\'-170141183460469231731687303715884105728\')) +1 -1 +""" + +I_check_tanh__with_Int256_using_max_and_min = r""" +tanh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) tanh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 -1 +""" + +I_check_tanh__with_UInt128_using_max_and_min = r""" +tanh(toUInt128(\'340282366920938463463374607431768211455\')) tanh(toUInt128(\'0\')) +1 0 +""" + +I_check_tanh__with_UInt256_using_max_and_min = r""" +tanh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) tanh(toUInt256(\'0\')) +1 0 +""" + +I_check_atanh__with_Int128_using_max_and_min = r""" +atanh(toInt128(\'170141183460469231731687303715884105727\')) atanh(toInt128(\'-170141183460469231731687303715884105728\')) +nan nan +""" + +I_check_atanh__with_Int256_using_max_and_min = r""" +atanh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) atanh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +nan nan +""" + +I_check_atanh__with_UInt128_using_max_and_min = r""" +atanh(toUInt128(\'340282366920938463463374607431768211455\')) atanh(toUInt128(\'0\')) +nan 0 +""" + +I_check_atanh__with_UInt256_using_max_and_min = r""" +atanh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) atanh(toUInt256(\'0\')) +nan 0 +""" + +I_check_log1p__with_Int128_using_max_and_min = r""" +log1p(toInt128(\'170141183460469231731687303715884105727\')) log1p(toInt128(\'-170141183460469231731687303715884105728\')) +88.02969193111305 nan +""" + +I_check_log1p__with_Int256_using_max_and_min = r""" +log1p(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log1p(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +176.75253104278605 nan +""" + +I_check_log1p__with_UInt128_using_max_and_min = r""" +log1p(toUInt128(\'340282366920938463463374607431768211455\')) log1p(toUInt128(\'0\')) +88.722839111673 0 +""" + +I_check_log1p__with_UInt256_using_max_and_min = r""" +log1p(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log1p(toUInt256(\'0\')) +177.445678223346 0 +""" + +I_check_sign__with_Int128_using_max_and_min = r""" +sign(toInt128(\'170141183460469231731687303715884105727\')) sign(toInt128(\'-170141183460469231731687303715884105728\')) +1 -1 +""" + +I_check_sign__with_Int256_using_max_and_min = r""" +sign(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sign(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 -1 +""" + +I_check_sign__with_UInt128_using_max_and_min = r""" +sign(toUInt128(\'340282366920938463463374607431768211455\')) sign(toUInt128(\'0\')) +1 0 +""" + +I_check_sign__with_UInt256_using_max_and_min = r""" +sign(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sign(toUInt256(\'0\')) +1 0 +""" + +I_check_the_outputs_of_exp__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_exp__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_log__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp2__with_Int128 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_exp2__with_Int256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_exp2__with_UInt128 = r""" +a +0 +1 +2 +""" + +I_check_the_outputs_of_exp2__with_UInt256 = r""" +a +0 +1 +2 +""" + +I_check_the_outputs_of_log2__with_Int128 = r""" +a +0 +0 +127 +""" + +I_check_the_outputs_of_log2__with_Int256 = r""" +a +0 +0 +255 +""" + +I_check_the_outputs_of_log2__with_UInt128 = r""" +a +0 +0 +128 +""" + +I_check_the_outputs_of_log2__with_UInt256 = r""" +a +0 +0 +256 +""" + +I_check_the_outputs_of_exp10__with_Int128 = r""" +a +0 +0 +10 +""" + +I_check_the_outputs_of_exp10__with_Int256 = r""" +a +0 +0 +10 +""" + +I_check_the_outputs_of_exp10__with_UInt128 = r""" +a +0 +1 +10 +""" + +I_check_the_outputs_of_exp10__with_UInt256 = r""" +a +0 +1 +10 +""" + +I_check_the_outputs_of_log10__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log10__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log10__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log10__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sqrt__with_Int128 = r""" +a +0 +1 +13043817825332783000 +""" + +I_check_the_outputs_of_sqrt__with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_sqrt__with_UInt128 = r""" +a +0 +1 +18446744073709552000 +""" + +I_check_the_outputs_of_sqrt__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erf__with_Int128 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_erf__with_Int256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_erf__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erf__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erfc__with_Int128 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_erfc__with_Int256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_erfc__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erfc__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_lgamma__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_lgamma__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_lgamma__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_lgamma__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tgamma__with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tgamma__with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tgamma__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tgamma__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_sin__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sin__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sin__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sin__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cos__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tan__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cosh__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_acosh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tanh__with_Int128 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_tanh__with_Int256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_tanh__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tanh__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_atanh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atanh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atanh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atanh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sign__with_Int128 = r""" +a +-1 +1 +1 +""" + +I_check_the_outputs_of_sign__with_Int256 = r""" +a +-1 +1 +1 +""" + +I_check_the_outputs_of_sign__with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_outputs_of_sign__with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_exp__with_Decimal256_using_max_and_min = r""" +exp(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) exp(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf 0 +""" + +I_check_log__with_Decimal256_using_max_and_min = r""" +log(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +172.69388197463743 nan +""" + +I_check_ln__with_Decimal256_using_max_and_min = r""" +log(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +172.69388197463743 nan +""" + +I_check_exp2__with_Decimal256_using_max_and_min = r""" +exp2(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) exp2(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf 0 +""" + +I_check_log2__with_Decimal256_using_max_and_min = r""" +log2(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log2(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +249.14460711655218 nan +""" + +I_check_exp10__with_Decimal256_using_max_and_min = r""" +exp10(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) exp10(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf 0 +""" + +I_check_log10__with_Decimal256_using_max_and_min = r""" +log10(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log10(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +75 nan +""" + +I_check_sqrt__with_Decimal256_using_max_and_min = r""" +sqrt(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sqrt(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +3.1622776601683794e37 nan +""" + +I_check_cbrt__with_Decimal256_using_max_and_min = r""" +cbrt(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) cbrt(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1e25 -1e25 +""" + +I_check_erf__with_Decimal256_using_max_and_min = r""" +erf(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) erf(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 -1 +""" + +I_check_erfc__with_Decimal256_using_max_and_min = r""" +erfc(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) erfc(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 2 +""" + +I_check_lgamma__with_Decimal256_using_max_and_min = r""" +lgamma(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) lgamma(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1.7169388197455342e77 -1.7169388197455342e77 +""" + +I_check_tgamma__with_Decimal256_using_max_and_min = r""" +tgamma(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) tgamma(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf nan +""" + +I_check_sin__with_Decimal256_using_max_and_min = r""" +sin(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sin(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0.66339975236386 -0.66339975236386 +""" + +I_check_cos__with_Decimal256_using_max_and_min = r""" +cos(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) cos(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-0.7482651726250322 -0.7482651726250322 +""" + +I_check_tan__with_Decimal256_using_max_and_min = r""" +tan(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) tan(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-0.8865837628611647 0.8865837628611647 +""" + +I_check_asin__with_Decimal256_using_max_and_min = r""" +asin(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) asin(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +nan nan +""" + +I_check_acos__with_Decimal256_using_max_and_min = r""" +acos(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) acos(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +nan nan +""" + +I_check_atan__with_Decimal256_using_max_and_min = r""" +atan(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) atan(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1.5707963267948966 -1.5707963267948966 +""" + +I_check_cosh__with_Decimal256_using_max_and_min = r""" +cosh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) cosh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf inf +""" + +I_check_acosh__with_Decimal256_using_max_and_min = r""" +acosh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) acosh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +173.38702915511337 nan +""" + +I_check_sinh__with_Decimal256_using_max_and_min = r""" +sinh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sinh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf -inf +""" + +I_check_asinh__with_Decimal256_using_max_and_min = r""" +asinh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) asinh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +173.38702915511337 -173.38702915511337 +""" + +I_check_tanh__with_Decimal256_using_max_and_min = r""" +tanh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) tanh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 -1 +""" + +I_check_atanh__with_Decimal256_using_max_and_min = r""" +atanh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) atanh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +nan nan +""" + +I_check_log1p__with_Decimal256_using_max_and_min = r""" +log1p(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log1p(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +172.69388197455342 nan +""" + +I_check_sign__with_Decimal256_using_max_and_min = r""" +sign(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sign(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 -1 +""" + +I_check_the_outputs_of_exp__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp2__with_Decimal256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_log2__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp10__with_Decimal256 = r""" +a +0 +0 +10 +""" + +I_check_the_outputs_of_log10__with_Decimal256 = r""" +a +0 +0 +75 +""" + +I_check_the_outputs_of_sqrt__with_Decimal256 = r""" +a +0 +1 +31622776601683794000000000000000000000 +""" + +I_check_the_outputs_of_cbrt__with_Decimal256 = r""" +a +-10000000000000000000000000 +1 +10000000000000000000000000 +""" + +I_check_the_outputs_of_erf__with_Decimal256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_erfc__with_Decimal256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_lgamma__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tgamma__with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_sin__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tanh__with_Decimal256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_atanh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sign__with_Decimal256 = r""" +a +-1 +1 +1 +""" + +I_check_ceil_with_Int128_using_min_and_max_values = r""" +ceil(toInt128(\'-170141183460469231731687303715884105728\')) ceil(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_ceil_with_Int256_using_min_and_max_values = r""" +ceil(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) ceil(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_ceil_with_UInt128_using_min_and_max_values = r""" +ceil(toUInt128(\'0\')) ceil(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_ceil_with_UInt256_using_min_and_max_values = r""" +ceil(toUInt256(\'0\')) ceil(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_floor_with_Int128_using_min_and_max_values = r""" +floor(toInt128(\'-170141183460469231731687303715884105728\')) floor(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_floor_with_Int256_using_min_and_max_values = r""" +floor(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) floor(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_floor_with_UInt128_using_min_and_max_values = r""" +floor(toUInt128(\'0\')) floor(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_floor_with_UInt256_using_min_and_max_values = r""" +floor(toUInt256(\'0\')) floor(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_trunc_with_Int128_using_min_and_max_values = r""" +trunc(toInt128(\'-170141183460469231731687303715884105728\')) trunc(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_trunc_with_Int256_using_min_and_max_values = r""" +trunc(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) trunc(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_trunc_with_UInt128_using_min_and_max_values = r""" +trunc(toUInt128(\'0\')) trunc(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_trunc_with_UInt256_using_min_and_max_values = r""" +trunc(toUInt256(\'0\')) trunc(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_round_with_Int128_using_min_and_max_values = r""" +round(toInt128(\'-170141183460469231731687303715884105728\')) round(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_round_with_Int256_using_min_and_max_values = r""" +round(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) round(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_round_with_UInt128_using_min_and_max_values = r""" +round(toUInt128(\'0\')) round(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_round_with_UInt256_using_min_and_max_values = r""" +round(toUInt256(\'0\')) round(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_roundBankers_with_Int128_using_min_and_max_values = r""" +roundBankers(toInt128(\'-170141183460469231731687303715884105728\')) roundBankers(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_roundBankers_with_Int256_using_min_and_max_values = r""" +roundBankers(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) roundBankers(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_roundBankers_with_UInt128_using_min_and_max_values = r""" +roundBankers(toUInt128(\'0\')) roundBankers(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_roundBankers_with_UInt256_using_min_and_max_values = r""" +roundBankers(toUInt256(\'0\')) roundBankers(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_roundDuration_with_Int128_using_min_and_max_values = r""" +roundDuration(toInt128(\'-170141183460469231731687303715884105728\')) roundDuration(toInt128(\'170141183460469231731687303715884105727\')) +0 36000 +""" + +I_check_roundDuration_with_Int256_using_min_and_max_values = r""" +roundDuration(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) roundDuration(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +0 36000 +""" + +I_check_roundDuration_with_UInt128_using_min_and_max_values = r""" +roundDuration(toUInt128(\'0\')) roundDuration(toUInt128(\'340282366920938463463374607431768211455\')) +0 36000 +""" + +I_check_roundDuration_with_UInt256_using_min_and_max_values = r""" +roundDuration(toUInt256(\'0\')) roundDuration(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 36000 +""" + +I_check_roundAge_with_Int128_using_min_and_max_values = r""" +roundAge(toInt128(\'-170141183460469231731687303715884105728\')) roundAge(toInt128(\'170141183460469231731687303715884105727\')) +0 55 +""" + +I_check_roundAge_with_Int256_using_min_and_max_values = r""" +roundAge(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) roundAge(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +0 55 +""" + +I_check_roundAge_with_UInt128_using_min_and_max_values = r""" +roundAge(toUInt128(\'0\')) roundAge(toUInt128(\'340282366920938463463374607431768211455\')) +0 55 +""" + +I_check_roundAge_with_UInt256_using_min_and_max_values = r""" +roundAge(toUInt256(\'0\')) roundAge(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 55 +""" + +I_select_the_output_of_ceil_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_ceil_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_ceil_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_ceil_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_floor_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_floor_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_floor_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_floor_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_trunc_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_trunc_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_trunc_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_trunc_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_round_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_round_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_round_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_round_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_roundBankers_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_roundBankers_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_roundBankers_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_roundBankers_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_roundDuration_with_Int128_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundDuration_with_Int256_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundDuration_with_UInt128_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundDuration_with_UInt256_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundAge_with_Int128_from_the_table = r""" +a +0 +17 +55 +""" + +I_select_the_output_of_roundAge_with_Int256_from_the_table = r""" +a +0 +17 +55 +""" + +I_select_the_output_of_roundAge_with_UInt128_from_the_table = r""" +a +0 +17 +55 +""" + +I_select_the_output_of_roundAge_with_UInt256_from_the_table = r""" +a +0 +17 +55 +""" + +I_check_ceil_with_Decimal256_using_min_and_max_values = r""" +ceil(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) ceil(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_floor_with_Decimal256_using_min_and_max_values = r""" +floor(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) floor(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_trunc_with_Decimal256_using_min_and_max_values = r""" +trunc(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) trunc(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_round_with_Decimal256_using_min_and_max_values = r""" +round(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) round(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_roundBankers_with_Decimal256_using_min_and_max_values = r""" +roundBankers(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) roundBankers(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_ceil_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_floor_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_trunc_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_round_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_roundBankers_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_bitAnd_with_Int128 = r""" +bitAnd(toInt128(1), 1) bitAnd(toInt128(\'170141183460469231731687303715884105727\'), 1) bitAnd(toInt128(\'-170141183460469231731687303715884105728\'), 1) +1 1 0 +""" + +I_check_bitAnd_with_Int256 = r""" +bitAnd(toInt256(1), 1) bitAnd(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitAnd(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +1 1 0 +""" + +I_check_bitAnd_with_UInt128 = r""" +bitAnd(toUInt128(1), 1) bitAnd(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitAnd(toUInt128(\'0\'), 1) +1 1 0 +""" + +I_check_bitAnd_with_UInt256 = r""" +bitAnd(toUInt256(1), 1) bitAnd(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitAnd(toUInt256(\'0\'), 1) +1 1 0 +""" + +I_check_bitOr_with_Int128 = r""" +bitOr(toInt128(1), 1) bitOr(toInt128(\'170141183460469231731687303715884105727\'), 1) bitOr(toInt128(\'-170141183460469231731687303715884105728\'), 1) +1 170141183460469231731687303715884105727 -170141183460469231731687303715884105727 +""" + +I_check_bitOr_with_Int256 = r""" +bitOr(toInt256(1), 1) bitOr(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitOr(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +1 57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_bitOr_with_UInt128 = r""" +bitOr(toUInt128(1), 1) bitOr(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitOr(toUInt128(\'0\'), 1) +1 340282366920938463463374607431768211455 1 +""" + +I_check_bitOr_with_UInt256 = r""" +bitOr(toUInt256(1), 1) bitOr(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitOr(toUInt256(\'0\'), 1) +1 115792089237316195423570985008687907853269984665640564039457584007913129639935 1 +""" + +I_check_bitXor_with_Int128 = r""" +bitXor(toInt128(1), 1) bitXor(toInt128(\'170141183460469231731687303715884105727\'), 1) bitXor(toInt128(\'-170141183460469231731687303715884105728\'), 1) +0 170141183460469231731687303715884105726 -170141183460469231731687303715884105727 +""" + +I_check_bitXor_with_Int256 = r""" +bitXor(toInt256(1), 1) bitXor(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitXor(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +0 57896044618658097711785492504343953926634992332820282019728792003956564819966 -57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_bitXor_with_UInt128 = r""" +bitXor(toUInt128(1), 1) bitXor(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitXor(toUInt128(\'0\'), 1) +0 340282366920938463463374607431768211454 1 +""" + +I_check_bitXor_with_UInt256 = r""" +bitXor(toUInt256(1), 1) bitXor(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitXor(toUInt256(\'0\'), 1) +0 115792089237316195423570985008687907853269984665640564039457584007913129639934 1 +""" + +I_check_bitShiftLeft_with_Int128 = r""" +bitShiftLeft(toInt128(1), 1) bitShiftLeft(toInt128(\'170141183460469231731687303715884105727\'), 1) bitShiftLeft(toInt128(\'-170141183460469231731687303715884105728\'), 1) +2 -2 0 +""" + +I_check_bitShiftLeft_with_Int256 = r""" +bitShiftLeft(toInt256(1), 1) bitShiftLeft(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitShiftLeft(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +2 -2 0 +""" + +I_check_bitShiftLeft_with_UInt128 = r""" +bitShiftLeft(toUInt128(1), 1) bitShiftLeft(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitShiftLeft(toUInt128(\'0\'), 1) +2 340282366920938463463374607431768211454 0 +""" + +I_check_bitShiftLeft_with_UInt256 = r""" +bitShiftLeft(toUInt256(1), 1) bitShiftLeft(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitShiftLeft(toUInt256(\'0\'), 1) +2 115792089237316195423570985008687907853269984665640564039457584007913129639934 0 +""" + +I_check_bitShiftRight_with_Int128 = r""" +bitShiftRight(toInt128(1), 1) bitShiftRight(toInt128(\'170141183460469231731687303715884105727\'), 1) bitShiftRight(toInt128(\'-170141183460469231731687303715884105728\'), 1) +0 85070591730234615865843651857942052863 -85070591730234615865843651857942052864 +""" + +I_check_bitShiftRight_with_Int256 = r""" +bitShiftRight(toInt256(1), 1) bitShiftRight(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitShiftRight(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +0 28948022309329048855892746252171976963317496166410141009864396001978282409983 -28948022309329048855892746252171976963317496166410141009864396001978282409984 +""" + +I_check_bitShiftRight_with_UInt128 = r""" +bitShiftRight(toUInt128(1), 1) bitShiftRight(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitShiftRight(toUInt128(\'0\'), 1) +0 170141183460469231731687303715884105727 0 +""" + +I_check_bitShiftRight_with_UInt256 = r""" +bitShiftRight(toUInt256(1), 1) bitShiftRight(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitShiftRight(toUInt256(\'0\'), 1) +0 57896044618658097711785492504343953926634992332820282019728792003956564819967 0 +""" + +Check_bitNot_with_Int128 = r""" +bitNot(toInt128(1)) bitNot(toInt128(\'170141183460469231731687303715884105727\')) bitNot(toInt128(\'-170141183460469231731687303715884105728\')) +-2 -170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +Check_bitNot_with_Int256 = r""" +bitNot(toInt256(1)) bitNot(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) bitNot(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +-2 -57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +Check_bitNot_with_UInt128 = r""" +bitNot(toUInt128(1)) bitNot(toUInt128(\'340282366920938463463374607431768211455\')) bitNot(toUInt128(\'0\')) +340282366920938463463374607431768211454 0 340282366920938463463374607431768211455 +""" + +Check_bitNot_with_UInt256 = r""" +bitNot(toUInt256(1)) bitNot(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) bitNot(toUInt256(\'0\')) +115792089237316195423570985008687907853269984665640564039457584007913129639934 0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +Check_bitCount_with_Int128 = r""" +bitCount(toInt128(1)) bitCount(toInt128(\'170141183460469231731687303715884105727\')) bitCount(toInt128(\'-170141183460469231731687303715884105728\')) +1 64 0 +""" + +Check_bitCount_with_Int256 = r""" +bitCount(toInt256(1)) bitCount(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) bitCount(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 64 0 +""" + +Check_bitCount_with_UInt128 = r""" +bitCount(toUInt128(1)) bitCount(toUInt128(\'340282366920938463463374607431768211455\')) bitCount(toUInt128(\'0\')) +1 64 0 +""" + +Check_bitCount_with_UInt256 = r""" +bitCount(toUInt256(1)) bitCount(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) bitCount(toUInt256(\'0\')) +1 64 0 +""" + +I_check_the_table_with_values_of_bitAnd_and_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitAnd_and_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitAnd_and_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitAnd_and_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitOr_and_Int128 = r""" +a +-170141183460469231731687303715884105727 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_with_values_of_bitOr_and_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_with_values_of_bitOr_and_UInt128 = r""" +a +1 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_with_values_of_bitOr_and_UInt256 = r""" +a +1 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_with_values_of_bitXor_and_Int128 = r""" +a +-170141183460469231731687303715884105727 +0 +170141183460469231731687303715884105726 +""" + +I_check_the_table_with_values_of_bitXor_and_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +0 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +""" + +I_check_the_table_with_values_of_bitXor_and_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211454 +""" + +I_check_the_table_with_values_of_bitXor_and_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_Int128 = r""" +a +-2 +0 +2 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_Int256 = r""" +a +-2 +0 +2 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_UInt128 = r""" +a +0 +2 +340282366920938463463374607431768211454 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_UInt256 = r""" +a +0 +2 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_Int128 = r""" +a +-85070591730234615865843651857942052864 +0 +85070591730234615865843651857942052863 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_Int256 = r""" +a +-28948022309329048855892746252171976963317496166410141009864396001978282409984 +0 +28948022309329048855892746252171976963317496166410141009864396001978282409983 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_UInt128 = r""" +a +0 +0 +170141183460469231731687303715884105727 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_UInt256 = r""" +a +0 +0 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_with_values_of_bitNot_and_Int128 = r""" +a +-170141183460469231731687303715884105728 +-2 +170141183460469231731687303715884105727 +""" + +I_check_the_table_with_values_of_bitNot_and_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-2 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_with_values_of_bitNot_and_UInt128 = r""" +a +0 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +""" + +I_check_the_table_with_values_of_bitNot_and_UInt256 = r""" +a +0 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_with_values_of_bitCount_and_Int128 = r""" +a +0 +1 +64 +""" + +I_check_the_table_with_values_of_bitCount_and_Int256 = r""" +a +0 +1 +64 +""" + +I_check_the_table_with_values_of_bitCount_and_UInt128 = r""" +a +0 +1 +64 +""" + +I_check_the_table_with_values_of_bitCount_and_UInt256 = r""" +a +0 +1 +64 +""" + +I_check_isNull__with_Int128_using_min_and_max = r""" +isNull(toInt128(\'-170141183460469231731687303715884105728\')) isNull(toInt128(\'170141183460469231731687303715884105727\')) +0 0 +""" + +I_check_isNull__with_Int256_using_min_and_max = r""" +isNull(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) isNull(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +0 0 +""" + +I_check_isNull__with_UInt128_using_min_and_max = r""" +isNull(toUInt128(\'0\')) isNull(toUInt128(\'340282366920938463463374607431768211455\')) +0 0 +""" + +I_check_isNull__with_UInt256_using_min_and_max = r""" +isNull(toUInt256(\'0\')) isNull(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 0 +""" + +I_check_isNotNull__with_Int128_using_min_and_max = r""" +isNotNull(toInt128(\'-170141183460469231731687303715884105728\')) isNotNull(toInt128(\'170141183460469231731687303715884105727\')) +1 1 +""" + +I_check_isNotNull__with_Int256_using_min_and_max = r""" +isNotNull(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) isNotNull(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +1 1 +""" + +I_check_isNotNull__with_UInt128_using_min_and_max = r""" +isNotNull(toUInt128(\'0\')) isNotNull(toUInt128(\'340282366920938463463374607431768211455\')) +1 1 +""" + +I_check_isNotNull__with_UInt256_using_min_and_max = r""" +isNotNull(toUInt256(\'0\')) isNotNull(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +1 1 +""" + +I_check_coalesce__with_Int128_using_min_and_max = r""" +coalesce(toInt128(\'-170141183460469231731687303715884105728\')) coalesce(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_coalesce__with_Int256_using_min_and_max = r""" +coalesce(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) coalesce(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_coalesce__with_UInt128_using_min_and_max = r""" +coalesce(toUInt128(\'0\')) coalesce(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_coalesce__with_UInt256_using_min_and_max = r""" +coalesce(toUInt256(\'0\')) coalesce(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_assumeNotNull__with_Int128_using_min_and_max = r""" +assumeNotNull(toInt128(\'-170141183460469231731687303715884105728\')) assumeNotNull(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_assumeNotNull__with_Int256_using_min_and_max = r""" +assumeNotNull(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) assumeNotNull(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_assumeNotNull__with_UInt128_using_min_and_max = r""" +assumeNotNull(toUInt128(\'0\')) assumeNotNull(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_assumeNotNull__with_UInt256_using_min_and_max = r""" +assumeNotNull(toUInt256(\'0\')) assumeNotNull(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_toNullable__with_Int128_using_min_and_max = r""" +toNullable(toInt128(\'-170141183460469231731687303715884105728\')) toNullable(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_toNullable__with_Int256_using_min_and_max = r""" +toNullable(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) toNullable(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_toNullable__with_UInt128_using_min_and_max = r""" +toNullable(toUInt128(\'0\')) toNullable(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_toNullable__with_UInt256_using_min_and_max = r""" +toNullable(toUInt256(\'0\')) toNullable(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_ifNull_1__with_Int128_using_min_and_max = r""" +ifNull(1, toInt128(\'-170141183460469231731687303715884105728\')) ifNull(1, toInt128(\'170141183460469231731687303715884105727\')) +1 1 +""" + +I_check_ifNull_1__with_Int256_using_min_and_max = r""" +ifNull(1, toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) ifNull(1, toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +1 1 +""" + +I_check_ifNull_1__with_UInt128_using_min_and_max = r""" +ifNull(1, toUInt128(\'0\')) ifNull(1, toUInt128(\'340282366920938463463374607431768211455\')) +1 1 +""" + +I_check_ifNull_1__with_UInt256_using_min_and_max = r""" +ifNull(1, toUInt256(\'0\')) ifNull(1, toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +1 1 +""" + +I_check_nullIf_1__with_Int128_using_min_and_max = r""" +nullIf(1, toInt128(\'-170141183460469231731687303715884105728\')) nullIf(1, toInt128(\'170141183460469231731687303715884105727\')) +1 1 +""" + +I_check_nullIf_1__with_Int256_using_min_and_max = r""" +nullIf(1, toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) nullIf(1, toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +1 1 +""" + +I_check_nullIf_1__with_UInt128_using_min_and_max = r""" +nullIf(1, toUInt128(\'0\')) nullIf(1, toUInt128(\'340282366920938463463374607431768211455\')) +1 1 +""" + +I_check_nullIf_1__with_UInt256_using_min_and_max = r""" +nullIf(1, toUInt256(\'0\')) nullIf(1, toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +1 1 +""" + +I_check_isNull__with_Int128_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNull__with_Int256_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNull__with_UInt128_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNull__with_UInt256_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNotNull__with_Int128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_isNotNull__with_Int256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_isNotNull__with_UInt128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_isNotNull__with_UInt256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_coalesce__with_Int128_on_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_coalesce__with_Int256_on_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_coalesce__with_UInt128_on_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_coalesce__with_UInt256_on_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_assumeNotNull__with_Int128_on_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_assumeNotNull__with_Int256_on_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_assumeNotNull__with_UInt128_on_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_assumeNotNull__with_UInt256_on_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_toNullable__with_Int128_on_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_toNullable__with_Int256_on_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_toNullable__with_UInt128_on_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_toNullable__with_UInt256_on_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_ifNull_1__with_Int128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_ifNull_1__with_Int256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_ifNull_1__with_UInt128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_ifNull_1__with_UInt256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_nullIf_1__with_Int128_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_nullIf_1__with_Int256_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_nullIf_1__with_UInt128_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_nullIf_1__with_UInt256_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_isNull__with_Decimal256_using_min_and_max = r""" +isNull(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) isNull(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 0 +""" + +I_check_isNotNull__with_Decimal256_using_min_and_max = r""" +isNotNull(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) isNotNull(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_coalesce__with_Decimal256_using_min_and_max = r""" +coalesce(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) coalesce(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_assumeNotNull__with_Decimal256_using_min_and_max = r""" +assumeNotNull(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) assumeNotNull(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_toNullable__with_Decimal256_using_min_and_max = r""" +toNullable(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) toNullable(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_ifNull_1__with_Decimal256_using_min_and_max = r""" +ifNull(1, toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) ifNull(1, toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_nullIf_1__with_Decimal256_using_min_and_max = r""" +nullIf(1, toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) nullIf(1, toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_isNull__with_Decimal256_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNotNull__with_Decimal256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_coalesce__with_Decimal256_on_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_assumeNotNull__with_Decimal256_on_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_toNullable__with_Decimal256_on_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_ifNull_1__with_Decimal256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_nullIf_1__with_Decimal256_on_the_table = r""" +a +1 +1 +\N +""" + From f57863b9749e2086a57aa9583172a90c7d95eced Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 2 Jun 2021 01:03:08 +0300 Subject: [PATCH 325/652] fix tests --- programs/copier/ClusterCopier.cpp | 6 ++---- .../test_cluster_copier/data/.gitkeep | 0 .../test_cluster_copier/test_three_nodes.py | 15 +++++++-------- .../test_cluster_copier/test_two_nodes.py | 18 +++++++++--------- 4 files changed, 18 insertions(+), 21 deletions(-) delete mode 100644 tests/integration/test_cluster_copier/data/.gitkeep diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d7e37a1a5d3..55cb1696ae8 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -742,10 +742,8 @@ ASTPtr ClusterCopier::removeAliasMaterializedAndTTLColumnsFromCreateQuery(const auto new_columns_list = std::make_shared(); new_columns_list->set(new_columns_list->columns, new_columns); - if (const auto * indices = query_ast->as()->columns_list->indices) - new_columns_list->set(new_columns_list->indices, indices->clone()); - if (const auto * projections = query_ast->as()->columns_list->projections) - new_columns_list->set(new_columns_list->projections, projections->clone()); + + /// Skip indices and projections are not needed, because distributed table doesn't support it. new_query.replace(new_query.columns_list, new_columns_list); diff --git a/tests/integration/test_cluster_copier/data/.gitkeep b/tests/integration/test_cluster_copier/data/.gitkeep deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index cd69e6e28f2..134b66196b6 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -23,7 +23,7 @@ def started_cluster(): for name in ["first", "second", "third"]: cluster.add_instance(name, main_configs=["configs_three_nodes/conf.d/clusters.xml", "configs_three_nodes/conf.d/ddl.xml"], user_configs=["configs_three_nodes/users.xml"], - with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) + with_zookeeper=True) cluster.start() yield cluster @@ -41,7 +41,7 @@ class Task: instance = cluster.instances[instance_name] instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, './task_taxi_data.xml'), self.container_task_file) print("Copied task file to container of '{}' instance. Path {}".format(instance_name, self.container_task_file)) - + def start(self): instance = cluster.instances['first'] @@ -53,7 +53,7 @@ class Task: id UUID DEFAULT generateUUIDv4(), vendor_id String, tpep_pickup_datetime DateTime('UTC'), - tpep_dropoff_datetime DateTime('UTC'), + tpep_dropoff_datetime DateTime('UTC'), passenger_count Nullable(Float64), trip_distance String, pickup_longitude Float64, @@ -90,7 +90,7 @@ class Task: 'id UUID DEFAULT generateUUIDv4(), vendor_id String, tpep_pickup_datetime DateTime(\\'UTC\\'), - tpep_dropoff_datetime DateTime(\\'UTC\\'), + tpep_dropoff_datetime DateTime(\\'UTC\\'), passenger_count Nullable(Float64), trip_distance String, pickup_longitude Float64, @@ -143,7 +143,7 @@ class Task: congestion_surcharge String, junk1 String, junk2 String - ) + ) Engine = ReplacingMergeTree() PRIMARY KEY (tpep_pickup_datetime, id) ORDER BY (tpep_pickup_datetime, id) @@ -151,9 +151,9 @@ class Task: instance.query("""CREATE TABLE monthlyhistory.yellow_tripdata ON CLUSTER events - AS monthlyhistory.yellow_tripdata_staging + AS monthlyhistory.yellow_tripdata_staging ENGINE = Distributed('events', 'monthlyhistory', yellow_tripdata_staging, sipHash64(id) % 3);""") - + def check(self): instance = cluster.instances["first"] @@ -226,4 +226,3 @@ def execute_task(task, cmd_options): @pytest.mark.timeout(600) def test(started_cluster): execute_task(Task(started_cluster), []) - diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 28c52473366..44cab1924aa 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -24,11 +24,11 @@ def started_cluster(): for name in ["first", "second"]: instance = cluster.add_instance(name, main_configs=[ - "configs_two_nodes/conf.d/clusters.xml", + "configs_two_nodes/conf.d/clusters.xml", "configs_two_nodes/conf.d/ddl.xml", "configs_two_nodes/conf.d/storage_configuration.xml"], user_configs=["configs_two_nodes/users.xml"], - with_zookeeper=True, external_data_path=os.path.join(CURRENT_TEST_DIR, "./data")) + with_zookeeper=True) cluster.start() @@ -107,7 +107,7 @@ class TaskWithDifferentSchema: ) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column9, Column1, Column2, Column3, Column4);""") - + print("Preparation completed") def check(self): @@ -177,7 +177,7 @@ class TaskTTL: ) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1);""") - + print("Preparation completed") def check(self): @@ -242,7 +242,7 @@ class TaskSkipIndex: ) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1);""") - + print("Preparation completed") def check(self): @@ -286,7 +286,7 @@ class TaskTTLMoveToVolume: PARTITION BY (toYYYYMMDD(Column3), Column3) PRIMARY KEY (Column1, Column2, Column3) ORDER BY (Column1, Column2, Column3) - TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' + TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' SETTINGS storage_policy = 'external_with_jbods';""") first.query("""INSERT INTO db_move_to_volume.source SELECT * FROM generateRandom( @@ -304,9 +304,9 @@ class TaskTTLMoveToVolume: ) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(Column3) ORDER BY (Column3, Column2, Column1) - TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' + TTL Column3 + INTERVAL 1 MONTH TO VOLUME 'external' SETTINGS storage_policy = 'external_with_jbods';""") - + print("Preparation completed") def check(self): @@ -369,7 +369,7 @@ class TaskDropTargetPartition: # Insert data in target too. It has to be dropped. first.query("""INSERT INTO db_drop_target_partition.destination SELECT * FROM db_drop_target_partition.source;""") - + print("Preparation completed") def check(self): From 1c595c127f45465cfc47082acefe560e64170de2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Jun 2021 08:54:27 +0300 Subject: [PATCH 326/652] clickhouse-client: echo queries only after "{ echo }" hint Before this patch clickhouse-client interprets the whole queries and if "{ echo }" found, it starts echoing queries, but this will make it impossible to skip some of lines. --- programs/client/Client.cpp | 32 ++++++++++--------- .../queries/0_stateless/01891_echo.reference | 4 +++ tests/queries/0_stateless/01891_echo.sql | 3 ++ 3 files changed, 24 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01891_echo.reference create mode 100644 tests/queries/0_stateless/01891_echo.sql diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 098f7e689c5..97d9f081dea 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -965,13 +965,10 @@ private: TestHint test_hint(test_mode, all_queries_text); if (test_hint.clientError() || test_hint.serverError()) processTextAsSingleQuery("SET send_logs_level = 'fatal'"); - - // Echo all queries if asked; makes for a more readable reference - // file. - if (test_hint.echoQueries()) - echo_queries = true; } + bool echo_queries_ = echo_queries; + /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. /// An exception is VALUES format where we also support semicolon in @@ -1104,9 +1101,21 @@ private: continue; } + // Now we know for sure where the query ends. + // Look for the hint in the text of query + insert data + trailing + // comments, + // e.g. insert into t format CSV 'a' -- { serverError 123 }. + // Use the updated query boundaries we just calculated. + TestHint test_hint(test_mode, std::string(this_query_begin, this_query_end - this_query_begin)); + + // Echo all queries if asked; makes for a more readable reference + // file. + if (test_hint.echoQueries()) + echo_queries_ = true; + try { - processParsedSingleQuery(); + processParsedSingleQuery(echo_queries_); } catch (...) { @@ -1128,13 +1137,6 @@ private: adjustQueryEnd(this_query_end, all_queries_end, context->getSettingsRef().max_parser_depth); } - // Now we know for sure where the query ends. - // Look for the hint in the text of query + insert data + trailing - // comments, - // e.g. insert into t format CSV 'a' -- { serverError 123 }. - // Use the updated query boundaries we just calculated. - TestHint test_hint(test_mode, std::string(this_query_begin, this_query_end - this_query_begin)); - // Check whether the error (or its absence) matches the test hints // (or their absence). bool error_matches_hint = true; @@ -1545,14 +1547,14 @@ private: // 'query_to_send' -- the query text that is sent to server, // 'full_query' -- for INSERT queries, contains the query and the data that // follow it. Its memory is referenced by ASTInsertQuery::begin, end. - void processParsedSingleQuery() + void processParsedSingleQuery(std::optional echo_queries_ = {}) { resetOutput(); client_exception.reset(); server_exception.reset(); have_error = false; - if (echo_queries) + if (echo_queries_.value_or(echo_queries)) { writeString(full_query, std_out); writeChar('\n', std_out); diff --git a/tests/queries/0_stateless/01891_echo.reference b/tests/queries/0_stateless/01891_echo.reference new file mode 100644 index 00000000000..19c9b793902 --- /dev/null +++ b/tests/queries/0_stateless/01891_echo.reference @@ -0,0 +1,4 @@ +1 +-- { echo } +select 1; +1 diff --git a/tests/queries/0_stateless/01891_echo.sql b/tests/queries/0_stateless/01891_echo.sql new file mode 100644 index 00000000000..58f575a8f1a --- /dev/null +++ b/tests/queries/0_stateless/01891_echo.sql @@ -0,0 +1,3 @@ +select 1; +-- { echo } +select 1; From 69cf881947bcf43afa2009f571211e480527c280 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Jun 2021 08:54:28 +0300 Subject: [PATCH 327/652] clickhouse-client: add echoOn/echoOff hints --- programs/client/Client.cpp | 3 +-- programs/client/TestHint.h | 10 +++++++--- tests/queries/0_stateless/01891_echo.reference | 4 ++++ tests/queries/0_stateless/01891_echo.sql | 4 ++++ 4 files changed, 16 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 97d9f081dea..e2fe7921b8c 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1110,8 +1110,7 @@ private: // Echo all queries if asked; makes for a more readable reference // file. - if (test_hint.echoQueries()) - echo_queries_ = true; + echo_queries_ = test_hint.echoQueries().value_or(echo_queries_); try { diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index 32dc1f0bc17..bffc9efbce5 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -59,13 +59,13 @@ public: int serverError() const { return server_error; } int clientError() const { return client_error; } - bool echoQueries() const { return echo; } + std::optional echoQueries() const { return echo; } private: const String & query; int server_error = 0; int client_error = 0; - bool echo = false; + std::optional echo; void parse(const String & hint, bool is_leading_hint) { @@ -88,7 +88,11 @@ private: } if (item == "echo") - echo = true; + echo.emplace(true); + if (item == "echoOn") + echo.emplace(true); + if (item == "echoOff") + echo.emplace(false); } } diff --git a/tests/queries/0_stateless/01891_echo.reference b/tests/queries/0_stateless/01891_echo.reference index 19c9b793902..1593069f395 100644 --- a/tests/queries/0_stateless/01891_echo.reference +++ b/tests/queries/0_stateless/01891_echo.reference @@ -2,3 +2,7 @@ -- { echo } select 1; 1 +2 +-- { echoOn } +select 2; +2 diff --git a/tests/queries/0_stateless/01891_echo.sql b/tests/queries/0_stateless/01891_echo.sql index 58f575a8f1a..fc46b64dc6b 100644 --- a/tests/queries/0_stateless/01891_echo.sql +++ b/tests/queries/0_stateless/01891_echo.sql @@ -1,3 +1,7 @@ select 1; -- { echo } select 1; +-- { echoOff } +select 2; +-- { echoOn } +select 2; From ffacb7dcdfd6d6614ca33f1df396d9dbc33bb0e3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Jun 2021 08:54:28 +0300 Subject: [PATCH 328/652] Add description for test hints --- programs/client/TestHint.h | 32 +++++++++++++++++++++++++++++--- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index bffc9efbce5..100d47f4dd2 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -11,9 +11,35 @@ namespace DB { -/// Checks expected server and client error codes in testmode. -/// To enable it add special comment after the query: "-- { serverError 60 }" or "-- { clientError 20 }". -/// Also you can enable echoing all queries by writing "-- { echo }". +/// Checks expected server and client error codes in --testmode. +/// +/// The following comment hints are supported: +/// +/// - "-- { serverError 60 }" -- in case of you are expecting server error. +/// +/// - "-- { clientError 20 }" -- in case of you are expecting client error. +/// +/// Remember that the client parse the query first (not the server), so for +/// example if you are expecting syntax error, then you should use +/// clientError not serverError. +/// +/// Examples: +/// +/// - echo 'select / -- { clientError 62 }' | clickhouse-client --testmode -nm +/// +// Here the client parses the query but it is incorrect, so it expects +/// SYNTAX_ERROR (62). +/// +/// - echo 'select foo -- { serverError 47 }' | clickhouse-client --testmode -nm +/// +/// But here the query is correct, but there is no such column "foo", so it +/// is UNKNOWN_IDENTIFIER server error. +/// +/// The following hints will control the query echo mode (i.e print each query): +/// +/// - "-- { echo }" +/// - "-- { echoOn }" +/// - "-- { echoOff }" class TestHint { public: From feed2636f48654c34b2c91fa7c21ea7f72c41adb Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Jun 2021 01:07:29 +0300 Subject: [PATCH 329/652] Add exponential smoothing to Throttler --- programs/client/Suggest.cpp | 1 + src/Common/Throttler.cpp | 116 ++++++++++++++++++ src/Common/Throttler.h | 106 +++------------- src/Common/ya.make | 1 + .../test_replicated_fetches_bandwidth/test.py | 28 ++--- 5 files changed, 148 insertions(+), 104 deletions(-) create mode 100644 src/Common/Throttler.cpp diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 8d4c0fdbd5a..39b859dd133 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp new file mode 100644 index 00000000000..73cca720f91 --- /dev/null +++ b/src/Common/Throttler.cpp @@ -0,0 +1,116 @@ +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event ThrottlerSleepMicroseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LIMIT_EXCEEDED; +} + +/// Just 10^9. +static constexpr auto NS = 1000000000UL; + +/// Tracking window. Actually the size is not really important. We just want to avoid +/// throttles when there are no actions for a long period time. +static const double window_ns = 7UL * NS; + +void Throttler::add(size_t amount) +{ + size_t new_count; + /// This outer variable is always equal to smoothed_speed. + /// We use to avoid race condition. + double current_speed = 0; + + { + std::lock_guard lock(mutex); + + auto now = clock_gettime_ns(); + /// If prev_ns is equal to zero (first `add` call) we known nothing about speed + /// and don't track anything. + if (max_speed && prev_ns != 0) + { + /// Time spent to process the amount of bytes + double time_spent = now - prev_ns; + + /// The speed in bytes per second is equal to amount / time_spent in seconds + auto new_speed = amount / (time_spent / NS); + + /// If we didn't measured any speed before than we just assign + if (smoothed_speed == 0) + { + smoothed_speed = new_speed; + } + else + { + /// We want to make old values of speed less important for our smoothed value + /// so we decay it's value with coef. + auto decay_coeff = std::pow(0.5, time_spent / window_ns); + + /// Weighted average between previous and new speed + smoothed_speed = smoothed_speed * decay_coeff + (1 - decay_coeff) * new_speed; + current_speed = smoothed_speed; + } + } + + count += amount; + new_count = count; + prev_ns = now; + } + + if (limit && new_count > limit) + throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); + + if (max_speed && current_speed > max_speed) + { + /// If we was too fast then we have to sleep util our smoothed speed became <= max_speed + int64_t sleep_time = -window_ns * std::log2(max_speed / current_speed); + + if (sleep_time > 0) + { + accumulated_sleep += sleep_time; + + sleepForNanoseconds(sleep_time); + + accumulated_sleep -= sleep_time; + + ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time / 1000UL); + } + } + + if (parent) + parent->add(amount); +} + +void Throttler::reset() +{ + std::lock_guard lock(mutex); + + count = 0; + accumulated_sleep = 0; + smoothed_speed = 0; + prev_ns = 0; +} + +bool Throttler::isThrottling() const +{ + if (accumulated_sleep != 0) + return true; + + if (parent) + return parent->isThrottling(); + + return false; +} + +} diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 41f5661f67e..29dd31400c1 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -2,32 +2,15 @@ #include #include -#include -#include -#include #include -#include - - -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; -} - namespace DB { -namespace ErrorCodes -{ - extern const int LIMIT_EXCEEDED; -} - - /** Allows you to limit the speed of something (in entities per second) using sleep. * Specifics of work: - * - only the average speed is considered, from the moment of the first call of `add` function; - * if there were periods with low speed, then during some time after them, the speed will be higher; + * Tracks exponentially (pow of 1/2) smoothed speed with hardcoded window. + * See more comments in .cpp file. * * Also allows you to set a limit on the maximum number of entities. If exceeded, an exception will be thrown. */ @@ -41,54 +24,9 @@ public: const std::shared_ptr & parent_ = nullptr) : max_speed(max_speed_), limit(limit_), limit_exceeded_exception_message(limit_exceeded_exception_message_), parent(parent_) {} - void add(const size_t amount) - { - size_t new_count; - UInt64 elapsed_ns = 0; - - { - std::lock_guard lock(mutex); - - if (max_speed) - { - if (0 == count) - { - watch.start(); - elapsed_ns = 0; - } - else - elapsed_ns = watch.elapsed(); - } - - count += amount; - new_count = count; - } - - if (limit && new_count > limit) - throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); - - if (max_speed) - { - /// How much time to wait for the average speed to become `max_speed`. - UInt64 desired_ns = new_count * 1000000000 / max_speed; - - if (desired_ns > elapsed_ns) - { - UInt64 sleep_ns = desired_ns - elapsed_ns; - - accumulated_sleep += sleep_ns; - - sleepForNanoseconds(sleep_ns); - - accumulated_sleep -= sleep_ns; - - ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_ns / 1000UL); - } - } - - if (parent) - parent->add(amount); - } + /// Calculates the smoothed speed, sleeps if required and throws exception on + /// limit overflow. + void add(size_t amount); /// Not thread safe void setParent(const std::shared_ptr & parent_) @@ -96,35 +34,23 @@ public: parent = parent_; } - void reset() - { - std::lock_guard lock(mutex); - - count = 0; - watch.reset(); - accumulated_sleep = 0; - } + /// Reset all throttlers internal stats + void reset(); /// Is throttler already accumulated some sleep time and throttling. - bool isThrottling() const - { - if (accumulated_sleep != 0) - return true; - - if (parent) - return parent->isThrottling(); - - return false; - } + bool isThrottling() const; private: - size_t count = 0; - const size_t max_speed = 0; - const UInt64 limit = 0; /// 0 - not limited. + size_t count{0}; + const size_t max_speed{0}; + const uint64_t limit{0}; /// 0 - not limited. const char * limit_exceeded_exception_message = nullptr; - Stopwatch watch {CLOCK_MONOTONIC_COARSE}; std::mutex mutex; - std::atomic accumulated_sleep{0}; + std::atomic accumulated_sleep{0}; + /// Smoothed value of current speed. Updated in `add` method. + double smoothed_speed{0}; + /// previous `add` call time (in nanoseconds) + uint64_t prev_ns{0}; /// Used to implement a hierarchy of throttlers std::shared_ptr parent; diff --git a/src/Common/ya.make b/src/Common/ya.make index 57b60e9cce5..2da45e02b41 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -80,6 +80,7 @@ SRCS( ThreadPool.cpp ThreadProfileEvents.cpp ThreadStatus.cpp + Throttler.cpp TimerDescriptor.cpp TraceCollector.cpp UTF8Helpers.cpp diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py index 80dbd11b41a..6b67c21fb28 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/test.py +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -48,9 +48,9 @@ def test_limited_fetch_single_table(start_cluster): n2_fetch_speed.append(n2_in) time.sleep(0.5) - mean_speed = statistics.mean(n2_fetch_speed) + median_speed = statistics.median(n2_fetch_speed) # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert mean_speed <= 15, "We exceeded max fetch speed for more than 10MB/s. Must be around 10 (+- 5), got " + str(mean_speed) + assert median_speed <= 15, "We exceeded max fetch speed for more than 10MB/s. Must be around 10 (+- 5), got " + str(median_speed) finally: for node in [node1, node2]: @@ -61,7 +61,7 @@ def test_limited_send_single_table(start_cluster): for i, node in enumerate([node1, node2]): node.query(f"CREATE TABLE limited_send_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetch_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_sends_network_bandwidth=5242880") - node1.query("SYSTEM STOP REPLICATED SENDS limited_send_table") + node2.query("SYSTEM STOP FETCHES limited_send_table") for i in range(5): node1.query("INSERT INTO limited_send_table SELECT {}, '{}' FROM numbers(150)".format(i, get_random_string(104857))) @@ -69,7 +69,7 @@ def test_limited_send_single_table(start_cluster): n1_net = NetThroughput(node1) n2_net = NetThroughput(node2) - node1.query("SYSTEM START REPLICATED SENDS limited_send_table") + node2.query("SYSTEM START FETCHES limited_send_table") n1_sends_speed = [] for i in range(10): n1_in, n1_out = n1_net.measure_speed('megabytes') @@ -79,9 +79,9 @@ def test_limited_send_single_table(start_cluster): n1_sends_speed.append(n1_out) time.sleep(0.5) - mean_speed = statistics.mean(n1_sends_speed) + median_speed = statistics.median(n1_sends_speed) # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert mean_speed <= 10, "We exceeded max send speed for more than 5MB/s. Must be around 5 (+- 5), got " + str(mean_speed) + assert median_speed <= 10, "We exceeded max send speed for more than 5MB/s. Must be around 5 (+- 5), got " + str(median_speed) finally: for node in [node1, node2]: @@ -106,7 +106,7 @@ def test_limited_fetches_for_server(start_cluster): node3.query(f"SYSTEM START FETCHES limited_fetches{j}") n3_fetches_speed = [] - for i in range(10): + for i in range(5): n1_in, n1_out = n1_net.measure_speed('megabytes') n3_in, n3_out = n3_net.measure_speed('megabytes') print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") @@ -114,9 +114,9 @@ def test_limited_fetches_for_server(start_cluster): n3_fetches_speed.append(n3_in) time.sleep(0.5) - mean_speed = statistics.mean(n3_fetches_speed) + median_speed = statistics.median(n3_fetches_speed) # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert mean_speed <= 15, "We exceeded max fetch speed for more than 15MB/s. Must be around 5 (+- 10), got " + str(mean_speed) + assert median_speed <= 15, "We exceeded max fetch speed for more than 15MB/s. Must be around 5 (+- 10), got " + str(median_speed) finally: for node in [node1, node3]: @@ -131,7 +131,7 @@ def test_limited_sends_for_server(start_cluster): node.query(f"CREATE TABLE limited_sends{j}(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_sends{j}', '{i}') ORDER BY tuple() PARTITION BY key") for j in range(5): - node3.query(f"SYSTEM STOP REPLICATED SENDS limited_sends{j}") + node1.query(f"SYSTEM STOP FETCHES limited_sends{j}") for i in range(5): node3.query("INSERT INTO limited_sends{} SELECT {}, '{}' FROM numbers(50)".format(j, i, get_random_string(104857))) @@ -139,10 +139,10 @@ def test_limited_sends_for_server(start_cluster): n3_net = NetThroughput(node3) for j in range(5): - node3.query(f"SYSTEM START REPLICATED SENDS limited_sends{j}") + node1.query(f"SYSTEM START FETCHES limited_sends{j}") n3_sends_speed = [] - for i in range(10): + for i in range(5): n1_in, n1_out = n1_net.measure_speed('megabytes') n3_in, n3_out = n3_net.measure_speed('megabytes') print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") @@ -150,9 +150,9 @@ def test_limited_sends_for_server(start_cluster): n3_sends_speed.append(n3_out) time.sleep(0.5) - mean_speed = statistics.mean(n3_sends_speed) + median_speed = statistics.median(n3_sends_speed) # approximate border. Without limit we will have more than 100 MB/s for very slow builds. - assert mean_speed <= 20, "We exceeded max send speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(mean_speed) + assert median_speed <= 20, "We exceeded max send speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(median_speed) finally: for node in [node1, node3]: From 9882d160d8ba9498297d53d0c6f6bbdae29641a2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Jun 2021 07:38:56 +0300 Subject: [PATCH 330/652] clichouse-client: fix readability-identifier-naming --- programs/client/Client.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e2fe7921b8c..87f5efdb212 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -967,7 +967,7 @@ private: processTextAsSingleQuery("SET send_logs_level = 'fatal'"); } - bool echo_queries_ = echo_queries; + bool echo_query = echo_queries; /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. @@ -1110,11 +1110,11 @@ private: // Echo all queries if asked; makes for a more readable reference // file. - echo_queries_ = test_hint.echoQueries().value_or(echo_queries_); + echo_query = test_hint.echoQueries().value_or(echo_query); try { - processParsedSingleQuery(echo_queries_); + processParsedSingleQuery(echo_query); } catch (...) { @@ -1546,14 +1546,14 @@ private: // 'query_to_send' -- the query text that is sent to server, // 'full_query' -- for INSERT queries, contains the query and the data that // follow it. Its memory is referenced by ASTInsertQuery::begin, end. - void processParsedSingleQuery(std::optional echo_queries_ = {}) + void processParsedSingleQuery(std::optional echo_query = {}) { resetOutput(); client_exception.reset(); server_exception.reset(); have_error = false; - if (echo_queries_.value_or(echo_queries)) + if (echo_query.value_or(echo_queries)) { writeString(full_query, std_out); writeChar('\n', std_out); From 931cfb548dd49bbe1c065b7269be92e7f80ab0b7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Jun 2021 09:10:03 +0300 Subject: [PATCH 331/652] docs: update requests (to fix conflicts with urllib3) Fixes: 94b18bdfb8301439593b69e28c46d3c6147102b0 --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index facfc2d1ba1..9bb4f57e9e2 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -27,7 +27,7 @@ pymdown-extensions==8.0 python-slugify==4.0.1 PyYAML==5.4.1 repackage==0.7.3 -requests==2.24.0 +requests==2.25.1 singledispatch==3.4.0.3 six==1.15.0 soupsieve==2.0.1 From 1b936c8af6edb8b17cabda7931d47a44857ddb07 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 2 Jun 2021 09:04:49 +0300 Subject: [PATCH 332/652] Remove example usage of allow_experimental_funnel_functions from windowFunnel --- src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp | 9 +-------- src/Core/Settings.h | 4 +++- .../0_stateless/00632_aggregation_window_funnel.sql | 3 --- .../0_stateless/01656_sequence_next_node_long.sql | 3 +++ 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp b/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp index 0c4e2d167a4..e5df58e75b5 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp @@ -25,15 +25,8 @@ namespace template