Consolidate GCD codec tests (follow up to #53149)

This commit is contained in:
Robert Schulze 2023-09-05 16:51:54 +00:00
parent 470ce3444c
commit d36edea9e2
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
12 changed files with 51 additions and 50 deletions

View File

@ -391,19 +391,19 @@ DEFLATE_QPL is not available in ClickHouse Cloud.
### Specialized Codecs
These codecs are designed to make compression more effective by using specific features of data. Some of these codecs do not compress data themself. Instead, they prepare the data for a common purpose codec, which compresses it better than without this preparation.
These codecs are designed to make compression more effective by exploiting specific features of the data. Some of these codecs do not compress data themself, they instead preprocess the data such that a second compression stage using a general-purpose codec can achieve a higher data compression rate.
#### Delta
`Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, its 1. Delta is a data preparation codec, i.e. cannot be used stand-alone.
`Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, its 1. Delta is a data preparation codec, i.e. it cannot be used stand-alone.
#### DoubleDelta
`DoubleDelta(bytes_size)` — Calculates delta of deltas and writes it in compact binary form. Possible `bytes_size` values: 1, 2, 4, 8, the default value is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, its 1. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-bit deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). DoubleDelta is a data preparation codec, i.e. cannot be used stand-alone.
`DoubleDelta(bytes_size)` — Calculates delta of deltas and writes it in compact binary form. Possible `bytes_size` values: 1, 2, 4, 8, the default value is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, its 1. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-bit deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). DoubleDelta is a data preparation codec, i.e. it cannot be used stand-alone.
#### GCD
`GCD()` - - Calculates the greatest common denominator (GCD) of the values in the column, then divides each value by the GCD. Can be used with integer, decimal and date/time columns. A viable use case are timestamps or monetary values with high precision. GCD is a data preparation codec, i.e. cannot be used stand-alone.
`GCD()` - - Calculates the greatest common denominator (GCD) of the values in the column, then divides each value by the GCD. Can be used with integer, decimal and date/time columns. A viable use case are values which change (increase or decrease) in multiples of the GCD, e.g. 24, 28, 16, 24, 8, 24 (GCD = 4). GCD is a data preparation codec, i.e. it cannot be used stand-alone.
#### Gorilla

View File

@ -5,7 +5,6 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <IO/WriteHelpers.h>
namespace DB

View File

@ -3,14 +3,8 @@
#include <Compression/CompressionFactory.h>
#include <base/unaligned.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <IO/WriteHelpers.h>
#include "Common/Exception.h"
#include "DataTypes/IDataType.h"
#include "base/Decimal_fwd.h"
#include "base/types.h"
#include "config.h"
#include <boost/integer/common_factor.hpp>
#include <libdivide-config.h>
@ -84,7 +78,7 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest)
const char * const source_end = source + source_size;
T gcd_divider{};
T gcd_divider = 0;
const auto * cur_source = source;
while (gcd_divider != T(1) && cur_source < source_end)
{
@ -100,7 +94,7 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest)
if constexpr (sizeof(T) <= 8)
{
/// libdivide support only UInt32 and UInt64.
/// libdivide supports only UInt32 and UInt64.
using LibdivideT = std::conditional_t<sizeof(T) <= 4, UInt32, UInt64>;
libdivide::divider<LibdivideT> divider(static_cast<LibdivideT>(gcd_divider));
cur_source = source;
@ -126,8 +120,6 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest)
template <typename T>
void decompressDataForType(const char * source, UInt32 source_size, char * dest, UInt32 output_size)
{
const char * const output_end = dest + output_size;
if (source_size % sizeof(T) != 0)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress, data size {} is not aligned to {}", source_size, sizeof(T));
@ -135,11 +127,14 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress, data size {} is less than {}", source_size, sizeof(T));
const char * const source_end = source + source_size;
const char * const dest_end = dest + output_size;
const T gcd_multiplier = unalignedLoad<T>(source);
source += sizeof(T);
while (source < source_end)
{
if (dest + sizeof(T) > output_end) [[unlikely]]
if (dest + sizeof(T) > dest_end) [[unlikely]]
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data");
unalignedStore<T>(dest, unalignedLoad<T>(source) * gcd_multiplier);

View File

@ -12,7 +12,7 @@ ORDER BY (d2, d3);
INSERT INTO decimal (d1, d2, d3) VALUES (4.2, 4.2, 4.2);
SELECT type FROM system.columns WHERE table = 'decimal' AND database = currentDatabase();
SELECT type FROM system.columns WHERE table = 'decimal' AND database = currentDatabase() ORDER BY type;
SELECT toTypeName(d2), toTypeName(d3) FROM decimal LIMIT 1;

View File

@ -1,15 +0,0 @@
DROP TABLE IF EXISTS table_none;
CREATE TABLE table_none (id UInt64, ui UInt256 CODEC(LZ4)) ENGINE = Memory;
INSERT INTO table_none SELECT * FROM generateRandom() LIMIT 50;
DROP TABLE IF EXISTS table_gcd_codec;
CREATE TABLE table_gcd_codec (id UInt64, ui UInt256 CODEC(GCD, LZ4)) ENGINE = Memory;
INSERT INTO table_gcd_codec SELECT * FROM table_none;
SELECT COUNT(*)
FROM (
SELECT table_none.id, table_none.ui AS ui1, table_gcd_codec.id, table_gcd_codec.ui AS ui2
FROM table_none
JOIN table_gcd_codec ON table_none.id = table_gcd_codec.id
)
WHERE ui1 != ui2;

View File

@ -1,2 +0,0 @@
DROP TABLE IF EXISTS table_gcd_codec;
CREATE TABLE table_gcd_codec (str UInt64 CODEC(GCD)) ENGINE = Memory; -- { serverError 36 }

View File

@ -1,4 +1,5 @@
0
0
1
2
3
@ -998,3 +999,6 @@
1970-01-01 02:00:47.000
1970-01-01 02:00:48.000
1970-01-01 02:00:49.000
0
0
0

View File

@ -1,4 +1,28 @@
-- Int
-- GCD codec can't be used stand-alone
CREATE TEMPORARY TABLE table_gcd_codec (n UInt64 CODEC(GCD)) ENGINE = Memory; -- { serverError BAD_ARGUMENTS }
-- GCD codec rejects non-integer/decimal/datetime types
CREATE TEMPORARY TABLE table_gcd_codec (str String CODEC(GCD, LZ4)) ENGINE = Memory; -- { serverError BAD_ARGUMENTS }
-- Basic random-based correctness test
CREATE TEMPORARY TABLE table_lz4 (id UInt64, ui UInt256 CODEC(LZ4)) ENGINE = Memory;
INSERT INTO table_lz4 SELECT * FROM generateRandom() LIMIT 50;
CREATE TEMPORARY TABLE table_gcd (id UInt64, ui UInt256 CODEC(GCD, LZ4)) ENGINE = Memory;
INSERT INTO table_gcd SELECT * FROM table_lz4;
SELECT COUNT(*)
FROM (
SELECT table_lz4.id, table_lz4.ui AS ui1, table_gcd.id, table_gcd.ui AS ui2
FROM table_lz4 JOIN table_gcd
ON table_lz4.id = table_gcd.id
)
WHERE ui1 != ui2;
-------------------------------------------------------------------------------------------
-- Compression/decompression works for all data types supported by GCD codec
-- Int*
CREATE TEMPORARY TABLE table_gcd_codec_uint8 (n UInt8 CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_uint16 (n UInt16 CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_uint32 (n UInt32 CODEC(GCD, LZ4)) ENGINE = Memory;
@ -20,8 +44,7 @@ SELECT * FROM table_gcd_codec_uint64;
SELECT * FROM table_gcd_codec_uint128;
SELECT * FROM table_gcd_codec_uint256;
-- UInt
-- UInt*
CREATE TEMPORARY TABLE table_gcd_codec_int8 (n Int8 CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_int16 (n Int16 CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_int32 (n Int32 CODEC(GCD, LZ4)) ENGINE = Memory;
@ -43,8 +66,7 @@ SELECT * FROM table_gcd_codec_int64;
SELECT * FROM table_gcd_codec_int128;
SELECT * FROM table_gcd_codec_int256;
-- Decimal
-- Decimal*
CREATE TEMPORARY TABLE table_gcd_codec_decimal32 (n Decimal32(1) CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_decimal64 (n Decimal64(1) CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_decimal128 (n Decimal128(1) CODEC(GCD, LZ4)) ENGINE = Memory;
@ -60,8 +82,7 @@ SELECT * FROM table_gcd_codec_decimal64;
SELECT * FROM table_gcd_codec_decimal128;
SELECT * FROM table_gcd_codec_decimal256;
-- Date
-- Date[32]
CREATE TEMPORARY TABLE table_gcd_codec_date (n Date CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_date32 (n Date32 CODEC(GCD, LZ4)) ENGINE = Memory;
@ -71,8 +92,7 @@ INSERT INTO table_gcd_codec_date32 SELECT number FROM system.numbers LIMIT 50;
SELECT * FROM table_gcd_codec_date;
SELECT * FROM table_gcd_codec_date32;
-- DateTime
-- DateTimeTime[64]
CREATE TEMPORARY TABLE table_gcd_codec_datetime (n DateTime('Asia/Istanbul') CODEC(GCD, LZ4)) ENGINE = Memory;
CREATE TEMPORARY TABLE table_gcd_codec_datetime64 (n DateTime64(3, 'Asia/Istanbul') CODEC(GCD, LZ4)) ENGINE = Memory;
@ -81,3 +101,10 @@ INSERT INTO table_gcd_codec_datetime64 SELECT number FROM system.numbers LIMIT 5
SELECT * FROM table_gcd_codec_datetime;
SELECT * FROM table_gcd_codec_datetime64;
-- A column with all 0 values can be compressed/decompressed
CREATE TEMPORARY TABLE table_gcd_codec_only_zero_values (n UInt8 CODEC(GCD, LZ4)) ENGINE = Memory;
INSERT INTO table_gcd_codec_only_zero_values VALUES (0), (0), (0);
SELECT * FROM table_gcd_codec_only_zero_values;

View File

@ -1,3 +0,0 @@
CREATE TABLE table_gcd_codec_only_zero_values (n UInt8 CODEC(GCD, LZ4)) ENGINE = Memory;
INSERT INTO table_gcd_codec_only_zero_values VALUES (0), (0), (0);
SELECT * FROM table_gcd_codec_only_zero_values;