Merge remote-tracking branch 'origin/master' into pr-distributed-prefer-localhost-replica

This commit is contained in:
Igor Nikonov 2023-09-06 18:47:43 +00:00
commit f938c38eb9
126 changed files with 3203 additions and 944 deletions

View File

@ -23,7 +23,7 @@ array or map columns and strings correspondingly in generated data.
Generate table engine supports only `SELECT` queries. Generate table engine supports only `SELECT` queries.
It supports all [DataTypes](../../../sql-reference/data-types/index.md) that can be stored in a table except `LowCardinality` and `AggregateFunction`. It supports all [DataTypes](../../../sql-reference/data-types/index.md) that can be stored in a table except `AggregateFunction`.
## Example {#example} ## Example {#example}

View File

@ -4,15 +4,17 @@ sidebar_position: 42
sidebar_label: Decimal sidebar_label: Decimal
--- ---
# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S), Decimal256(S) # Decimal, Decimal(P), Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S), Decimal256(S)
Signed fixed-point numbers that keep precision during add, subtract and multiply operations. For division least significant digits are discarded (not rounded). Signed fixed-point numbers that keep precision during add, subtract and multiply operations. For division least significant digits are discarded (not rounded).
## Parameters ## Parameters
- P - precision. Valid range: \[ 1 : 76 \]. Determines how many decimal digits number can have (including fraction). - P - precision. Valid range: \[ 1 : 76 \]. Determines how many decimal digits number can have (including fraction). By default the precision is 10.
- S - scale. Valid range: \[ 0 : P \]. Determines how many decimal digits fraction can have. - S - scale. Valid range: \[ 0 : P \]. Determines how many decimal digits fraction can have.
Decimal(P) is equivalent to Decimal(P, 0). Similarly, the syntax Decimal is equivalent to Decimal(10, 0).
Depending on P parameter value Decimal(P, S) is a synonym for: Depending on P parameter value Decimal(P, S) is a synonym for:
- P from \[ 1 : 9 \] - for Decimal32(S) - P from \[ 1 : 9 \] - for Decimal32(S)
- P from \[ 10 : 18 \] - for Decimal64(S) - P from \[ 10 : 18 \] - for Decimal64(S)

View File

@ -92,6 +92,50 @@ Result:
└───┘ └───┘
``` ```
## isZeroOrNull
Returns whether the argument is 0 (zero) or [NULL](../../sql-reference/syntax.md#null-literal).
``` sql
isZeroOrNull(x)
```
**Arguments:**
- `x` — A value of non-compound data type.
**Returned value**
- `1` if `x` is 0 (zero) or `NULL`.
- `0` else.
**Example**
Table:
``` text
┌─x─┬────y─┐
│ 1 │ ᴺᵁᴸᴸ │
│ 2 │ 0 │
│ 3 │ 3 │
└───┴──────┘
```
Query:
``` sql
SELECT x FROM t_null WHERE isZeroOrNull(y);
```
Result:
``` text
┌─x─┐
│ 1 │
│ 2 │
└───┘
```
## coalesce ## coalesce
Returns the leftmost non-`NULL` argument. Returns the leftmost non-`NULL` argument.

View File

@ -395,11 +395,15 @@ These codecs are designed to make compression more effective by using specific f
#### Delta #### 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(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.
#### DoubleDelta #### 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(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.
#### 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.
#### Gorilla #### Gorilla

View File

@ -242,6 +242,7 @@ ClickHouse поддерживает кодеки общего назначени
- `Delta(delta_bytes)` — Метод, в котором исходные значения заменяются разностью двух соседних значений, за исключением первого значения, которое остаётся неизменным. Для хранения разниц используется до `delta_bytes`, т.е. `delta_bytes` — это максимальный размер исходных данных. Возможные значения `delta_bytes`: 1, 2, 4, 8. Значение по умолчанию для `delta_bytes` равно `sizeof(type)`, если результат 1, 2, 4, or 8. Во всех других случаях — 1. - `Delta(delta_bytes)` — Метод, в котором исходные значения заменяются разностью двух соседних значений, за исключением первого значения, которое остаётся неизменным. Для хранения разниц используется до `delta_bytes`, т.е. `delta_bytes` — это максимальный размер исходных данных. Возможные значения `delta_bytes`: 1, 2, 4, 8. Значение по умолчанию для `delta_bytes` равно `sizeof(type)`, если результат 1, 2, 4, or 8. Во всех других случаях — 1.
- `DoubleDelta` — Вычисляется разницу от разниц и сохраняет её в компакном бинарном виде. Оптимальная степень сжатия достигается для монотонных последовательностей с постоянным шагом, наподобие временных рядов. Можно использовать с любым типом данных фиксированного размера. Реализует алгоритм, используемый в TSDB Gorilla, поддерживает 64-битные типы данных. Использует 1 дополнительный бит для 32-байтовых значений: 5-битные префиксы вместо 4-битных префиксов. Подробнее читайте в разделе «Compressing Time Stamps» документа [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). - `DoubleDelta` — Вычисляется разницу от разниц и сохраняет её в компакном бинарном виде. Оптимальная степень сжатия достигается для монотонных последовательностей с постоянным шагом, наподобие временных рядов. Можно использовать с любым типом данных фиксированного размера. Реализует алгоритм, используемый в TSDB Gorilla, поддерживает 64-битные типы данных. Использует 1 дополнительный бит для 32-байтовых значений: 5-битные префиксы вместо 4-битных префиксов. Подробнее читайте в разделе «Compressing Time Stamps» документа [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf).
- `GCD` - Вычисляет НОД всех чисел, а затем делит их на него. Этот кодек предназначен для подготовки данных и не подходит для использования без дополнительного кодека. GCD-кодек может использоваться с Integer, Decimal и DateTime. Хорошим вариантом использования было бы хранение временных меток или денежных значений с высокой точностью.
- `Gorilla` — Вычисляет XOR между текущим и предыдущим значением и записывает результат в компактной бинарной форме. Еффективно сохраняет ряды медленно изменяющихся чисел с плавающей запятой, поскольку наилучший коэффициент сжатия достигается, если соседние значения одинаковые. Реализует алгоритм, используемый в TSDB Gorilla, адаптируя его для работы с 64-битными значениями. Подробнее читайте в разделе «Compressing Values» документа [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). - `Gorilla` — Вычисляет XOR между текущим и предыдущим значением и записывает результат в компактной бинарной форме. Еффективно сохраняет ряды медленно изменяющихся чисел с плавающей запятой, поскольку наилучший коэффициент сжатия достигается, если соседние значения одинаковые. Реализует алгоритм, используемый в TSDB Gorilla, адаптируя его для работы с 64-битными значениями. Подробнее читайте в разделе «Compressing Values» документа [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf).
- `T64` — Метод сжатия который обрезает неиспользуемые старшие биты целочисленных значений (включая `Enum`, `Date` и `DateTime`). На каждом шаге алгоритма, кодек помещает блок из 64 значений в матрицу 64✕64, транспонирует её, обрезает неиспользуемые биты, а то, что осталось возвращает в виде последовательности. Неиспользуемые биты, это биты, которые не изменяются от минимального к максимальному на всём диапазоне значений куска данных. - `T64` — Метод сжатия который обрезает неиспользуемые старшие биты целочисленных значений (включая `Enum`, `Date` и `DateTime`). На каждом шаге алгоритма, кодек помещает блок из 64 значений в матрицу 64✕64, транспонирует её, обрезает неиспользуемые биты, а то, что осталось возвращает в виде последовательности. Неиспользуемые биты, это биты, которые не изменяются от минимального к максимальному на всём диапазоне значений куска данных.

View File

@ -555,11 +555,13 @@ catch (...)
void Keeper::logRevision() const void Keeper::logRevision() const
{ {
Poco::Logger::root().information("Starting ClickHouse Keeper " + std::string{VERSION_STRING} LOG_INFO(&Poco::Logger::get("Application"),
+ "(revision : " + std::to_string(ClickHouseRevision::getVersionRevision()) "Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}",
+ ", git hash: " + (git_hash.empty() ? "<unknown>" : git_hash) VERSION_STRING,
+ ", build id: " + (build_id.empty() ? "<unknown>" : build_id) + ")" ClickHouseRevision::getVersionRevision(),
+ ", PID " + std::to_string(getpid())); git_hash.empty() ? "<unknown>" : git_hash,
build_id.empty() ? "<unknown>" : build_id,
getpid());
} }

View File

@ -31,6 +31,7 @@ namespace ErrorCodes
M(Bool, read_from_filesystem_cache) \ M(Bool, read_from_filesystem_cache) \
M(UInt64, shard_num) \ M(UInt64, shard_num) \
M(UInt64, replica_num) \ M(UInt64, replica_num) \
M(Bool, check_parts) \
M(Bool, internal) \ M(Bool, internal) \
M(String, host_id) \ M(String, host_id) \
M(OptionalUUID, backup_uuid) M(OptionalUUID, backup_uuid)

View File

@ -59,6 +59,9 @@ struct BackupSettings
/// Can only be used with BACKUP ON CLUSTER. /// Can only be used with BACKUP ON CLUSTER.
size_t replica_num = 0; size_t replica_num = 0;
/// Check checksums of the data parts before writing them to a backup.
bool check_parts = true;
/// Internal, should not be specified by user. /// Internal, should not be specified by user.
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER. /// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
bool internal = false; bool internal = false;

View File

@ -164,6 +164,7 @@ namespace
M(RestoreUDFCreationMode, create_function) \ M(RestoreUDFCreationMode, create_function) \
M(Bool, allow_s3_native_copy) \ M(Bool, allow_s3_native_copy) \
M(Bool, use_same_s3_credentials_for_base_backup) \ M(Bool, use_same_s3_credentials_for_base_backup) \
M(Bool, restore_broken_parts_as_detached) \
M(Bool, internal) \ M(Bool, internal) \
M(String, host_id) \ M(String, host_id) \
M(OptionalString, storage_policy) \ M(OptionalString, storage_policy) \

View File

@ -113,6 +113,10 @@ struct RestoreSettings
/// Whether base backup from S3 should inherit credentials from the RESTORE query. /// Whether base backup from S3 should inherit credentials from the RESTORE query.
bool use_same_s3_credentials_for_base_backup = false; bool use_same_s3_credentials_for_base_backup = false;
/// If it's true RESTORE won't stop on broken parts while restoring, instead they will be restored as detached parts
/// to the `detached` folder with names starting with `broken-from-backup'.
bool restore_broken_parts_as_detached = false;
/// Internal, should not be specified by user. /// Internal, should not be specified by user.
bool internal = false; bool internal = false;

View File

@ -92,7 +92,7 @@ void IBridge::defineOptions(Poco::Util::OptionSet & options)
Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024").argument("max-server-connections").binding("max-server-connections")); Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024").argument("max-server-connections").binding("max-server-connections"));
options.addOption( options.addOption(
Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10").argument("keep-alive-timeout").binding("keep-alive-timeout")); Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 30").argument("keep-alive-timeout").binding("keep-alive-timeout"));
options.addOption( options.addOption(
Poco::Util::Option("http-max-field-value-size", "", "max http field value size, default 1048576").argument("http-max-field-value-size").binding("http-max-field-value-size")); Poco::Util::Option("http-max-field-value-size", "", "max http field value size, default 1048576").argument("http-max-field-value-size").binding("http-max-field-value-size"));
@ -164,7 +164,7 @@ void IBridge::initialize(Application & self)
http_timeout = config().getUInt64("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); http_timeout = config().getUInt64("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
max_server_connections = config().getUInt("max-server-connections", 1024); max_server_connections = config().getUInt("max-server-connections", 1024);
keep_alive_timeout = config().getUInt64("keep-alive-timeout", 10); keep_alive_timeout = config().getUInt64("keep-alive-timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
http_max_field_value_size = config().getUInt64("http-max-field-value-size", 128 * 1024); http_max_field_value_size = config().getUInt64("http-max-field-value-size", 128 * 1024);
struct rlimit limit; struct rlimit limit;

View File

@ -134,6 +134,8 @@ void Connection::disconnect()
if (!is_initialized) if (!is_initialized)
return; return;
// If driver->free_me, then mysql_close will deallocate memory by calling 'free' function.
assert(driver && !driver->free_me);
mysql_close(driver.get()); mysql_close(driver.get());
memset(driver.get(), 0, sizeof(*driver)); memset(driver.get(), 0, sizeof(*driver));

View File

@ -0,0 +1,271 @@
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#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>
#include <libdivide.h>
namespace DB
{
class CompressionCodecGCD : public ICompressionCodec
{
public:
explicit CompressionCodecGCD(UInt8 gcd_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 false; }
bool isGenericCompression() const override { return false; }
private:
const UInt8 gcd_bytes_size;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int BAD_ARGUMENTS;
}
CompressionCodecGCD::CompressionCodecGCD(UInt8 gcd_bytes_size_)
: gcd_bytes_size(gcd_bytes_size_)
{
setCodecDescription("GCD", {});
}
UInt32 CompressionCodecGCD::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{
return uncompressed_size
+ gcd_bytes_size // To store gcd
+ 2; // Local header
}
uint8_t CompressionCodecGCD::getMethodByte() const
{
return static_cast<uint8_t>(CompressionMethodByte::GCD);
}
void CompressionCodecGCD::updateHash(SipHash & hash) const
{
getCodecDesc()->updateTreeHash(hash);
}
namespace
{
template <typename T>
void compressDataForType(const char * source, UInt32 source_size, char * dest)
{
if (source_size % sizeof(T) != 0)
throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot GCD compress, data size {} is not aligned to {}", source_size, sizeof(T));
const char * const source_end = source + source_size;
T gcd_divider{};
const auto * cur_source = source;
while (gcd_divider != T(1) && cur_source < source_end)
{
if (cur_source == source)
gcd_divider = unalignedLoad<T>(cur_source);
else
gcd_divider = boost::integer::gcd(gcd_divider, unalignedLoad<T>(cur_source));
cur_source += sizeof(T);
}
unalignedStore<T>(dest, gcd_divider);
dest += sizeof(T);
if constexpr (sizeof(T) <= 8)
{
/// libdivide support 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;
while (cur_source < source_end)
{
unalignedStore<T>(dest, static_cast<T>(static_cast<LibdivideT>(unalignedLoad<T>(cur_source)) / divider));
cur_source += sizeof(T);
dest += sizeof(T);
}
}
else
{
cur_source = source;
while (cur_source < source_end)
{
unalignedStore<T>(dest, unalignedLoad<T>(cur_source) / gcd_divider);
cur_source += sizeof(T);
dest += sizeof(T);
}
}
}
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));
if (source_size < sizeof(T))
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 T gcd_multiplier = unalignedLoad<T>(source);
source += sizeof(T);
while (source < source_end)
{
if (dest + sizeof(T) > output_end) [[unlikely]]
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data");
unalignedStore<T>(dest, unalignedLoad<T>(source) * gcd_multiplier);
source += sizeof(T);
dest += sizeof(T);
}
}
}
UInt32 CompressionCodecGCD::doCompressData(const char * source, UInt32 source_size, char * dest) const
{
UInt8 bytes_to_skip = source_size % gcd_bytes_size;
dest[0] = gcd_bytes_size;
dest[1] = bytes_to_skip; /// unused (backward compatibility)
memcpy(&dest[2], source, bytes_to_skip);
size_t start_pos = 2 + bytes_to_skip;
switch (gcd_bytes_size)
{
case 1:
compressDataForType<UInt8>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 2:
compressDataForType<UInt16>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 4:
compressDataForType<UInt32>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 8:
compressDataForType<UInt64>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 16:
compressDataForType<UInt128>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 32:
compressDataForType<UInt256>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
}
return 2 + gcd_bytes_size + source_size;
}
void CompressionCodecGCD::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
{
if (source_size < 2)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header");
if (uncompressed_size == 0)
return;
UInt8 bytes_size = source[0];
if (!(bytes_size == 1 || bytes_size == 2 || bytes_size == 4 || bytes_size == 8 || bytes_size == 16 || bytes_size == 32))
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header");
UInt8 bytes_to_skip = uncompressed_size % bytes_size;
UInt32 output_size = uncompressed_size - bytes_to_skip;
if (static_cast<UInt32>(2 + bytes_to_skip) > source_size)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header");
memcpy(dest, &source[2], bytes_to_skip);
UInt32 source_size_no_header = source_size - bytes_to_skip - 2;
switch (bytes_size)
{
case 1:
decompressDataForType<UInt8>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size);
break;
case 2:
decompressDataForType<UInt16>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size);
break;
case 4:
decompressDataForType<UInt32>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size);
break;
case 8:
decompressDataForType<UInt64>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size);
break;
case 16:
decompressDataForType<UInt128>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size);
break;
case 32:
decompressDataForType<UInt256>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size);
break;
}
}
namespace
{
UInt8 getGCDBytesSize(const IDataType * column_type)
{
WhichDataType which(column_type);
if (!(which.isInt() || which.isUInt() || which.isDecimal() || which.isDateOrDate32() || which.isDateTime() ||which.isDateTime64()))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec GCD is not applicable for {} because the data type is not of fixed size",
column_type->getName());
size_t max_size = column_type->getSizeOfValueInMemory();
if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8 || max_size == 16 || max_size == 32)
return static_cast<UInt8>(max_size);
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec GCD is only applicable for data types of size 1, 2, 4, 8, 16, 32 bytes. Given type {}",
column_type->getName());
}
}
void registerCodecGCD(CompressionCodecFactory & factory)
{
UInt8 method_code = static_cast<UInt8>(CompressionMethodByte::GCD);
auto codec_builder = [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
{
/// Default bytes size is 1.
UInt8 gcd_bytes_size = 1;
if (arguments && !arguments->children.empty())
throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE, "GCD codec must have 0 parameters, given {}", arguments->children.size());
else if (column_type)
gcd_bytes_size = getGCDBytesSize(column_type);
return std::make_shared<CompressionCodecGCD>(gcd_bytes_size);
};
factory.registerCompressionCodecWithType("GCD", method_code, codec_builder);
}
CompressionCodecPtr getCompressionCodecGCD(UInt8 gcd_bytes_size)
{
return std::make_shared<CompressionCodecGCD>(gcd_bytes_size);
}
}

View File

@ -168,7 +168,9 @@ void registerCodecLZ4(CompressionCodecFactory & factory);
void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory);
void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory);
void registerCodecMultiple(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory);
#ifdef ENABLE_QPL_COMPRESSION
void registerCodecDeflateQpl(CompressionCodecFactory & factory); void registerCodecDeflateQpl(CompressionCodecFactory & factory);
#endif
/// Keeper use only general-purpose codecs, so we don't need these special codecs /// Keeper use only general-purpose codecs, so we don't need these special codecs
/// in standalone build /// in standalone build
@ -179,6 +181,7 @@ void registerCodecDoubleDelta(CompressionCodecFactory & factory);
void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecGorilla(CompressionCodecFactory & factory);
void registerCodecEncrypted(CompressionCodecFactory & factory); void registerCodecEncrypted(CompressionCodecFactory & factory);
void registerCodecFPC(CompressionCodecFactory & factory); void registerCodecFPC(CompressionCodecFactory & factory);
void registerCodecGCD(CompressionCodecFactory & factory);
#endif #endif
CompressionCodecFactory::CompressionCodecFactory() CompressionCodecFactory::CompressionCodecFactory()
@ -198,6 +201,7 @@ CompressionCodecFactory::CompressionCodecFactory()
#ifdef ENABLE_QPL_COMPRESSION #ifdef ENABLE_QPL_COMPRESSION
registerCodecDeflateQpl(*this); registerCodecDeflateQpl(*this);
#endif #endif
registerCodecGCD(*this);
#endif #endif
default_codec = get("LZ4", {}); default_codec = get("LZ4", {});

View File

@ -47,6 +47,7 @@ enum class CompressionMethodByte : uint8_t
AES_256_GCM_SIV = 0x97, AES_256_GCM_SIV = 0x97,
FPC = 0x98, FPC = 0x98,
DeflateQpl = 0x99, DeflateQpl = 0x99,
GCD = 0x9a,
}; };
} }

View File

@ -18,3 +18,6 @@ target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms)
clickhouse_add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp) clickhouse_add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp)
target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms) target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms)
clickhouse_add_executable (gcd_decompress_fuzzer gcd_decompress_fuzzer.cpp)
target_link_libraries (gcd_decompress_fuzzer PRIVATE dbms)

View File

@ -0,0 +1,45 @@
#include <iostream>
#include <string>
#include <Compression/ICompressionCodec.h>
#include <IO/BufferWithOwnMemory.h>
#include "base/types.h"
namespace DB
{
CompressionCodecPtr getCompressionCodecGCD(UInt8 gcd_bytes_size);
}
struct AuxiliaryRandomData
{
UInt8 gcd_size_bytes;
size_t decompressed_size;
};
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try
{
if (size < sizeof(AuxiliaryRandomData))
return 0;
const auto * p = reinterpret_cast<const AuxiliaryRandomData *>(data);
auto codec = DB::getCompressionCodecGCD(p->gcd_size_bytes);
size_t output_buffer_size = p->decompressed_size % 65536;
size -= sizeof(AuxiliaryRandomData);
data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t);
// std::string input = std::string(reinterpret_cast<const char*>(data), size);
// fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size);
DB::Memory<> memory;
memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer());
codec->doDecompressData(reinterpret_cast<const char *>(data), static_cast<UInt32>(size), memory.data(), static_cast<UInt32>(output_buffer_size));
return 0;
}
catch (...)
{
return 1;
}

View File

@ -48,7 +48,7 @@
/// the number is unmotivated /// the number is unmotivated
#define DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT 15 #define DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT 15
#define DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT 10 #define DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT 30
#define DBMS_DEFAULT_PATH "/var/lib/clickhouse/" #define DBMS_DEFAULT_PATH "/var/lib/clickhouse/"

View File

@ -74,21 +74,30 @@ SerializationPtr DataTypeDecimal<T>::doGetDefaultSerialization() const
static DataTypePtr create(const ASTPtr & arguments) static DataTypePtr create(const ASTPtr & arguments)
{ {
if (!arguments || arguments->children.size() != 2) UInt64 precision = 10;
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, UInt64 scale = 0;
"Decimal data type family must have exactly two arguments: precision and scale"); if (arguments)
{
if (arguments->children.empty() || arguments->children.size() > 2)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Decimal data type family must have precision and optional scale arguments");
const auto * precision = arguments->children[0]->as<ASTLiteral>(); const auto * precision_arg = arguments->children[0]->as<ASTLiteral>();
const auto * scale = arguments->children[1]->as<ASTLiteral>(); if (!precision_arg || precision_arg->value.getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument precision is invalid");
precision = precision_arg->value.get<UInt64>();
if (!precision || precision->value.getType() != Field::Types::UInt64 || if (arguments->children.size() == 2)
!scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) {
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal data type family must have two numbers as its arguments"); const auto * scale_arg = arguments->children[1]->as<ASTLiteral>();
if (!scale_arg || !isInt64OrUInt64FieldType(scale_arg->value.getType()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument scale is invalid");
scale = scale_arg->value.get<UInt64>();
}
}
UInt64 precision_value = precision->value.get<UInt64>(); return createDecimal<DataTypeDecimal>(precision, scale);
UInt64 scale_value = scale->value.get<UInt64>();
return createDecimal<DataTypeDecimal>(precision_value, scale_value);
} }
template <typename T> template <typename T>

View File

@ -395,11 +395,15 @@ void FlatDictionary::updateData()
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
{ {
QueryPipeline pipeline(source_ptr->loadUpdatedAll()); QueryPipeline pipeline(source_ptr->loadUpdatedAll());
PullingPipelineExecutor executor(pipeline); PullingPipelineExecutor executor(pipeline);
update_field_loaded_block.reset();
Block block; Block block;
while (executor.pull(block)) while (executor.pull(block))
{ {
if (!block.rows())
continue;
convertToFullIfSparse(block); convertToFullIfSparse(block);
/// We are using this to keep saved data if input stream consists of multiple blocks /// We are using this to keep saved data if input stream consists of multiple blocks

View File

@ -409,11 +409,17 @@ void HashedArrayDictionary<dictionary_key_type>::updateData()
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
{ {
QueryPipeline pipeline(source_ptr->loadUpdatedAll()); QueryPipeline pipeline(source_ptr->loadUpdatedAll());
PullingPipelineExecutor executor(pipeline); PullingPipelineExecutor executor(pipeline);
update_field_loaded_block.reset();
Block block; Block block;
while (executor.pull(block)) while (executor.pull(block))
{ {
if (!block.rows())
continue;
convertToFullIfSparse(block);
/// We are using this to keep saved data if input stream consists of multiple blocks /// We are using this to keep saved data if input stream consists of multiple blocks
if (!update_field_loaded_block) if (!update_field_loaded_block)
update_field_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty()); update_field_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty());

View File

@ -709,11 +709,15 @@ void HashedDictionary<dictionary_key_type, sparse, sharded>::updateData()
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
{ {
QueryPipeline pipeline(source_ptr->loadUpdatedAll()); QueryPipeline pipeline(source_ptr->loadUpdatedAll());
PullingPipelineExecutor executor(pipeline); PullingPipelineExecutor executor(pipeline);
update_field_loaded_block.reset();
Block block; Block block;
while (executor.pull(block)) while (executor.pull(block))
{ {
if (!block.rows())
continue;
convertToFullIfSparse(block); convertToFullIfSparse(block);
/// We are using this to keep saved data if input stream consists of multiple blocks /// We are using this to keep saved data if input stream consists of multiple blocks

View File

@ -919,11 +919,17 @@ void RangeHashedDictionary<dictionary_key_type>::updateData()
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
{ {
QueryPipeline pipeline(source_ptr->loadUpdatedAll()); QueryPipeline pipeline(source_ptr->loadUpdatedAll());
PullingPipelineExecutor executor(pipeline); PullingPipelineExecutor executor(pipeline);
update_field_loaded_block.reset();
Block block; Block block;
while (executor.pull(block)) while (executor.pull(block))
{ {
if (!block.rows())
continue;
convertToFullIfSparse(block);
/// We are using this to keep saved data if input stream consists of multiple blocks /// We are using this to keep saved data if input stream consists of multiple blocks
if (!update_field_loaded_block) if (!update_field_loaded_block)
update_field_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty()); update_field_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty());

View File

@ -124,11 +124,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl()
read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds);
break; break;
} }
catch (const Azure::Core::Http::TransportException & e) catch (const Azure::Core::RequestFailedException & e)
{
handle_exception(e, i);
}
catch (const Azure::Storage::StorageException & e)
{ {
handle_exception(e, i); handle_exception(e, i);
} }
@ -240,10 +236,6 @@ void ReadBufferFromAzureBlobStorage::initialize()
data_stream = std::move(download_response.Value.BodyStream); data_stream = std::move(download_response.Value.BodyStream);
break; break;
} }
catch (const Azure::Core::Http::TransportException & e)
{
handle_exception(e, i);
}
catch (const Azure::Core::RequestFailedException & e) catch (const Azure::Core::RequestFailedException & e)
{ {
handle_exception(e,i); handle_exception(e,i);

View File

@ -55,7 +55,7 @@ std::unique_ptr<ReadBuffer> ReadBufferFromWebServer::initialize()
const auto & settings = context->getSettingsRef(); const auto & settings = context->getSettingsRef();
const auto & config = context->getConfigRef(); const auto & config = context->getConfigRef();
Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 20), 0}; Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0};
auto res = std::make_unique<ReadWriteBufferFromHTTP>( auto res = std::make_unique<ReadWriteBufferFromHTTP>(
uri, uri,

View File

@ -62,10 +62,6 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function<void()> func,
func(); func();
break; break;
} }
catch (const Azure::Core::Http::TransportException & e)
{
handle_exception(e, i);
}
catch (const Azure::Core::RequestFailedException & e) catch (const Azure::Core::RequestFailedException & e)
{ {
handle_exception(e, i); handle_exception(e, i);

View File

@ -62,7 +62,8 @@ std::unique_ptr<S3::Client> getClient(
client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000); client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000);
client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100);
client_configuration.endpointOverride = uri.endpoint; client_configuration.endpointOverride = uri.endpoint;
client_configuration.http_keep_alive_timeout_ms = config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", 10000); client_configuration.http_keep_alive_timeout_ms
= config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000);
client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000);
client_configuration.wait_on_pool_size_limit = false; client_configuration.wait_on_pool_size_limit = false;

View File

@ -35,7 +35,7 @@ TEST(AzureBlobContainerClient, CurlMemoryLeak)
options.Retry.MaxRetries = 0; options.Retry.MaxRetries = 0;
auto client = std::make_unique<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(unavailable_url, container, options)); auto client = std::make_unique<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(unavailable_url, container, options));
EXPECT_THROW({ client->ListBlobs(); }, Azure::Core::Http::TransportException); EXPECT_THROW({ client->ListBlobs(); }, Azure::Core::RequestFailedException);
} }
#endif #endif

View File

@ -82,6 +82,7 @@ try
bool is_eof = false; bool is_eof = false;
try try
{ {
read_buffer_iterator.setPreviousReadBuffer(std::move(buf));
buf = read_buffer_iterator.next(); buf = read_buffer_iterator.next();
if (!buf) if (!buf)
break; break;

View File

@ -11,6 +11,8 @@ struct IReadBufferIterator
{ {
virtual ~IReadBufferIterator() = default; virtual ~IReadBufferIterator() = default;
virtual void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> /* buffer */) {}
virtual std::unique_ptr<ReadBuffer> next() = 0; virtual std::unique_ptr<ReadBuffer> next() = 0;
virtual std::optional<ColumnsDescription> getCachedColumns() { return std::nullopt; } virtual std::optional<ColumnsDescription> getCachedColumns() { return std::nullopt; }
@ -21,7 +23,7 @@ struct IReadBufferIterator
struct SingleReadBufferIterator : public IReadBufferIterator struct SingleReadBufferIterator : public IReadBufferIterator
{ {
public: public:
SingleReadBufferIterator(std::unique_ptr<ReadBuffer> buf_) : buf(std::move(buf_)) explicit SingleReadBufferIterator(std::unique_ptr<ReadBuffer> buf_) : buf(std::move(buf_))
{ {
} }

View File

@ -87,6 +87,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int CANNOT_PARSE_BOOL;
} }
@ -1597,7 +1598,19 @@ struct ConvertImplGenericFromString
const auto & val = col_from_string->getDataAt(i); const auto & val = col_from_string->getDataAt(i);
ReadBufferFromMemory read_buffer(val.data, val.size); ReadBufferFromMemory read_buffer(val.data, val.size);
try
{
serialization_from.deserializeWholeText(column_to, read_buffer, format_settings); serialization_from.deserializeWholeText(column_to, read_buffer, format_settings);
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::CANNOT_PARSE_BOOL && typeid_cast<ColumnNullable *>(&column_to))
{
column_to.insertDefault();
continue;
}
throw;
}
if (!read_buffer.eof()) if (!read_buffer.eof())
{ {
@ -4054,15 +4067,21 @@ private:
{ {
if constexpr (std::is_same_v<ToDataType, DataTypeIPv4>) if constexpr (std::is_same_v<ToDataType, DataTypeIPv4>)
{ {
ret = [cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv4_default_on_conversion_error_value, requested_result_is_nullable]( ret = [cast_ipv4_ipv6_default_on_conversion_error_value,
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) input_format_ipv4_default_on_conversion_error_value,
-> ColumnPtr requested_result_is_nullable](
ColumnsWithTypeAndName & arguments,
const DataTypePtr & result_type,
const ColumnNullable * column_nullable,
size_t) -> ColumnPtr
{ {
if (!WhichDataType(result_type).isIPv4()) if (!WhichDataType(result_type).isIPv4())
throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName());
const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr;
if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value || requested_result_is_nullable) if (requested_result_is_nullable)
return convertToIPv4<IPStringToNumExceptionMode::Null>(arguments[0].column, null_map);
else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value)
return convertToIPv4<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map); return convertToIPv4<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map);
else else
return convertToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map); return convertToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
@ -4073,16 +4092,22 @@ private:
if constexpr (std::is_same_v<ToDataType, DataTypeIPv6>) if constexpr (std::is_same_v<ToDataType, DataTypeIPv6>)
{ {
ret = [cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( ret = [cast_ipv4_ipv6_default_on_conversion_error_value,
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) input_format_ipv6_default_on_conversion_error_value,
-> ColumnPtr requested_result_is_nullable](
ColumnsWithTypeAndName & arguments,
const DataTypePtr & result_type,
const ColumnNullable * column_nullable,
size_t) -> ColumnPtr
{ {
if (!WhichDataType(result_type).isIPv6()) if (!WhichDataType(result_type).isIPv6())
throw Exception( throw Exception(
ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName()); ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName());
const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr;
if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value || requested_result_is_nullable) if (requested_result_is_nullable)
return convertToIPv6<IPStringToNumExceptionMode::Null>(arguments[0].column, null_map);
else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value)
return convertToIPv6<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map); return convertToIPv6<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map);
else else
return convertToIPv6<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map); return convertToIPv6<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
@ -4093,7 +4118,18 @@ private:
if (to_type->getCustomSerialization() && to_type->getCustomName()) if (to_type->getCustomSerialization() && to_type->getCustomName())
{ {
ret = &ConvertImplGenericFromString<typename FromDataType::ColumnType>::execute; ret = [requested_result_is_nullable](
ColumnsWithTypeAndName & arguments,
const DataTypePtr & result_type,
const ColumnNullable * column_nullable,
size_t input_rows_count) -> ColumnPtr
{
auto wrapped_result_type = result_type;
if (requested_result_is_nullable)
wrapped_result_type = makeNullable(result_type);
return ConvertImplGenericFromString<typename FromDataType::ColumnType>::execute(
arguments, wrapped_result_type, column_nullable, input_rows_count);
};
return true; return true;
} }
} }
@ -4108,7 +4144,9 @@ private:
ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName());
const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr;
if (cast_ipv4_ipv6_default_on_conversion_error_value || requested_result_is_nullable) if (requested_result_is_nullable)
return convertIPv6ToIPv4<IPStringToNumExceptionMode::Null>(arguments[0].column, null_map);
else if (cast_ipv4_ipv6_default_on_conversion_error_value)
return convertIPv6ToIPv4<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map); return convertIPv6ToIPv4<IPStringToNumExceptionMode::Default>(arguments[0].column, null_map);
else else
return convertIPv6ToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map); return convertIPv6ToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);

View File

@ -102,17 +102,13 @@ private:
if (key_argument_data_type.isArray()) if (key_argument_data_type.isArray())
{ {
DataTypePtr value_type; if (arguments.size() < 2 || !arguments[1] || !isArray(arguments[1]))
if (1 < arguments.size())
value_type = arguments[1];
if (arguments.size() < 2 || (value_type && !isArray(value_type)))
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Function {} if array argument is passed as key, additional array argument as value must be passed", "Function {} if array argument is passed as key, additional array argument as value must be passed",
getName()); getName());
const auto & key_array_type = assert_cast<const DataTypeArray &>(*arguments[0]); const auto & key_array_type = assert_cast<const DataTypeArray &>(*arguments[0]);
const auto & value_array_type = assert_cast<const DataTypeArray &>(*value_type); const auto & value_array_type = assert_cast<const DataTypeArray &>(*arguments[1]);
key_argument_series_type = key_array_type.getNestedType(); key_argument_series_type = key_array_type.getNestedType();
value_argument_series_type = value_array_type.getNestedType(); value_argument_series_type = value_array_type.getNestedType();

View File

@ -44,14 +44,18 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & types) const override DataTypePtr getReturnTypeImpl(const DataTypes & types) const override
{ {
if (!isNumber(removeNullable(types.at(0)))) if (!isNumber(removeNullable(types.at(0))) && !isNothing(removeNullable(types.at(0))))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The argument of function {} must have simple numeric type, possibly Nullable", name); throw Exception(
ErrorCodes::BAD_ARGUMENTS, "The argument of function {} must have simple numeric type, possibly Nullable or Null", name);
return std::make_shared<DataTypeUInt8>(); return std::make_shared<DataTypeUInt8>();
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
if (isNothing(removeNullable(arguments[0].type)))
return DataTypeUInt8{}.createColumnConst(input_rows_count, 1);
const ColumnPtr & input_column = arguments[0].column; const ColumnPtr & input_column = arguments[0].column;
ColumnPtr res; ColumnPtr res;
@ -72,7 +76,10 @@ public:
return true; return true;
})) }))
{ {
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The argument of function {} must have simple numeric type, possibly Nullable", name); throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"The argument of function {} must have simple numeric type, possibly Nullable or Null",
name);
} }
} }
else else
@ -89,7 +96,10 @@ public:
return true; return true;
})) }))
{ {
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The argument of function {} must have simple numeric type, possibly Nullable", name); throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"The argument of function {} must have simple numeric type, possibly Nullable or Null",
name);
} }
} }

View File

@ -11,6 +11,9 @@
# include <aws/core/utils/UUID.h> # include <aws/core/utils/UUID.h>
# include <aws/core/http/HttpClientFactory.h> # include <aws/core/http/HttpClientFactory.h>
# include <aws/core/utils/HashingUtils.h>
# include <aws/core/platform/FileSystem.h>
# include <Common/logger_useful.h> # include <Common/logger_useful.h>
# include <IO/S3/PocoHTTPClient.h> # include <IO/S3/PocoHTTPClient.h>
@ -43,6 +46,8 @@ bool areCredentialsEmptyOrExpired(const Aws::Auth::AWSCredentials & credentials,
return now >= credentials.GetExpiration() - std::chrono::seconds(expiration_window_seconds); return now >= credentials.GetExpiration() - std::chrono::seconds(expiration_window_seconds);
} }
const char SSO_CREDENTIALS_PROVIDER_LOG_TAG[] = "SSOCredentialsProvider";
} }
AWSEC2MetadataClient::AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration, const char * endpoint_) AWSEC2MetadataClient::AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration, const char * endpoint_)
@ -449,6 +454,139 @@ void AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::refreshIfExpired()
Reload(); Reload();
} }
SSOCredentialsProvider::SSOCredentialsProvider(DB::S3::PocoHTTPClientConfiguration aws_client_configuration_, uint64_t expiration_window_seconds_)
: profile_to_use(Aws::Auth::GetConfigProfileName())
, aws_client_configuration(std::move(aws_client_configuration_))
, expiration_window_seconds(expiration_window_seconds_)
, logger(&Poco::Logger::get(SSO_CREDENTIALS_PROVIDER_LOG_TAG))
{
LOG_INFO(logger, "Setting sso credentials provider to read config from {}", profile_to_use);
}
Aws::Auth::AWSCredentials SSOCredentialsProvider::GetAWSCredentials()
{
refreshIfExpired();
Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock);
return credentials;
}
void SSOCredentialsProvider::Reload()
{
auto profile = Aws::Config::GetCachedConfigProfile(profile_to_use);
const auto access_token = [&]
{
// If we have an SSO Session set, use the refreshed token.
if (profile.IsSsoSessionSet())
{
sso_region = profile.GetSsoSession().GetSsoRegion();
auto token = bearer_token_provider.GetAWSBearerToken();
expires_at = token.GetExpiration();
return token.GetToken();
}
Aws::String hashed_start_url = Aws::Utils::HashingUtils::HexEncode(Aws::Utils::HashingUtils::CalculateSHA1(profile.GetSsoStartUrl()));
auto profile_directory = Aws::Auth::ProfileConfigFileAWSCredentialsProvider::GetProfileDirectory();
Aws::StringStream ss_token;
ss_token << profile_directory;
ss_token << Aws::FileSystem::PATH_DELIM << "sso" << Aws::FileSystem::PATH_DELIM << "cache" << Aws::FileSystem::PATH_DELIM << hashed_start_url << ".json";
auto sso_token_path = ss_token.str();
LOG_INFO(logger, "Loading token from: {}", sso_token_path);
sso_region = profile.GetSsoRegion();
return loadAccessTokenFile(sso_token_path);
}();
if (access_token.empty())
{
LOG_TRACE(logger, "Access token for SSO not available");
return;
}
if (expires_at < Aws::Utils::DateTime::Now())
{
LOG_TRACE(logger, "Cached Token expired at {}", expires_at.ToGmtString(Aws::Utils::DateFormat::ISO_8601));
return;
}
Aws::Internal::SSOCredentialsClient::SSOGetRoleCredentialsRequest request;
request.m_ssoAccountId = profile.GetSsoAccountId();
request.m_ssoRoleName = profile.GetSsoRoleName();
request.m_accessToken = access_token;
aws_client_configuration.scheme = Aws::Http::Scheme::HTTPS;
aws_client_configuration.region = sso_region;
LOG_TRACE(logger, "Passing config to client for region: {}", sso_region);
Aws::Vector<Aws::String> retryable_errors;
retryable_errors.push_back("TooManyRequestsException");
aws_client_configuration.retryStrategy = Aws::MakeShared<Aws::Client::SpecifiedRetryableErrorsRetryStrategy>(SSO_CREDENTIALS_PROVIDER_LOG_TAG, retryable_errors, /*maxRetries=*/3);
client = Aws::MakeUnique<Aws::Internal::SSOCredentialsClient>(SSO_CREDENTIALS_PROVIDER_LOG_TAG, aws_client_configuration);
LOG_TRACE(logger, "Requesting credentials with AWS_ACCESS_KEY: {}", sso_account_id);
auto result = client->GetSSOCredentials(request);
LOG_TRACE(logger, "Successfully retrieved credentials with AWS_ACCESS_KEY: {}", result.creds.GetAWSAccessKeyId());
credentials = result.creds;
}
void SSOCredentialsProvider::refreshIfExpired()
{
Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock);
if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds))
return;
guard.UpgradeToWriterLock();
if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds)) // double-checked lock to avoid refreshing twice
return;
Reload();
}
Aws::String SSOCredentialsProvider::loadAccessTokenFile(const Aws::String & sso_access_token_path)
{
LOG_TRACE(logger, "Preparing to load token from: {}", sso_access_token_path);
Aws::IFStream input_file(sso_access_token_path.c_str());
if (input_file)
{
LOG_TRACE(logger, "Reading content from token file: {}", sso_access_token_path);
Aws::Utils::Json::JsonValue token_doc(input_file);
if (!token_doc.WasParseSuccessful())
{
LOG_TRACE(logger, "Failed to parse token file: {}", sso_access_token_path);
return "";
}
Aws::Utils::Json::JsonView token_view(token_doc);
Aws::String tmp_access_token, expiration_str;
tmp_access_token = token_view.GetString("accessToken");
expiration_str = token_view.GetString("expiresAt");
Aws::Utils::DateTime expiration(expiration_str, Aws::Utils::DateFormat::ISO_8601);
LOG_TRACE(logger, "Token cache file contains accessToken [{}], expiration [{}]", tmp_access_token, expiration_str);
if (tmp_access_token.empty() || !expiration.WasParseSuccessful())
{
LOG_TRACE(logger, R"(The SSO session associated with this profile has expired or is otherwise invalid. To refresh this SSO session run aws sso login with the corresponding profile.)");
LOG_TRACE(
logger,
"Token cache file failed because {}{}",
(tmp_access_token.empty() ? "AccessToken was empty " : ""),
(!expiration.WasParseSuccessful() ? "failed to parse expiration" : ""));
return "";
}
expires_at = expiration;
return tmp_access_token;
}
else
{
LOG_TRACE(logger, "Unable to open token file on path: {}", sso_access_token_path);
return "";
}
}
S3CredentialsProviderChain::S3CredentialsProviderChain( S3CredentialsProviderChain::S3CredentialsProviderChain(
const DB::S3::PocoHTTPClientConfiguration & configuration, const DB::S3::PocoHTTPClientConfiguration & configuration,
const Aws::Auth::AWSCredentials & credentials, const Aws::Auth::AWSCredentials & credentials,
@ -494,6 +632,18 @@ S3CredentialsProviderChain::S3CredentialsProviderChain(
AddProvider(std::make_shared<Aws::Auth::EnvironmentAWSCredentialsProvider>()); AddProvider(std::make_shared<Aws::Auth::EnvironmentAWSCredentialsProvider>());
{
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(
configuration.region,
configuration.remote_host_filter,
configuration.s3_max_redirects,
configuration.enable_s3_requests_logging,
configuration.for_disk_s3,
configuration.get_request_throttler,
configuration.put_request_throttler);
AddProvider(std::make_shared<SSOCredentialsProvider>(
std::move(aws_client_configuration), credentials_configuration.expiration_window_seconds));
}
/// ECS TaskRole Credentials only available when ENVIRONMENT VARIABLE is set. /// ECS TaskRole Credentials only available when ENVIRONMENT VARIABLE is set.
const auto relative_uri = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI); const auto relative_uri = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI);

View File

@ -8,6 +8,7 @@
# include <aws/core/internal/AWSHttpResourceClient.h> # include <aws/core/internal/AWSHttpResourceClient.h>
# include <aws/core/config/AWSProfileConfigLoader.h> # include <aws/core/config/AWSProfileConfigLoader.h>
# include <aws/core/auth/AWSCredentialsProviderChain.h> # include <aws/core/auth/AWSCredentialsProviderChain.h>
# include <aws/core/auth/bearer-token-provider/SSOBearerTokenProvider.h>
# include <IO/S3/PocoHTTPClient.h> # include <IO/S3/PocoHTTPClient.h>
@ -124,6 +125,39 @@ private:
uint64_t expiration_window_seconds; uint64_t expiration_window_seconds;
}; };
class SSOCredentialsProvider : public Aws::Auth::AWSCredentialsProvider
{
public:
SSOCredentialsProvider(DB::S3::PocoHTTPClientConfiguration aws_client_configuration_, uint64_t expiration_window_seconds_);
Aws::Auth::AWSCredentials GetAWSCredentials() override;
private:
Aws::UniquePtr<Aws::Internal::SSOCredentialsClient> client;
Aws::Auth::AWSCredentials credentials;
// Profile description variables
Aws::String profile_to_use;
// The AWS account ID that temporary AWS credentials are resolved for.
Aws::String sso_account_id;
// The AWS region where the SSO directory for the given sso_start_url is hosted.
// This is independent of the general region configuration and MUST NOT be conflated.
Aws::String sso_region;
// The expiration time of the accessToken.
Aws::Utils::DateTime expires_at;
// The SSO Token Provider
Aws::Auth::SSOBearerTokenProvider bearer_token_provider;
DB::S3::PocoHTTPClientConfiguration aws_client_configuration;
uint64_t expiration_window_seconds;
Poco::Logger * logger;
void Reload() override;
void refreshIfExpired();
Aws::String loadAccessTokenFile(const Aws::String & sso_access_token_path);
};
struct CredentialsConfiguration struct CredentialsConfiguration
{ {
bool use_environment_credentials = false; bool use_environment_credentials = false;

View File

@ -2506,6 +2506,96 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
return result_dag; return result_dag;
} }
ActionsDAG::NodeRawConstPtrs ActionsDAG::extractConjunctionAtoms(const Node * predicate)
{
NodeRawConstPtrs atoms;
std::stack<const ActionsDAG::Node *> stack;
stack.push(predicate);
while (!stack.empty())
{
const auto * node = stack.top();
stack.pop();
if (node->type == ActionsDAG::ActionType::FUNCTION)
{
const auto & name = node->function_base->getName();
if (name == "and")
{
for (const auto * arg : node->children)
stack.push(arg);
continue;
}
}
atoms.push_back(node);
}
return atoms;
}
ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs(
NodeRawConstPtrs nodes,
const std::unordered_set<const Node *> & allowed_inputs)
{
size_t result_size = 0;
std::unordered_map<const ActionsDAG::Node *, bool> can_compute;
struct Frame
{
const ActionsDAG::Node * node;
size_t next_child_to_visit = 0;
bool can_compute_all_childern = true;
};
std::stack<Frame> stack;
for (const auto * node : nodes)
{
if (!can_compute.contains(node))
stack.push({node});
while (!stack.empty())
{
auto & frame = stack.top();
bool need_visit_child = false;
while (frame.next_child_to_visit < frame.node->children.size())
{
auto it = can_compute.find(frame.node->children[frame.next_child_to_visit]);
if (it == can_compute.end())
{
stack.push({frame.node->children[frame.next_child_to_visit]});
need_visit_child = true;
break;
}
frame.can_compute_all_childern &= it->second;
++frame.next_child_to_visit;
}
if (need_visit_child)
continue;
if (frame.node->type == ActionsDAG::ActionType::INPUT)
can_compute[frame.node] = allowed_inputs.contains(frame.node);
else
can_compute[frame.node] = frame.can_compute_all_childern;
stack.pop();
}
if (can_compute.at(node))
{
nodes[result_size] = node;
++result_size;
}
}
nodes.resize(result_size);
return nodes;
}
FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_) FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_)
:actions(actions_) :actions(actions_)
{ {

View File

@ -384,6 +384,16 @@ public:
const ContextPtr & context, const ContextPtr & context,
bool single_output_condition_node = true); bool single_output_condition_node = true);
/// Check if `predicate` is a combination of AND functions.
/// Returns a list of nodes representing atomic predicates.
static NodeRawConstPtrs extractConjunctionAtoms(const Node * predicate);
/// Get a list of nodes. For every node, check if it can be compused using allowed subset of inputs.
/// Returns only those nodes from the list which can be computed.
static NodeRawConstPtrs filterNodesByAllowedInputs(
NodeRawConstPtrs nodes,
const std::unordered_set<const Node *> & allowed_inputs);
private: private:
NodeRawConstPtrs getParents(const Node * target) const; NodeRawConstPtrs getParents(const Node * target) const;

View File

@ -345,7 +345,7 @@ Block createBlockForSet(
{ {
auto get_tuple_type_from_ast = [context](const auto & func) -> DataTypePtr auto get_tuple_type_from_ast = [context](const auto & func) -> DataTypePtr
{ {
if (func && (func->name == "tuple" || func->name == "array") && !func->arguments->children.empty()) if ((func->name == "tuple" || func->name == "array") && !func->arguments->children.empty())
{ {
/// Won't parse all values of outer tuple. /// Won't parse all values of outer tuple.
auto element = func->arguments->children.at(0); auto element = func->arguments->children.at(0);
@ -356,6 +356,7 @@ Block createBlockForSet(
return evaluateConstantExpression(func, context).second; return evaluateConstantExpression(func, context).second;
}; };
assert(right_arg);
const DataTypePtr & right_arg_type = get_tuple_type_from_ast(right_arg); const DataTypePtr & right_arg_type = get_tuple_type_from_ast(right_arg);
size_t left_tuple_depth = getTypeDepth(left_arg_type); size_t left_tuple_depth = getTypeDepth(left_arg_type);

View File

@ -123,7 +123,7 @@ void SelectStreamFactory::createForShard(
auto emplace_local_stream = [&]() auto emplace_local_stream = [&]()
{ {
local_plans.emplace_back(createLocalPlan( local_plans.emplace_back(createLocalPlan(
query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); query_ast, header, context, processed_stage, shard_info.shard_num, shard_count));
}; };
auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0)

View File

@ -604,15 +604,13 @@ std::optional<bool> tryEvaluateConstCondition(ASTPtr expr, ContextPtr context)
Field eval_res; Field eval_res;
DataTypePtr eval_res_type; DataTypePtr eval_res_type;
try
{ {
std::tie(eval_res, eval_res_type) = evaluateConstantExpression(expr, context); auto constant_expression_result = tryEvaluateConstantExpression(expr, context);
} if (!constant_expression_result)
catch (DB::Exception &)
{
/// not a constant expression
return {}; return {};
std::tie(eval_res, eval_res_type) = std::move(constant_expression_result.value());
} }
/// UInt8, maybe Nullable, maybe LowCardinality, and NULL are allowed /// UInt8, maybe Nullable, maybe LowCardinality, and NULL are allowed
eval_res_type = removeNullable(removeLowCardinality(eval_res_type)); eval_res_type = removeNullable(removeLowCardinality(eval_res_type));
if (auto which = WhichDataType(eval_res_type); !which.isUInt8() && !which.isNothing()) if (auto which = WhichDataType(eval_res_type); !which.isUInt8() && !which.isNothing())
@ -959,7 +957,7 @@ void TreeRewriterResult::collectSourceColumns(bool add_special)
/// Calculate which columns are required to execute the expression. /// Calculate which columns are required to execute the expression.
/// Then, delete all other columns from the list of available columns. /// Then, delete all other columns from the list of available columns.
/// After execution, columns will only contain the list of columns needed to read from the table. /// After execution, columns will only contain the list of columns needed to read from the table.
void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint) bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw)
{ {
/// We calculate required_source_columns with source_columns modifications and swap them on exit /// We calculate required_source_columns with source_columns modifications and swap them on exit
required_source_columns = source_columns; required_source_columns = source_columns;
@ -1178,6 +1176,8 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
ss << " '" << name << "'"; ss << " '" << name << "'";
} }
if (no_throw)
return false;
throw Exception(PreformattedMessage{ss.str(), format_string}, ErrorCodes::UNKNOWN_IDENTIFIER); throw Exception(PreformattedMessage{ss.str(), format_string}, ErrorCodes::UNKNOWN_IDENTIFIER);
} }
@ -1186,6 +1186,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
{ {
source_column_names.insert(column.name); source_column_names.insert(column.name);
} }
return true;
} }
NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const
@ -1395,7 +1396,9 @@ TreeRewriterResultPtr TreeRewriter::analyze(
else else
assertNoAggregates(query, "in wrong place"); assertNoAggregates(query, "in wrong place");
result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key); bool is_ok = result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key, no_throw);
if (!is_ok)
return {};
return std::make_shared<const TreeRewriterResult>(result); return std::make_shared<const TreeRewriterResult>(result);
} }

View File

@ -87,7 +87,7 @@ struct TreeRewriterResult
bool add_special = true); bool add_special = true);
void collectSourceColumns(bool add_special); void collectSourceColumns(bool add_special);
void collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint); bool collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw = false);
Names requiredSourceColumns() const { return required_source_columns.getNames(); } Names requiredSourceColumns() const { return required_source_columns.getNames(); }
const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; } const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; }
NameSet getArrayJoinSourceNameSet() const; NameSet getArrayJoinSourceNameSet() const;
@ -108,7 +108,10 @@ using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
class TreeRewriter : WithContext class TreeRewriter : WithContext
{ {
public: public:
explicit TreeRewriter(ContextPtr context_) : WithContext(context_) {} explicit TreeRewriter(ContextPtr context_, bool no_throw_ = false)
: WithContext(context_)
, no_throw(no_throw_)
{}
/// Analyze and rewrite not select query /// Analyze and rewrite not select query
TreeRewriterResultPtr analyze( TreeRewriterResultPtr analyze(
@ -132,6 +135,9 @@ public:
private: private:
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false); static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false);
/// Do not throw exception from analyze on unknown identifiers, but only return nullptr.
bool no_throw = false;
}; };
} }

View File

@ -28,7 +28,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
static std::pair<Field, std::shared_ptr<const IDataType>> getFieldAndDataTypeFromLiteral(ASTLiteral * literal) static EvaluateConstantExpressionResult getFieldAndDataTypeFromLiteral(ASTLiteral * literal)
{ {
auto type = applyVisitor(FieldToDataType(), literal->value); auto type = applyVisitor(FieldToDataType(), literal->value);
/// In case of Array field nested fields can have different types. /// In case of Array field nested fields can have different types.
@ -39,7 +39,7 @@ static std::pair<Field, std::shared_ptr<const IDataType>> getFieldAndDataTypeFro
return {res, type}; return {res, type};
} }
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context) std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(const ASTPtr & node, const ContextPtr & context, bool no_throw)
{ {
if (ASTLiteral * literal = node->as<ASTLiteral>()) if (ASTLiteral * literal = node->as<ASTLiteral>())
return getFieldAndDataTypeFromLiteral(literal); return getFieldAndDataTypeFromLiteral(literal);
@ -67,7 +67,9 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
if (context->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && context->getSettingsRef().normalize_function_names) if (context->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && context->getSettingsRef().normalize_function_names)
FunctionNameNormalizer().visit(ast.get()); FunctionNameNormalizer().visit(ast.get());
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns); auto syntax_result = TreeRewriter(context, no_throw).analyze(ast, source_columns);
if (!syntax_result)
return {};
/// AST potentially could be transformed to literal during TreeRewriter analyze. /// AST potentially could be transformed to literal during TreeRewriter analyze.
/// For example if we have SQL user defined function that return literal AS subquery. /// For example if we have SQL user defined function that return literal AS subquery.
@ -108,6 +110,18 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
return std::make_pair((*result_column)[0], result_type); return std::make_pair((*result_column)[0], result_type);
} }
std::optional<EvaluateConstantExpressionResult> tryEvaluateConstantExpression(const ASTPtr & node, const ContextPtr & context)
{
return evaluateConstantExpressionImpl(node, context, true);
}
EvaluateConstantExpressionResult evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context)
{
auto res = evaluateConstantExpressionImpl(node, context, false);
if (!res)
throw Exception(ErrorCodes::LOGICAL_ERROR, "evaluateConstantExpression expected to return a result or throw an exception");
return *res;
}
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context) ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context)
{ {

View File

@ -17,13 +17,16 @@ class IDataType;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>; using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
using EvaluateConstantExpressionResult = std::pair<Field, std::shared_ptr<const IDataType>>;
/** Evaluate constant expression and its type. /** Evaluate constant expression and its type.
* Used in rare cases - for elements of set for IN, for data to INSERT. * Used in rare cases - for elements of set for IN, for data to INSERT.
* Throws exception if it's not a constant expression. * Throws exception if it's not a constant expression.
* Quite suboptimal. * Quite suboptimal.
*/ */
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context); EvaluateConstantExpressionResult evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context);
std::optional<EvaluateConstantExpressionResult> tryEvaluateConstantExpression(const ASTPtr & node, const ContextPtr & context);
/** Evaluate constant expression and returns ASTLiteral with its value. /** Evaluate constant expression and returns ASTLiteral with its value.
*/ */

View File

@ -720,6 +720,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
case TypeIndex::Decimal256: case TypeIndex::Decimal256:
return readColumnWithBigNumberFromBinaryData<ColumnDecimal<Decimal256>>(arrow_column, column_name, type_hint); return readColumnWithBigNumberFromBinaryData<ColumnDecimal<Decimal256>>(arrow_column, column_name, type_hint);
default: default:
break;
} }
} }
return readColumnWithStringData<arrow::BinaryArray>(arrow_column, column_name); return readColumnWithStringData<arrow::BinaryArray>(arrow_column, column_name);
@ -739,6 +740,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
case TypeIndex::UInt256: case TypeIndex::UInt256:
return readColumnWithBigIntegerFromFixedBinaryData<UInt256>(arrow_column, column_name, type_hint); return readColumnWithBigIntegerFromFixedBinaryData<UInt256>(arrow_column, column_name, type_hint);
default: default:
break;
} }
} }

View File

@ -245,6 +245,7 @@ static void insertString(IColumn & column, DataTypePtr type, const char * value,
insertFromBinaryRepresentation<ColumnDecimal<Decimal256>>(column, type, value, size); insertFromBinaryRepresentation<ColumnDecimal<Decimal256>>(column, type, value, size);
return; return;
default: default:
break;
} }
} }

View File

@ -44,11 +44,7 @@ std::unique_ptr<QueryPlan> createLocalPlan(
ContextPtr context, ContextPtr context,
QueryProcessingStage::Enum processed_stage, QueryProcessingStage::Enum processed_stage,
size_t shard_num, size_t shard_num,
size_t shard_count, size_t shard_count)
size_t replica_num,
size_t replica_count,
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator,
UUID group_uuid)
{ {
checkStackSize(); checkStackSize();
@ -67,26 +63,6 @@ std::unique_ptr<QueryPlan> createLocalPlan(
.setShardInfo(static_cast<UInt32>(shard_num), static_cast<UInt32>(shard_count)) .setShardInfo(static_cast<UInt32>(shard_num), static_cast<UInt32>(shard_count))
.ignoreASTOptimizations(); .ignoreASTOptimizations();
/// There are much things that are needed for coordination
/// during reading with parallel replicas
if (coordinator)
{
new_context->parallel_reading_coordinator = coordinator;
new_context->setClientInterface(ClientInfo::Interface::LOCAL);
new_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY);
new_context->setReplicaInfo(true, replica_count, replica_num);
new_context->setConnectionClientVersion(VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH, DBMS_TCP_PROTOCOL_VERSION);
new_context->setParallelReplicasGroupUUID(group_uuid);
new_context->setMergeTreeAllRangesCallback([coordinator](InitialAllRangesAnnouncement announcement)
{
coordinator->handleInitialAllRangesAnnouncement(announcement);
});
new_context->setMergeTreeReadTaskCallback([coordinator](ParallelReadRequest request) -> std::optional<ParallelReadResponse>
{
return coordinator->handleRequest(request);
});
}
if (context->getSettingsRef().allow_experimental_analyzer) if (context->getSettingsRef().allow_experimental_analyzer)
{ {
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options);

View File

@ -19,10 +19,5 @@ std::unique_ptr<QueryPlan> createLocalPlan(
ContextPtr context, ContextPtr context,
QueryProcessingStage::Enum processed_stage, QueryProcessingStage::Enum processed_stage,
size_t shard_num, size_t shard_num,
size_t shard_count, size_t shard_count);
size_t replica_num,
size_t replica_count,
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator,
UUID group_uuid = UUIDHelpers::Nil);
} }

View File

@ -1226,6 +1226,7 @@ static void buildIndexes(
std::optional<ReadFromMergeTree::Indexes> & indexes, std::optional<ReadFromMergeTree::Indexes> & indexes,
ActionsDAGPtr filter_actions_dag, ActionsDAGPtr filter_actions_dag,
const MergeTreeData & data, const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,
const ContextPtr & context, const ContextPtr & context,
const SelectQueryInfo & query_info, const SelectQueryInfo & query_info,
const StorageMetadataPtr & metadata_snapshot) const StorageMetadataPtr & metadata_snapshot)
@ -1248,7 +1249,7 @@ static void buildIndexes(
context, context,
primary_key_column_names, primary_key_column_names,
primary_key.expression, primary_key.expression,
array_join_name_set}, {}, {}, {}, false}); array_join_name_set}, {}, {}, {}, false, {}});
} }
else else
{ {
@ -1256,7 +1257,7 @@ static void buildIndexes(
query_info, query_info,
context, context,
primary_key_column_names, primary_key_column_names,
primary_key.expression}, {}, {}, {}, false}); primary_key.expression}, {}, {}, {}, false, {}});
} }
if (metadata_snapshot->hasPartitionKey()) if (metadata_snapshot->hasPartitionKey())
@ -1269,6 +1270,9 @@ static void buildIndexes(
indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */);
} }
/// TODO Support row_policy_filter and additional_filters
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context);
indexes->use_skip_indexes = settings.use_skip_indexes; indexes->use_skip_indexes = settings.use_skip_indexes;
bool final = query_info.isFinal(); bool final = query_info.isFinal();
@ -1346,7 +1350,7 @@ static void buildIndexes(
void ReadFromMergeTree::applyFilters() void ReadFromMergeTree::applyFilters()
{ {
auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info);
buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading); buildIndexes(indexes, filter_actions_dag, data, prepared_parts, context, query_info, metadata_for_reading);
} }
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
@ -1424,11 +1428,6 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
size_t total_parts = parts.size(); size_t total_parts = parts.size();
/// TODO Support row_policy_filter and additional_filters
auto part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context);
if (part_values && part_values->empty())
return std::make_shared<MergeTreeDataSelectAnalysisResult>(MergeTreeDataSelectAnalysisResult{.result = std::move(result)});
result.column_names_to_read = real_column_names; 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 there are only virtual columns in the query, you must request at least one non-virtual one.
@ -1443,7 +1442,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
const Names & primary_key_column_names = primary_key.column_names; const Names & primary_key_column_names = primary_key.column_names;
if (!indexes) if (!indexes)
buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot); buildIndexes(indexes, query_info.filter_actions_dag, data, parts, context, query_info, metadata_snapshot);
if (indexes->part_values && indexes->part_values->empty())
return std::make_shared<MergeTreeDataSelectAnalysisResult>(MergeTreeDataSelectAnalysisResult{.result = std::move(result)});
if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue())
{ {
@ -1467,7 +1469,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
indexes->minmax_idx_condition, indexes->minmax_idx_condition,
parts, parts,
alter_conversions, alter_conversions,
part_values, indexes->part_values,
metadata_snapshot_base, metadata_snapshot_base,
data, data,
context, context,

View File

@ -171,6 +171,7 @@ public:
std::optional<KeyCondition> minmax_idx_condition; std::optional<KeyCondition> minmax_idx_condition;
UsefulSkipIndexes skip_indexes; UsefulSkipIndexes skip_indexes;
bool use_skip_indexes; bool use_skip_indexes;
std::optional<std::unordered_set<String>> part_values;
}; };
static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(

View File

@ -187,7 +187,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream
if (try_results.empty() || local_delay < max_remote_delay) if (try_results.empty() || local_delay < max_remote_delay)
{ {
auto plan = createLocalPlan( auto plan = createLocalPlan(
query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count);
return std::move(*plan->buildQueryPipeline( return std::move(*plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(my_context), QueryPlanOptimizationSettings::fromContext(my_context),
@ -245,6 +245,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact
LOG_INFO(log, "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is used: {}", LOG_INFO(log, "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is used: {}",
cluster_for_parallel_replicas, cluster_name); cluster_for_parallel_replicas, cluster_name);
} }
LOG_TRACE(&Poco::Logger::get("ReadFromRemote"), "Setting `cluster_for_parallel_replicas` to {}", cluster_name);
context->setSetting("cluster_for_parallel_replicas", cluster_name); context->setSetting("cluster_for_parallel_replicas", cluster_name);
} }

View File

@ -57,7 +57,7 @@ private:
std::shared_ptr<const StorageLimitsList> storage_limits; std::shared_ptr<const StorageLimitsList> storage_limits;
Poco::Logger * log; Poco::Logger * log;
UInt32 shard_count; UInt32 shard_count;
String cluster_name; const String cluster_name;
void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard); void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard); void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);

View File

@ -595,7 +595,7 @@ void HTTPHandler::processQuery(
size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE;
size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0;
unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
used_output.out = std::make_shared<WriteBufferFromHTTPServerResponse>( used_output.out = std::make_shared<WriteBufferFromHTTPServerResponse>(
response, response,

View File

@ -88,7 +88,7 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe
Output used_output; Output used_output;
const auto & config = server.config(); const auto & config = server.config();
unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
used_output.out = std::make_shared<WriteBufferFromHTTPServerResponse>( used_output.out = std::make_shared<WriteBufferFromHTTPServerResponse>(
response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout);

View File

@ -18,7 +18,7 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
try try
{ {
const auto & config = server.config(); const auto & config = server.config();
unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
setResponseDefaultHeaders(response, keep_alive_timeout); setResponseDefaultHeaders(response, keep_alive_timeout);

View File

@ -79,7 +79,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
} }
const auto & config = getContext()->getConfigRef(); const auto & config = getContext()->getConfigRef();
setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT));
if (!ok) if (!ok)
{ {

View File

@ -90,7 +90,7 @@ static inline void trySendExceptionToClient(
void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
auto keep_alive_timeout = server.config().getUInt("keep_alive_timeout", 10); auto keep_alive_timeout = server.config().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
const auto & out = responseWriteBuffer(request, response, keep_alive_timeout); const auto & out = responseWriteBuffer(request, response, keep_alive_timeout);
try try

View File

@ -30,7 +30,7 @@ WebUIRequestHandler::WebUIRequestHandler(IServer & server_)
void WebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void WebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
auto keep_alive_timeout = server.config().getUInt("keep_alive_timeout", 10); auto keep_alive_timeout = server.config().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
response.setContentType("text/html; charset=UTF-8"); response.setContentType("text/html; charset=UTF-8");

View File

@ -57,7 +57,7 @@ LocalFileHolder::~LocalFileHolder()
{ {
if (original_readbuffer) if (original_readbuffer)
{ {
dynamic_cast<SeekableReadBuffer *>(original_readbuffer.get())->seek(0, SEEK_SET); assert_cast<SeekableReadBuffer *>(original_readbuffer.get())->seek(0, SEEK_SET);
file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool); file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool);
} }
} }
@ -122,7 +122,7 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence)
{ {
if (local_file_holder->original_readbuffer) if (local_file_holder->original_readbuffer)
{ {
auto ret = dynamic_cast<SeekableReadBuffer *>(local_file_holder->original_readbuffer.get())->seek(offset, whence); auto ret = assert_cast<SeekableReadBuffer *>(local_file_holder->original_readbuffer.get())->seek(offset, whence);
BufferBase::set( BufferBase::set(
local_file_holder->original_readbuffer->buffer().begin(), local_file_holder->original_readbuffer->buffer().begin(),
local_file_holder->original_readbuffer->buffer().size(), local_file_holder->original_readbuffer->buffer().size(),
@ -147,7 +147,7 @@ off_t RemoteReadBuffer::getPosition()
{ {
if (local_file_holder->original_readbuffer) if (local_file_holder->original_readbuffer)
{ {
return dynamic_cast<SeekableReadBuffer *>(local_file_holder->original_readbuffer.get())->getPosition(); return assert_cast<SeekableReadBuffer *>(local_file_holder->original_readbuffer.get())->getPosition();
} }
return local_file_holder->file_buffer->getPosition(); return local_file_holder->file_buffer->getPosition();
} }

View File

@ -1909,6 +1909,13 @@ void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) cons
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'checkConsistency' is not implemented for part with type {}", getType().toString()); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'checkConsistency' is not implemented for part with type {}", getType().toString());
} }
void IMergeTreeDataPart::checkConsistencyWithProjections(bool require_part_metadata) const
{
checkConsistency(require_part_metadata);
for (const auto & [_, proj_part] : projection_parts)
proj_part->checkConsistency(require_part_metadata);
}
void IMergeTreeDataPart::calculateColumnsAndSecondaryIndicesSizesOnDisk() void IMergeTreeDataPart::calculateColumnsAndSecondaryIndicesSizesOnDisk()
{ {
calculateColumnsSizesOnDisk(); calculateColumnsSizesOnDisk();

View File

@ -489,6 +489,12 @@ public:
void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings); void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings);
/// Checks the consistency of this data part.
virtual void checkConsistency(bool require_part_metadata) const;
/// Checks the consistency of this data part, and check the consistency of its projections (if any) as well.
void checkConsistencyWithProjections(bool require_part_metadata) const;
/// "delete-on-destroy.txt" is deprecated. It is no longer being created, only is removed. /// "delete-on-destroy.txt" is deprecated. It is no longer being created, only is removed.
/// TODO: remove this method after some time. /// TODO: remove this method after some time.
void removeDeleteOnDestroyMarker(); void removeDeleteOnDestroyMarker();
@ -534,7 +540,6 @@ protected:
void removeIfNeeded(); void removeIfNeeded();
virtual void checkConsistency(bool require_part_metadata) const;
void checkConsistencyBase() const; void checkConsistencyBase() const;
/// Fill each_columns_size and total_size with sizes from columns files on /// Fill each_columns_size and total_size with sizes from columns files on

View File

@ -1244,32 +1244,13 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart(
.withPartFormatFromDisk() .withPartFormatFromDisk()
.build(); .build();
} }
catch (const Exception & e) catch (...)
{ {
/// Don't count the part as broken if there was a retryalbe error /// Don't count the part as broken if there was a retryalbe error
/// during loading, such as "not enough memory" or network error. /// during loading, such as "not enough memory" or network error.
if (isRetryableException(e)) if (isRetryableException(std::current_exception()))
throw; throw;
LOG_DEBUG(log, "Failed to load data part {}, unknown exception", part_name);
mark_broken();
return res;
}
catch (const Poco::Net::NetException &)
{
throw;
}
catch (const Poco::TimeoutException &)
{
throw;
}
#if USE_AZURE_BLOB_STORAGE
catch (const Azure::Core::Http::TransportException &)
{
throw;
}
#endif
catch (...)
{
mark_broken(); mark_broken();
return res; return res;
} }
@ -1294,18 +1275,12 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart(
{ {
res.part->loadColumnsChecksumsIndexes(require_part_metadata, true); res.part->loadColumnsChecksumsIndexes(require_part_metadata, true);
} }
catch (const Exception & e) catch (...)
{ {
/// Don't count the part as broken if there was a retryalbe error /// Don't count the part as broken if there was a retryalbe error
/// during loading, such as "not enough memory" or network error. /// during loading, such as "not enough memory" or network error.
if (isRetryableException(e)) if (isRetryableException(std::current_exception()))
throw; throw;
mark_broken();
return res;
}
catch (...)
{
mark_broken(); mark_broken();
return res; return res;
} }
@ -1416,11 +1391,28 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPartWithRetries(
size_t max_backoff_ms, size_t max_backoff_ms,
size_t max_tries) size_t max_tries)
{ {
auto handle_exception = [&, this](String exception_message, size_t try_no) auto handle_exception = [&, this](std::exception_ptr exception_ptr, size_t try_no)
{ {
if (try_no + 1 == max_tries) if (try_no + 1 == max_tries)
throw; throw;
String exception_message;
try
{
rethrow_exception(exception_ptr);
}
catch (const Exception & e)
{
exception_message = e.message();
}
#if USE_AZURE_BLOB_STORAGE
catch (const Azure::Core::RequestFailedException & e)
{
exception_message = e.Message;
}
#endif
LOG_DEBUG(log, "Failed to load data part {} at try {} with retryable error: {}. Will retry in {} ms", LOG_DEBUG(log, "Failed to load data part {} at try {} with retryable error: {}. Will retry in {} ms",
part_name, try_no, exception_message, initial_backoff_ms); part_name, try_no, exception_message, initial_backoff_ms);
@ -1434,19 +1426,13 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPartWithRetries(
{ {
return loadDataPart(part_info, part_name, part_disk_ptr, to_state, part_loading_mutex); return loadDataPart(part_info, part_name, part_disk_ptr, to_state, part_loading_mutex);
} }
catch (const Exception & e) catch (...)
{ {
if (isRetryableException(e)) if (isRetryableException(std::current_exception()))
handle_exception(e.message(),try_no); handle_exception(std::current_exception(),try_no);
else else
throw; throw;
} }
#if USE_AZURE_BLOB_STORAGE
catch (const Azure::Core::Http::TransportException & e)
{
handle_exception(e.Message,try_no);
}
#endif
} }
UNREACHABLE(); UNREACHABLE();
} }
@ -4636,7 +4622,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy, DataPar
/// Move parts are non replicated operations, so we take lock here. /// Move parts are non replicated operations, so we take lock here.
/// All other locks are taken in StorageReplicatedMergeTree /// All other locks are taken in StorageReplicatedMergeTree
lockSharedData(*part_copy); lockSharedData(*part_copy, /* replace_existing_lock */ true);
return; return;
} }
@ -5255,6 +5241,9 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts(
if (hold_table_lock && !table_lock) if (hold_table_lock && !table_lock)
table_lock = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); table_lock = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
if (backup_settings.check_parts)
part->checkConsistencyWithProjections(/* require_part_metadata= */ true);
BackupEntries backup_entries_from_part; BackupEntries backup_entries_from_part;
part->getDataPartStorage().backup( part->getDataPartStorage().backup(
part->checksums, part->checksums,
@ -5315,8 +5304,8 @@ void MergeTreeData::restoreDataFromBackup(RestorerFromBackup & restorer, const S
class MergeTreeData::RestoredPartsHolder class MergeTreeData::RestoredPartsHolder
{ {
public: public:
RestoredPartsHolder(const std::shared_ptr<MergeTreeData> & storage_, const BackupPtr & backup_, size_t num_parts_) RestoredPartsHolder(const std::shared_ptr<MergeTreeData> & storage_, const BackupPtr & backup_)
: storage(storage_), backup(backup_), num_parts(num_parts_) : storage(storage_), backup(backup_)
{ {
} }
@ -5329,6 +5318,13 @@ public:
attachIfAllPartsRestored(); attachIfAllPartsRestored();
} }
void increaseNumBrokenParts()
{
std::lock_guard lock{mutex};
++num_broken_parts;
attachIfAllPartsRestored();
}
void addPart(MutableDataPartPtr part) void addPart(MutableDataPartPtr part)
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
@ -5348,7 +5344,7 @@ public:
private: private:
void attachIfAllPartsRestored() void attachIfAllPartsRestored()
{ {
if (!num_parts || (parts.size() < num_parts)) if (!num_parts || (parts.size() + num_broken_parts < num_parts))
return; return;
/// Sort parts by min_block (because we need to preserve the order of parts). /// Sort parts by min_block (because we need to preserve the order of parts).
@ -5363,9 +5359,10 @@ private:
num_parts = 0; num_parts = 0;
} }
std::shared_ptr<MergeTreeData> storage; const std::shared_ptr<MergeTreeData> storage;
BackupPtr backup; const BackupPtr backup;
size_t num_parts = 0; size_t num_parts = 0;
size_t num_broken_parts = 0;
MutableDataPartsVector parts; MutableDataPartsVector parts;
std::map<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>> temp_dirs; std::map<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>> temp_dirs;
mutable std::mutex mutex; mutable std::mutex mutex;
@ -5381,8 +5378,9 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
Strings part_names = backup->listFiles(data_path_in_backup); Strings part_names = backup->listFiles(data_path_in_backup);
boost::remove_erase(part_names, "mutations"); boost::remove_erase(part_names, "mutations");
auto restored_parts_holder bool restore_broken_parts_as_detached = restorer.getRestoreSettings().restore_broken_parts_as_detached;
= std::make_shared<RestoredPartsHolder>(std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup, part_names.size());
auto restored_parts_holder = std::make_shared<RestoredPartsHolder>(std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup);
fs::path data_path_in_backup_fs = data_path_in_backup; fs::path data_path_in_backup_fs = data_path_in_backup;
size_t num_parts = 0; size_t num_parts = 0;
@ -5404,8 +5402,9 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
backup, backup,
part_path_in_backup = data_path_in_backup_fs / part_name, part_path_in_backup = data_path_in_backup_fs / part_name,
my_part_info = *part_info, my_part_info = *part_info,
restore_broken_parts_as_detached,
restored_parts_holder] restored_parts_holder]
{ storage->restorePartFromBackup(restored_parts_holder, my_part_info, part_path_in_backup); }); { storage->restorePartFromBackup(restored_parts_holder, my_part_info, part_path_in_backup, restore_broken_parts_as_detached); });
++num_parts; ++num_parts;
} }
@ -5413,11 +5412,12 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
restored_parts_holder->setNumParts(num_parts); restored_parts_holder->setNumParts(num_parts);
} }
void MergeTreeData::restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) const void MergeTreeData::restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup, bool detach_if_broken) const
{ {
String part_name = part_info.getPartNameAndCheckFormat(format_version); String part_name = part_info.getPartNameAndCheckFormat(format_version);
auto backup = restored_parts_holder->getBackup(); auto backup = restored_parts_holder->getBackup();
/// Calculate the total size of the part.
UInt64 total_size_of_part = 0; UInt64 total_size_of_part = 0;
Strings filenames = backup->listFiles(part_path_in_backup, /* recursive= */ true); Strings filenames = backup->listFiles(part_path_in_backup, /* recursive= */ true);
fs::path part_path_in_backup_fs = part_path_in_backup; fs::path part_path_in_backup_fs = part_path_in_backup;
@ -5425,21 +5425,22 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> r
total_size_of_part += backup->getFileSize(part_path_in_backup_fs / filename); total_size_of_part += backup->getFileSize(part_path_in_backup_fs / filename);
std::shared_ptr<IReservation> reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part); std::shared_ptr<IReservation> reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part);
auto disk = reservation->getDisk();
fs::path temp_dir = restored_parts_holder->getTemporaryDirectory(disk); /// Calculate paths, for example:
fs::path temp_part_dir = temp_dir / part_path_in_backup_fs.relative_path();
disk->createDirectories(temp_part_dir);
/// For example:
/// part_name = 0_1_1_0 /// part_name = 0_1_1_0
/// part_path_in_backup = /data/test/table/0_1_1_0 /// part_path_in_backup = /data/test/table/0_1_1_0
/// tmp_dir = tmp/1aaaaaa /// tmp_dir = tmp/1aaaaaa
/// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0 /// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0
auto disk = reservation->getDisk();
fs::path temp_dir = restored_parts_holder->getTemporaryDirectory(disk);
fs::path temp_part_dir = temp_dir / part_path_in_backup_fs.relative_path();
/// Subdirectories in the part's directory. It's used to restore projections. /// Subdirectories in the part's directory. It's used to restore projections.
std::unordered_set<String> subdirs; std::unordered_set<String> subdirs;
/// Copy files from the backup to the directory `tmp_part_dir`.
disk->createDirectories(temp_part_dir);
for (const String & filename : filenames) for (const String & filename : filenames)
{ {
/// Needs to create subdirectories before copying the files. Subdirectories are used to represent projections. /// Needs to create subdirectories before copying the files. Subdirectories are used to represent projections.
@ -5459,14 +5460,102 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> r
reservation->update(reservation->getSize() - file_size); reservation->update(reservation->getSize() - file_size);
} }
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0); if (auto part = loadPartRestoredFromBackup(disk, temp_part_dir.parent_path(), part_name, detach_if_broken))
MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, temp_part_dir.parent_path(), part_name);
builder.withPartFormatFromDisk();
auto part = std::move(builder).build();
part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
part->loadColumnsChecksumsIndexes(false, true);
restored_parts_holder->addPart(part); restored_parts_holder->addPart(part);
else
restored_parts_holder->increaseNumBrokenParts();
}
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(const DiskPtr & disk, const String & temp_dir, const String & part_name, bool detach_if_broken) const
{
MutableDataPartPtr part;
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
/// Load this part from the directory `tmp_part_dir`.
auto load_part = [&]
{
MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, temp_dir, part_name);
builder.withPartFormatFromDisk();
part = std::move(builder).build();
part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
part->loadColumnsChecksumsIndexes(/* require_columns_checksums= */ false, /* check_consistency= */ true);
};
/// Broken parts can appear in a backup sometimes.
auto mark_broken = [&](const std::exception_ptr error)
{
tryLogException(error, log,
fmt::format("Part {} will be restored as detached because it's broken. You need to resolve this manually", part_name));
if (!part)
{
/// Make a fake data part only to copy its files to /detached/.
part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, temp_dir, part_name}
.withPartStorageType(MergeTreeDataPartStorageType::Full)
.withPartType(MergeTreeDataPartType::Wide)
.build();
}
part->renameToDetached("broken-from-backup");
};
/// Try to load this part multiple times.
auto backoff_ms = loading_parts_initial_backoff_ms;
for (size_t try_no = 0; try_no < loading_parts_max_tries; ++try_no)
{
std::exception_ptr error;
bool retryable = false;
try
{
load_part();
}
catch (const Poco::Net::NetException &)
{
error = std::current_exception();
retryable = true;
}
catch (const Poco::TimeoutException &)
{
error = std::current_exception();
retryable = true;
}
catch (...)
{
error = std::current_exception();
retryable = isRetryableException(std::current_exception());
}
if (!error)
return part;
if (!retryable && detach_if_broken)
{
mark_broken(error);
return nullptr;
}
if (!retryable)
{
LOG_ERROR(log,
"Failed to restore part {} because it's broken. You can skip broken parts while restoring by setting "
"'restore_broken_parts_as_detached = true'",
part_name);
}
if (!retryable || (try_no + 1 == loading_parts_max_tries))
{
if (Exception * e = exception_cast<Exception *>(error))
e->addMessage("while restoring part {} of table {}", part->name, getStorageID());
std::rethrow_exception(error);
}
tryLogException(error, log,
fmt::format("Failed to load part {} at try {} with a retryable error. Will retry in {} ms", part_name, try_no, backoff_ms));
std::this_thread::sleep_for(std::chrono::milliseconds(backoff_ms));
backoff_ms = std::min(backoff_ms * 2, loading_parts_max_backoff_ms);
}
UNREACHABLE();
} }

View File

@ -1357,7 +1357,8 @@ protected:
/// Restores the parts of this table from backup. /// Restores the parts of this table from backup.
void restorePartsFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions); void restorePartsFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions);
void restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) const; void restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup, bool detach_if_broken) const;
MutableDataPartPtr loadPartRestoredFromBackup(const DiskPtr & disk, const String & temp_dir, const String & part_name, bool detach_if_broken) const;
/// Attaches restored parts to the storage. /// Attaches restored parts to the storage.
virtual void attachRestoredParts(MutableDataPartsVector && parts) = 0; virtual void attachRestoredParts(MutableDataPartsVector && parts) = 0;

View File

@ -43,6 +43,7 @@
#include <DataTypes/DataTypeTuple.h> #include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <Functions/IFunction.h>
#include <IO/WriteBufferFromOStream.h> #include <IO/WriteBufferFromOStream.h>
@ -772,6 +773,37 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling(
return sampling; return sampling;
} }
std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(
const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,
const ActionsDAGPtr & filter_dag,
ContextPtr context)
{
if (!filter_dag)
return {};
auto sample = data.getSampleBlockWithVirtualColumns();
std::unordered_set<const ActionsDAG::Node *> allowed_inputs;
for (const auto * input : filter_dag->getInputs())
if (sample.has(input->result_name))
allowed_inputs.insert(input);
if (allowed_inputs.empty())
return {};
auto atoms = filter_dag->extractConjunctionAtoms(filter_dag->getOutputs().at(0));
atoms = ActionsDAG::filterNodesByAllowedInputs(std::move(atoms), allowed_inputs);
if (atoms.empty())
return {};
auto dag = ActionsDAG::buildFilterActionsDAG(atoms, {}, context);
auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */);
VirtualColumnUtils::filterBlockWithQuery(dag, virtual_columns_block, context);
return VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
}
std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(
const MergeTreeData & data, const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts, const MergeTreeData::DataPartsVector & parts,

View File

@ -170,6 +170,12 @@ public:
const ASTPtr & query, const ASTPtr & query,
ContextPtr context); ContextPtr context);
static std::optional<std::unordered_set<String>> filterPartsByVirtualColumns(
const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,
const ActionsDAGPtr & filter_dag,
ContextPtr context);
/// Filter parts using minmax index and partition key. /// Filter parts using minmax index and partition key.
static void filterPartsByPartition( static void filterPartsByPartition(
std::optional<PartitionPruner> & partition_pruner, std::optional<PartitionPruner> & partition_pruner,

View File

@ -163,7 +163,8 @@ struct DetachedPartInfo : public MergeTreePartInfo
"tmp-fetch", "tmp-fetch",
"covered-by-broken", "covered-by-broken",
"merge-not-byte-identical", "merge-not-byte-identical",
"mutate-not-byte-identical" "mutate-not-byte-identical",
"broken-from-backup",
}); });
static constexpr auto DETACHED_REASONS_REMOVABLE_BY_TIMEOUT = std::to_array<std::string_view>({ static constexpr auto DETACHED_REASONS_REMOVABLE_BY_TIMEOUT = std::to_array<std::string_view>({
@ -175,7 +176,8 @@ struct DetachedPartInfo : public MergeTreePartInfo
"deleting", "deleting",
"clone", "clone",
"merge-not-byte-identical", "merge-not-byte-identical",
"mutate-not-byte-identical" "mutate-not-byte-identical",
"broken-from-backup",
}); });
/// NOTE: It may parse part info incorrectly. /// NOTE: It may parse part info incorrectly.

View File

@ -112,14 +112,9 @@ void MergeTreeReaderCompact::initialize()
compressed_data_buffer = non_cached_buffer.get(); compressed_data_buffer = non_cached_buffer.get();
} }
} }
catch (const Exception & e)
{
if (!isRetryableException(e))
data_part_info_for_read->reportBroken();
throw;
}
catch (...) catch (...)
{ {
if (!isRetryableException(std::current_exception()))
data_part_info_for_read->reportBroken(); data_part_info_for_read->reportBroken();
throw; throw;
} }
@ -239,18 +234,21 @@ size_t MergeTreeReaderCompact::readRows(
"Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.", "Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.",
read_rows_in_column, rows_to_read); read_rows_in_column, rows_to_read);
} }
catch (Exception & e) catch (...)
{ {
if (!isRetryableException(e)) if (!isRetryableException(std::current_exception()))
data_part_info_for_read->reportBroken(); data_part_info_for_read->reportBroken();
/// Better diagnostics. /// Better diagnostics.
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read)); try
throw;
}
catch (...)
{ {
data_part_info_for_read->reportBroken(); rethrow_exception(std::current_exception());
}
catch (Exception & e)
{
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
}
throw; throw;
} }
} }
@ -396,14 +394,9 @@ try
seekToMark(all_mark_ranges.front().begin, 0); seekToMark(all_mark_ranges.front().begin, 0);
data_buffer->prefetch(priority); data_buffer->prefetch(priority);
} }
catch (const Exception & e)
{
if (!isRetryableException(e))
data_part_info_for_read->reportBroken();
throw;
}
catch (...) catch (...)
{ {
if (!isRetryableException(std::current_exception()))
data_part_info_for_read->reportBroken(); data_part_info_for_read->reportBroken();
throw; throw;
} }

View File

@ -47,14 +47,9 @@ MergeTreeReaderWide::MergeTreeReaderWide(
for (size_t i = 0; i < columns_to_read.size(); ++i) for (size_t i = 0; i < columns_to_read.size(); ++i)
addStreams(columns_to_read[i], serializations[i], profile_callback_, clock_type_); addStreams(columns_to_read[i], serializations[i], profile_callback_, clock_type_);
} }
catch (const Exception & e)
{
if (!isRetryableException(e))
data_part_info_for_read->reportBroken();
throw;
}
catch (...) catch (...)
{ {
if (!isRetryableException(std::current_exception()))
data_part_info_for_read->reportBroken(); data_part_info_for_read->reportBroken();
throw; throw;
} }
@ -78,14 +73,9 @@ void MergeTreeReaderWide::prefetchBeginOfRange(Priority priority)
/// of range only once so there is no such problem. /// of range only once so there is no such problem.
/// 4. continue_reading == false, as we haven't read anything yet. /// 4. continue_reading == false, as we haven't read anything yet.
} }
catch (const Exception & e)
{
if (!isRetryableException(e))
data_part_info_for_read->reportBroken();
throw;
}
catch (...) catch (...)
{ {
if (!isRetryableException(std::current_exception()))
data_part_info_for_read->reportBroken(); data_part_info_for_read->reportBroken();
throw; throw;
} }
@ -184,18 +174,21 @@ size_t MergeTreeReaderWide::readRows(
/// In particular, even if for some streams there are no rows to be read, /// In particular, even if for some streams there are no rows to be read,
/// you must ensure that no seeks are skipped and at this point they all point to to_mark. /// you must ensure that no seeks are skipped and at this point they all point to to_mark.
} }
catch (Exception & e) catch (...)
{ {
if (!isRetryableException(e)) if (!isRetryableException(std::current_exception()))
data_part_info_for_read->reportBroken(); data_part_info_for_read->reportBroken();
/// Better diagnostics. /// Better diagnostics.
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read)); try
throw;
}
catch (...)
{ {
data_part_info_for_read->reportBroken(); rethrow_exception(std::current_exception());
}
catch (Exception & e)
{
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
}
throw; throw;
} }

View File

@ -377,13 +377,9 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St
result.action = ReplicatedCheckResult::DoNothing; result.action = ReplicatedCheckResult::DoNothing;
return result; return result;
} }
catch (const Exception & e) catch (...)
{ {
/// Don't count the part as broken if we got known retryable exception. if (isRetryableException(std::current_exception()))
/// In fact, there can be other similar situations because not all
/// of the exceptions are classified as retryable/non-retryable. But it is OK,
/// because there is a safety guard against deleting too many parts.
if (isRetryableException(e))
throw; throw;
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
@ -395,6 +391,7 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St
result.status = {part_name, false, message}; result.status = {part_name, false, message};
result.action = ReplicatedCheckResult::TryFetchMissing; result.action = ReplicatedCheckResult::TryFetchMissing;
return result; return result;
} }
} }
else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < current_time) else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < current_time)

View File

@ -15,6 +15,11 @@
#include <IO/HashingReadBuffer.h> #include <IO/HashingReadBuffer.h>
#include <IO/S3Common.h> #include <IO/S3Common.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Poco/Net/NetException.h>
#if USE_AZURE_BLOB_STORAGE
#include <azure/core/http/http.hpp>
#endif
namespace CurrentMetrics namespace CurrentMetrics
{ {
@ -49,19 +54,41 @@ bool isNotEnoughMemoryErrorCode(int code)
|| code == ErrorCodes::CANNOT_MREMAP; || code == ErrorCodes::CANNOT_MREMAP;
} }
bool isRetryableException(const Exception & e) bool isRetryableException(const std::exception_ptr exception_ptr)
{
try
{
rethrow_exception(exception_ptr);
}
#if USE_AWS_S3
catch (const S3Exception & s3_exception)
{
if (s3_exception.isRetryableError())
return true;
}
#endif
#if USE_AZURE_BLOB_STORAGE
catch (const Azure::Core::RequestFailedException &)
{
return true;
}
#endif
catch (const Exception & e)
{ {
if (isNotEnoughMemoryErrorCode(e.code())) if (isNotEnoughMemoryErrorCode(e.code()))
return true; return true;
if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT)
return true; return true;
}
#if USE_AWS_S3 catch (const Poco::Net::NetException &)
const auto * s3_exception = dynamic_cast<const S3Exception *>(&e); {
if (s3_exception && s3_exception->isRetryableError())
return true; return true;
#endif }
catch (const Poco::TimeoutException &)
{
return true;
}
/// In fact, there can be other similar situations. /// In fact, there can be other similar situations.
/// But it is OK, because there is a safety guard against deleting too many parts. /// But it is OK, because there is a safety guard against deleting too many parts.
@ -321,15 +348,10 @@ IMergeTreeDataPart::Checksums checkDataPart(
require_checksums, require_checksums,
is_cancelled); is_cancelled);
} }
catch (const Exception & e)
{
if (isRetryableException(e))
throw;
return drop_cache_and_check();
}
catch (...) catch (...)
{ {
if (isRetryableException(std::current_exception()))
throw;
return drop_cache_and_check(); return drop_cache_and_check();
} }
} }

View File

@ -13,6 +13,6 @@ IMergeTreeDataPart::Checksums checkDataPart(
std::function<bool()> is_cancelled = []{ return false; }); std::function<bool()> is_cancelled = []{ return false; });
bool isNotEnoughMemoryErrorCode(int code); bool isNotEnoughMemoryErrorCode(int code);
bool isRetryableException(const Exception & e); bool isRetryableException(const std::exception_ptr exception_ptr);
} }

View File

@ -520,7 +520,6 @@ namespace
std::unique_ptr<ReadBuffer> next() override std::unique_ptr<ReadBuffer> next() override
{ {
std::unique_ptr<ReadBuffer> read_buf; std::unique_ptr<ReadBuffer> read_buf;
struct stat file_stat;
while (true) while (true)
{ {
if (current_archive_index == archive_info.paths_to_archives.size()) if (current_archive_index == archive_info.paths_to_archives.size())
@ -535,6 +534,7 @@ namespace
} }
const auto & archive = archive_info.paths_to_archives[current_archive_index]; const auto & archive = archive_info.paths_to_archives[current_archive_index];
struct stat file_stat;
file_stat = getFileStat(archive, false, -1, "File"); file_stat = getFileStat(archive, false, -1, "File");
if (file_stat.st_size == 0) if (file_stat.st_size == 0)
{ {
@ -554,30 +554,6 @@ namespace
auto archive_reader = createArchiveReader(archive); auto archive_reader = createArchiveReader(archive);
auto try_get_columns_from_schema_cache = [&](const std::string & full_path) -> std::optional<ColumnsDescription>
{
auto context = getContext();
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file)
return std::nullopt;
auto & schema_cache = StorageFile::getSchemaCache(context);
auto get_last_mod_time = [&]() -> std::optional<time_t>
{
if (0 != stat(archive_reader->getPath().c_str(), &file_stat))
return std::nullopt;
return file_stat.st_mtime;
};
auto cache_key = getKeyForSchemaCache(full_path, format, format_settings, context);
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
return std::nullopt;
};
if (archive_info.isSingleFileRead()) if (archive_info.isSingleFileRead())
{ {
read_buf = archive_reader->readFile(archive_info.path_in_archive, false); read_buf = archive_reader->readFile(archive_info.path_in_archive, false);
@ -586,33 +562,26 @@ namespace
continue; continue;
last_read_file_path = processed_files.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), archive_info.path_in_archive)); last_read_file_path = processed_files.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), archive_info.path_in_archive));
columns_from_cache = try_get_columns_from_schema_cache(last_read_file_path); columns_from_cache = tryGetColumnsFromSchemaCache(archive, last_read_file_path);
if (columns_from_cache) if (columns_from_cache)
return nullptr; return nullptr;
} }
else else
{ {
auto file_enumerator = archive_reader->firstFile(); if (last_read_buffer)
file_enumerator = archive_reader->nextFile(std::move(last_read_buffer));
else
file_enumerator = archive_reader->firstFile();
if (!file_enumerator) if (!file_enumerator)
{ {
if (getContext()->getSettingsRef().engine_file_skip_empty_files)
{
read_files_from_archive.clear();
++current_archive_index; ++current_archive_index;
continue; continue;
} }
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because the archive {} has no files. "
"You must specify table structure manually",
format,
archive);
}
const auto * filename = &file_enumerator->getFileName(); const auto * filename = &file_enumerator->getFileName();
while (read_files_from_archive.contains(*filename) || !archive_info.filter(*filename)) while (!archive_info.filter(*filename))
{ {
if (!file_enumerator->nextFile()) if (!file_enumerator->nextFile())
{ {
@ -625,18 +594,16 @@ namespace
if (!archive_reader) if (!archive_reader)
{ {
read_files_from_archive.clear();
++current_archive_index; ++current_archive_index;
continue; continue;
} }
last_read_file_path = processed_files.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), *filename)); last_read_file_path = processed_files.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), *filename));
columns_from_cache = try_get_columns_from_schema_cache(last_read_file_path); columns_from_cache = tryGetColumnsFromSchemaCache(archive, last_read_file_path);
if (columns_from_cache) if (columns_from_cache)
return nullptr; return nullptr;
read_files_from_archive.insert(*filename);
read_buf = archive_reader->readFile(std::move(file_enumerator)); read_buf = archive_reader->readFile(std::move(file_enumerator));
} }
@ -647,6 +614,16 @@ namespace
return read_buf; return read_buf;
} }
std::optional<ColumnsDescription> getCachedColumns() override
{
return columns_from_cache;
}
void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> buffer) override
{
last_read_buffer = std::move(buffer);
}
void setNumRowsToLastFile(size_t num_rows) override void setNumRowsToLastFile(size_t num_rows) override
{ {
if (!getContext()->getSettingsRef().use_cache_for_count_from_files) if (!getContext()->getSettingsRef().use_cache_for_count_from_files)
@ -657,20 +634,79 @@ namespace
} }
std::vector<std::string> processed_files; std::vector<std::string> processed_files;
std::optional<ColumnsDescription> columns_from_cache;
private: private:
std::optional<ColumnsDescription> tryGetColumnsFromSchemaCache(const std::string & archive_path, const std::string & full_path)
{
auto context = getContext();
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file)
return std::nullopt;
struct stat file_stat;
auto & schema_cache = StorageFile::getSchemaCache(context);
auto get_last_mod_time = [&]() -> std::optional<time_t>
{
if (0 != stat(archive_path.c_str(), &file_stat))
return std::nullopt;
return file_stat.st_mtime;
};
auto cache_key = getKeyForSchemaCache(full_path, format, format_settings, context);
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
return std::nullopt;
}
const StorageFile::ArchiveInfo & archive_info; const StorageFile::ArchiveInfo & archive_info;
size_t current_archive_index = 0; size_t current_archive_index = 0;
std::unordered_set<std::string> read_files_from_archive;
bool is_first = true; bool is_first = true;
std::string last_read_file_path; std::string last_read_file_path;
std::optional<ColumnsDescription> columns_from_cache;
std::unique_ptr<IArchiveReader::FileEnumerator> file_enumerator;
std::unique_ptr<ReadBuffer> last_read_buffer;
String format; String format;
const std::optional<FormatSettings> & format_settings; const std::optional<FormatSettings> & format_settings;
}; };
std::optional<ColumnsDescription> tryGetColumnsFromCacheForArchives(
const StorageFile::ArchiveInfo & archive_info,
std::vector<std::string> & paths_for_schema_cache,
const String & format,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context)
{
struct stat file_stat{};
std::optional<ColumnsDescription> columns_from_cache;
for (const auto & archive : archive_info.paths_to_archives)
{
const auto & full_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive, archive_info.path_in_archive));
auto & schema_cache = StorageFile::getSchemaCache(context);
auto get_last_mod_time = [&]() -> std::optional<time_t>
{
if (0 != stat(archive.c_str(), &file_stat))
return std::nullopt;
return file_stat.st_mtime;
};
auto cache_key = getKeyForSchemaCache(full_path, format, format_settings, context);
columns_from_cache = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
}
return columns_from_cache;
}
} }
ColumnsDescription StorageFile::getTableStructureFromFileDescriptor(ContextPtr context) ColumnsDescription StorageFile::getTableStructureFromFileDescriptor(ContextPtr context)
@ -724,34 +760,15 @@ ColumnsDescription StorageFile::getTableStructureFromFile(
"You must specify table structure manually", format); "You must specify table structure manually", format);
ColumnsDescription columns; ColumnsDescription columns;
if (archive_info) std::vector<std::string> archive_paths_for_schema_cache;
{
std::vector<std::string> paths_for_schema_cache;
std::optional<ColumnsDescription> columns_from_cache; std::optional<ColumnsDescription> columns_from_cache;
if (context->getSettingsRef().schema_inference_use_cache_for_file) if (context->getSettingsRef().schema_inference_use_cache_for_file)
{ {
paths_for_schema_cache.reserve(archive_info->paths_to_archives.size()); if (archive_info)
struct stat file_stat{}; columns_from_cache = tryGetColumnsFromCacheForArchives(*archive_info, archive_paths_for_schema_cache, format, format_settings, context);
for (const auto & archive : archive_info->paths_to_archives) else
{ columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context);
const auto & full_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive, archive_info->path_in_archive));
if (!columns_from_cache)
{
auto & schema_cache = getSchemaCache(context);
auto get_last_mod_time = [&]() -> std::optional<time_t>
{
if (0 != stat(archive.c_str(), &file_stat))
return std::nullopt;
return file_stat.st_mtime;
};
auto cache_key = getKeyForSchemaCache(full_path, format, format_settings, context);
columns_from_cache = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
}
}
} }
if (columns_from_cache) if (columns_from_cache)
@ -760,58 +777,28 @@ ColumnsDescription StorageFile::getTableStructureFromFile(
} }
else else
{ {
ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); if (archive_info)
try
{ {
ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context);
columns = readSchemaFromFormat( columns = readSchemaFromFormat(
format, format,
format_settings, format_settings,
read_buffer_iterator, read_buffer_iterator,
/*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(), /*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(),
context); context);
}
catch (const DB::Exception & e)
{
/// maybe we found something in cache while iterating files
if (e.code() == ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE)
{
if (read_buffer_iterator.columns_from_cache)
columns = std::move(*read_buffer_iterator.columns_from_cache);
else
throw;
}
else
{
throw;
}
}
for (auto & file : read_buffer_iterator.processed_files) for (auto & file : read_buffer_iterator.processed_files)
paths_for_schema_cache.push_back(std::move(file)); archive_paths_for_schema_cache.push_back(std::move(file));
}
if (context->getSettingsRef().schema_inference_use_cache_for_file)
addColumnsToCache(paths_for_schema_cache, columns, format, format_settings, context);
}
else
{
std::optional<ColumnsDescription> columns_from_cache;
if (context->getSettingsRef().schema_inference_use_cache_for_file)
columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context);
if (columns_from_cache)
{
columns = *columns_from_cache;
} }
else else
{ {
ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context);
columns = readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context); columns = readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context);
} }
}
if (context->getSettingsRef().schema_inference_use_cache_for_file) if (context->getSettingsRef().schema_inference_use_cache_for_file)
addColumnsToCache(archive_info ? archive_info->paths_to_archives : paths, columns, format, format_settings, context); addColumnsToCache(archive_info.has_value() ? archive_paths_for_schema_cache : paths, columns, format, format_settings, context);
}
return columns; return columns;
} }

View File

@ -2511,7 +2511,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
renameTempPartAndReplace(part_desc->res_part, transaction); renameTempPartAndReplace(part_desc->res_part, transaction);
getCommitPartOps(ops, part_desc->res_part); getCommitPartOps(ops, part_desc->res_part);
lockSharedData(*part_desc->res_part, false, part_desc->hardlinked_files); lockSharedData(*part_desc->res_part, /* replace_existing_lock */ true, part_desc->hardlinked_files);
} }
@ -9681,6 +9681,15 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(
/// In rare case other replica can remove path between createAncestors and createIfNotExists /// In rare case other replica can remove path between createAncestors and createIfNotExists
/// So we make up to 5 attempts /// So we make up to 5 attempts
auto is_ephemeral = [&](const String & node_path) -> bool
{
String dummy_res;
Coordination::Stat node_stat;
if (zookeeper->tryGet(node_path, dummy_res, &node_stat))
return node_stat.ephemeralOwner;
return false;
};
bool created = false; bool created = false;
for (int attempts = 5; attempts > 0; --attempts) for (int attempts = 5; attempts > 0; --attempts)
{ {
@ -9700,6 +9709,9 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(
if (error == Coordination::Error::ZNODEEXISTS) if (error == Coordination::Error::ZNODEEXISTS)
{ {
if (is_ephemeral(zookeeper_node))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Node {} already exists, but it is ephemeral", zookeeper_node);
size_t failed_op = zkutil::getFailedOpIndex(error, responses); size_t failed_op = zkutil::getFailedOpIndex(error, responses);
/// Part was locked before, unfortunately it's possible during moves /// Part was locked before, unfortunately it's possible during moves
if (ops[failed_op]->getPath() == zookeeper_node) if (ops[failed_op]->getPath() == zookeeper_node)

View File

@ -490,7 +490,8 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
continue; continue;
auto & task = list_tasks[list_task_idx]; auto & task = list_tasks[list_task_idx];
context->getProcessListElement()->checkTimeLimit(); if (auto elem = context->getProcessListElement())
elem->checkTimeLimit();
Strings nodes = std::move(list_result.names); Strings nodes = std::move(list_result.names);
@ -525,7 +526,8 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
auto & get_task = get_tasks[i]; auto & get_task = get_tasks[i];
auto & list_task = list_tasks[get_task.list_task_idx]; auto & list_task = list_tasks[get_task.list_task_idx];
context->getProcessListElement()->checkTimeLimit(); if (auto elem = context->getProcessListElement())
elem->checkTimeLimit();
// Deduplication // Deduplication
String key = list_task.path_part + '/' + get_task.node; String key = list_task.path_part + '/' + get_task.node;

View File

@ -197,22 +197,8 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
return unmodified; return unmodified;
} }
void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast) static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & context)
{ {
if (block.rows() == 0)
return;
if (!expression_ast)
prepareFilterBlockWithQuery(query, context, block, expression_ast);
if (!expression_ast)
return;
/// Let's analyze and calculate the prepared expression.
auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList());
ExpressionAnalyzer analyzer(expression_ast, syntax_result, context);
ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes);
for (const auto & node : actions->getNodes()) for (const auto & node : actions->getNodes())
{ {
if (node.type == ActionsDAG::ActionType::COLUMN) if (node.type == ActionsDAG::ActionType::COLUMN)
@ -229,6 +215,10 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex
if (auto * set_from_subquery = typeid_cast<FutureSetFromSubquery *>(future_set.get())) if (auto * set_from_subquery = typeid_cast<FutureSetFromSubquery *>(future_set.get()))
{ {
auto plan = set_from_subquery->build(context); auto plan = set_from_subquery->build(context);
if (!plan)
continue;
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
pipeline.complete(std::make_shared<EmptySink>(Block())); pipeline.complete(std::make_shared<EmptySink>(Block()));
@ -240,6 +230,58 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex
} }
} }
} }
}
void filterBlockWithQuery(ActionsDAGPtr dag, Block & block, ContextPtr context)
{
auto actions = std::make_shared<ExpressionActions>(dag);
makeSets(actions, context);
Block block_with_filter = block;
actions->execute(block_with_filter);
/// Filter the block.
String filter_column_name = dag->getOutputs().at(0)->result_name;
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst();
ConstantFilterDescription constant_filter(*filter_column);
if (constant_filter.always_true)
{
return;
}
if (constant_filter.always_false)
{
block = block.cloneEmpty();
return;
}
FilterDescription filter(*filter_column);
for (size_t i = 0; i < block.columns(); ++i)
{
ColumnPtr & column = block.safeGetByPosition(i).column;
column = column->filter(*filter.data, -1);
}
}
void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast)
{
if (block.rows() == 0)
return;
if (!expression_ast)
prepareFilterBlockWithQuery(query, context, block, expression_ast);
if (!expression_ast)
return;
/// Let's analyze and calculate the prepared expression.
auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList());
ExpressionAnalyzer analyzer(expression_ast, syntax_result, context);
ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes);
makeSets(actions, context);
Block block_with_filter = block; Block block_with_filter = block;
actions->execute(block_with_filter); actions->execute(block_with_filter);

View File

@ -33,6 +33,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
/// Only elements of the outer conjunction are considered, depending only on the columns present in the block. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block.
/// If `expression_ast` is passed, use it to filter block. /// If `expression_ast` is passed, use it to filter block.
void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast = {}); void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast = {});
void filterBlockWithQuery(ActionsDAGPtr dag, Block & block, ContextPtr context);
/// Extract from the input stream a set of `name` column values /// Extract from the input stream a set of `name` column values
template <typename T> template <typename T>

View File

@ -160,7 +160,7 @@ def main():
s3_helper = S3Helper() s3_helper = S3Helper()
for f in paths: for f in paths:
try: try:
paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f) paths[f] = s3_helper.upload_test_report_to_s3(Path(paths[f]), s3_prefix + f)
except Exception as ex: except Exception as ex:
logging.info("Exception uploading file %s text %s", f, ex) logging.info("Exception uploading file %s text %s", f, ex)
paths[f] = "" paths[f] = ""

View File

@ -1,10 +1,9 @@
#!/usr/bin/env python3 #!/usr/bin/env python3
from pathlib import Path from pathlib import Path
from typing import List, Tuple from typing import Tuple
import subprocess import subprocess
import logging import logging
import json
import os import os
import sys import sys
import time import time
@ -22,6 +21,7 @@ from env_helper import (
) )
from git_helper import Git, git_runner from git_helper import Git, git_runner
from pr_info import PRInfo from pr_info import PRInfo
from report import BuildResult, FAILURE, StatusType, SUCCESS
from s3_helper import S3Helper from s3_helper import S3Helper
from tee_popen import TeePopen from tee_popen import TeePopen
from version_helper import ( from version_helper import (
@ -98,7 +98,7 @@ def get_packager_cmd(
def build_clickhouse( def build_clickhouse(
packager_cmd: str, logs_path: Path, build_output_path: Path packager_cmd: str, logs_path: Path, build_output_path: Path
) -> Tuple[Path, bool]: ) -> Tuple[Path, StatusType]:
build_log_path = logs_path / BUILD_LOG_NAME build_log_path = logs_path / BUILD_LOG_NAME
success = False success = False
with TeePopen(packager_cmd, build_log_path) as process: with TeePopen(packager_cmd, build_log_path) as process:
@ -118,15 +118,16 @@ def build_clickhouse(
) )
else: else:
logging.info("Build failed") logging.info("Build failed")
return build_log_path, success return build_log_path, SUCCESS if success else FAILURE
def check_for_success_run( def check_for_success_run(
s3_helper: S3Helper, s3_helper: S3Helper,
s3_prefix: str, s3_prefix: str,
build_name: str, build_name: str,
build_config: BuildConfig, version: ClickHouseVersion,
) -> None: ) -> None:
# TODO: Remove after S3 artifacts
# the final empty argument is necessary for distinguish build and build_suffix # the final empty argument is necessary for distinguish build and build_suffix
logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix, "") logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix, "")
logging.info("Checking for artifacts in %s", logged_prefix) logging.info("Checking for artifacts in %s", logged_prefix)
@ -155,15 +156,16 @@ def check_for_success_run(
return return
success = len(build_urls) > 0 success = len(build_urls) > 0
create_json_artifact( build_result = BuildResult(
TEMP_PATH,
build_name, build_name,
log_url, log_url,
build_urls, build_urls,
build_config, version.describe,
SUCCESS if success else FAILURE,
0, 0,
success, GITHUB_JOB,
) )
build_result.write_json(Path(TEMP_PATH))
# Fail build job if not successeded # Fail build job if not successeded
if not success: if not success:
sys.exit(1) sys.exit(1)
@ -171,36 +173,6 @@ def check_for_success_run(
sys.exit(0) sys.exit(0)
def create_json_artifact(
temp_path: str,
build_name: str,
log_url: str,
build_urls: List[str],
build_config: BuildConfig,
elapsed: int,
success: bool,
) -> None:
subprocess.check_call(
f"echo 'BUILD_URLS=build_urls_{build_name}' >> $GITHUB_ENV", shell=True
)
result = {
"log_url": log_url,
"build_urls": build_urls,
"build_config": build_config.__dict__,
"elapsed_seconds": elapsed,
"status": success,
"job_name": GITHUB_JOB,
}
json_name = "build_urls_" + build_name + ".json"
print(f"Dump json report {result} to {json_name} with env build_urls_{build_name}")
with open(os.path.join(temp_path, json_name), "w", encoding="utf-8") as build_links:
json.dump(result, build_links)
def get_release_or_pr(pr_info: PRInfo, version: ClickHouseVersion) -> Tuple[str, str]: def get_release_or_pr(pr_info: PRInfo, version: ClickHouseVersion) -> Tuple[str, str]:
"Return prefixes for S3 artifacts paths" "Return prefixes for S3 artifacts paths"
# FIXME performance # FIXME performance
@ -269,7 +241,7 @@ def main():
# If this is rerun, then we try to find already created artifacts and just # If this is rerun, then we try to find already created artifacts and just
# put them as github actions artifact (result) # put them as github actions artifact (result)
check_for_success_run(s3_helper, s3_path_prefix, build_name, build_config) check_for_success_run(s3_helper, s3_path_prefix, build_name, version)
docker_image = get_image_with_version(IMAGES_PATH, IMAGE_NAME) docker_image = get_image_with_version(IMAGES_PATH, IMAGE_NAME)
image_version = docker_image.version image_version = docker_image.version
@ -312,16 +284,17 @@ def main():
os.makedirs(logs_path, exist_ok=True) os.makedirs(logs_path, exist_ok=True)
start = time.time() start = time.time()
log_path, success = build_clickhouse(packager_cmd, logs_path, build_output_path) log_path, build_status = build_clickhouse(
packager_cmd, logs_path, build_output_path
)
elapsed = int(time.time() - start) elapsed = int(time.time() - start)
subprocess.check_call( subprocess.check_call(
f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True
) )
logging.info("Build finished with %s, log path %s", success, log_path) logging.info("Build finished as %s, log path %s", build_status, log_path)
if success: if build_status == SUCCESS:
cargo_cache.upload() cargo_cache.upload()
else:
if not success:
# We check if docker works, because if it's down, it's infrastructure # We check if docker works, because if it's down, it's infrastructure
try: try:
subprocess.check_call("docker info", shell=True) subprocess.check_call("docker info", shell=True)
@ -345,7 +318,7 @@ def main():
os.remove(performance_path) os.remove(performance_path)
build_urls = ( build_urls = (
s3_helper.upload_build_folder_to_s3( s3_helper.upload_build_directory_to_s3(
build_output_path, build_output_path,
s3_path_prefix, s3_path_prefix,
keep_dirs_in_s3_path=False, keep_dirs_in_s3_path=False,
@ -367,8 +340,20 @@ def main():
print(f"::notice ::Log URL: {log_url}") print(f"::notice ::Log URL: {log_url}")
create_json_artifact( build_result = BuildResult(
TEMP_PATH, build_name, log_url, build_urls, build_config, elapsed, success build_name,
log_url,
build_urls,
version.describe,
build_status,
elapsed,
GITHUB_JOB,
)
result_json_path = build_result.write_json(temp_path)
logging.info(
"Build result file %s is written, content:\n %s",
result_json_path,
result_json_path.read_text(encoding="utf-8"),
) )
upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path)
@ -449,7 +434,7 @@ FORMAT JSONCompactEachRow"""
prepared_events = prepare_tests_results_for_clickhouse( prepared_events = prepare_tests_results_for_clickhouse(
pr_info, pr_info,
[], [],
"success" if success else "failure", build_status,
stopwatch.duration_seconds, stopwatch.duration_seconds,
stopwatch.start_time_str, stopwatch.start_time_str,
log_url, log_url,
@ -458,7 +443,7 @@ FORMAT JSONCompactEachRow"""
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
# Fail the build job if it didn't succeed # Fail the build job if it didn't succeed
if not success: if build_status != SUCCESS:
sys.exit(1) sys.exit(1)

View File

@ -6,7 +6,7 @@ import os
import sys import sys
import time import time
from pathlib import Path from pathlib import Path
from typing import Any, Callable, List from typing import Any, Callable, List, Union
import requests # type: ignore import requests # type: ignore
@ -98,7 +98,7 @@ def get_build_name_for_check(check_name: str) -> str:
return CI_CONFIG.test_configs[check_name].required_build return CI_CONFIG.test_configs[check_name].required_build
def read_build_urls(build_name: str, reports_path: str) -> List[str]: def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]:
for root, _, files in os.walk(reports_path): for root, _, files in os.walk(reports_path):
for f in files: for f in files:
if build_name in f: if build_name in f:

View File

@ -5,19 +5,25 @@ import logging
import os import os
import sys import sys
import atexit import atexit
from typing import Dict, List, Tuple from pathlib import Path
from github import Github from github import Github
from env_helper import ( from env_helper import (
GITHUB_JOB_URL, GITHUB_JOB_URL,
GITHUB_REPOSITORY, GITHUB_REPOSITORY,
GITHUB_RUN_URL,
GITHUB_SERVER_URL, GITHUB_SERVER_URL,
REPORTS_PATH, REPORTS_PATH,
TEMP_PATH, TEMP_PATH,
) )
from report import create_build_html_report, BuildResult, BuildResults from report import (
BuildResult,
ERROR,
PENDING,
SUCCESS,
create_build_html_report,
get_worst_status,
)
from s3_helper import S3Helper from s3_helper import S3Helper
from get_robot_token import get_best_robot_token from get_robot_token import get_best_robot_token
from pr_info import NeedsDataType, PRInfo from pr_info import NeedsDataType, PRInfo
@ -34,95 +40,17 @@ from ci_config import CI_CONFIG
NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "") NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "")
def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]:
groups = {
"apk": [],
"deb": [],
"binary": [],
"tgz": [],
"rpm": [],
"performance": [],
} # type: Dict[str, List[str]]
for url in build_urls:
if url.endswith("performance.tar.zst"):
groups["performance"].append(url)
elif (
url.endswith(".deb")
or url.endswith(".buildinfo")
or url.endswith(".changes")
or url.endswith(".tar.gz")
):
groups["deb"].append(url)
elif url.endswith(".apk"):
groups["apk"].append(url)
elif url.endswith(".rpm"):
groups["rpm"].append(url)
elif url.endswith(".tgz") or url.endswith(".tgz.sha512"):
groups["tgz"].append(url)
else:
groups["binary"].append(url)
return groups
def get_failed_report(
job_name: str,
) -> Tuple[BuildResults, List[List[str]], List[str]]:
message = f"{job_name} failed"
build_result = BuildResult(
compiler="unknown",
debug_build=False,
sanitizer="unknown",
status=message,
elapsed_seconds=0,
comment="",
)
return [build_result], [[""]], [GITHUB_RUN_URL]
def process_report(
build_report: dict,
) -> Tuple[BuildResults, List[List[str]], List[str]]:
build_config = build_report["build_config"]
build_result = BuildResult(
compiler=build_config["compiler"],
debug_build=build_config["debug_build"],
sanitizer=build_config["sanitizer"],
status="success" if build_report["status"] else "failure",
elapsed_seconds=build_report["elapsed_seconds"],
comment=build_config["comment"],
)
build_results = []
build_urls = []
build_logs_urls = []
urls_groups = group_by_artifacts(build_report["build_urls"])
found_group = False
for _, group_urls in urls_groups.items():
if group_urls:
build_results.append(build_result)
build_urls.append(group_urls)
build_logs_urls.append(build_report["log_url"])
found_group = True
# No one group of urls is found, a failed report
if not found_group:
build_results.append(build_result)
build_urls.append([""])
build_logs_urls.append(build_report["log_url"])
return build_results, build_urls, build_logs_urls
def get_build_name_from_file_name(file_name):
return file_name.replace("build_urls_", "").replace(".json", "")
def main(): def main():
logging.basicConfig(level=logging.INFO) logging.basicConfig(level=logging.INFO)
temp_path = TEMP_PATH temp_path = Path(TEMP_PATH)
logging.info("Reports path %s", REPORTS_PATH) temp_path.mkdir(parents=True, exist_ok=True)
if not os.path.exists(temp_path): logging.info("Reports path %s", REPORTS_PATH)
os.makedirs(temp_path) reports_path = Path(REPORTS_PATH)
logging.info(
"Reports found:\n %s",
"\n ".join(p.as_posix() for p in reports_path.rglob("*.json")),
)
build_check_name = sys.argv[1] build_check_name = sys.argv[1]
needs_data = {} # type: NeedsDataType needs_data = {} # type: NeedsDataType
@ -132,12 +60,12 @@ def main():
needs_data = json.load(file_handler) needs_data = json.load(file_handler)
required_builds = len(needs_data) required_builds = len(needs_data)
if needs_data and all(i["result"] == "skipped" for i in needs_data.values()): if needs_data:
logging.info("The next builds are required: %s", ", ".join(needs_data))
if all(i["result"] == "skipped" for i in needs_data.values()):
logging.info("All builds are skipped, exiting") logging.info("All builds are skipped, exiting")
sys.exit(0) sys.exit(0)
logging.info("The next builds are required: %s", ", ".join(needs_data))
gh = Github(get_best_robot_token(), per_page=100) gh = Github(get_best_robot_token(), per_page=100)
pr_info = PRInfo() pr_info = PRInfo()
commit = get_commit(gh, pr_info.sha) commit = get_commit(gh, pr_info.sha)
@ -153,73 +81,41 @@ def main():
required_builds = required_builds or len(builds_for_check) required_builds = required_builds or len(builds_for_check)
# Collect reports from json artifacts # Collect reports from json artifacts
builds_report_map = {} build_results = []
for root, _, files in os.walk(REPORTS_PATH): for build_name in builds_for_check:
for f in files: report_name = BuildResult.get_report_name(build_name).stem
if f.startswith("build_urls_") and f.endswith(".json"): build_result = BuildResult.read_json(reports_path / report_name, build_name)
logging.info("Found build report json %s", f) if build_result.is_missing:
build_name = get_build_name_from_file_name(f) logging.warning("Build results for %s are missing", build_name)
if build_name in builds_for_check: continue
with open(os.path.join(root, f), "rb") as file_handler: build_results.append(build_result)
builds_report_map[build_name] = json.load(file_handler)
else:
logging.info(
"Skipping report %s for build %s, it's not in our reports list",
f,
build_name,
)
# Sort reports by config order # The code to collect missing reports for failed jobs
build_reports = [ missing_job_names = [
builds_report_map[build_name]
for build_name in builds_for_check
if build_name in builds_report_map
]
some_builds_are_missing = len(build_reports) < required_builds
missing_build_names = []
if some_builds_are_missing:
logging.warning(
"Expected to get %s build results, got only %s",
required_builds,
len(build_reports),
)
missing_build_names = [
name name
for name in needs_data for name in needs_data
if not any(rep for rep in build_reports if rep["job_name"] == name) if not any(1 for build_result in build_results if build_result.job_name == name)
] ]
else: missing_builds = len(missing_job_names)
logging.info("Got exactly %s builds", len(builds_report_map)) for job_name in reversed(missing_job_names):
build_result = BuildResult.missing_result("missing")
# Group build artifacts by groups build_result.job_name = job_name
build_results = [] # type: BuildResults build_result.status = PENDING
build_artifacts = [] # type: List[List[str]]
build_logs = [] # type: List[str]
for build_report in build_reports:
_build_results, build_artifacts_url, build_logs_url = process_report(
build_report
)
logging.info( logging.info(
"Got %s artifact groups for build report report", len(_build_results) "There is missing report for %s, created a dummy result %s",
job_name,
build_result,
) )
build_results.extend(_build_results) build_results.insert(0, build_result)
build_artifacts.extend(build_artifacts_url)
build_logs.extend(build_logs_url)
for failed_job in missing_build_names: # Calculate artifact groups like packages and binaries
_build_results, build_artifacts_url, build_logs_url = get_failed_report( total_groups = sum(len(br.grouped_urls) for br in build_results)
failed_job ok_groups = sum(
len(br.grouped_urls) for br in build_results if br.status == SUCCESS
) )
build_results.extend(_build_results)
build_artifacts.extend(build_artifacts_url)
build_logs.extend(build_logs_url)
total_groups = len(build_results)
logging.info("Totally got %s artifact groups", total_groups) logging.info("Totally got %s artifact groups", total_groups)
if total_groups == 0: if total_groups == 0:
logging.error("No success builds, failing check") logging.error("No success builds, failing check without creating a status")
sys.exit(1) sys.exit(1)
s3_helper = S3Helper() s3_helper = S3Helper()
@ -234,17 +130,14 @@ def main():
report = create_build_html_report( report = create_build_html_report(
build_check_name, build_check_name,
build_results, build_results,
build_logs,
build_artifacts,
task_url, task_url,
branch_url, branch_url,
branch_name, branch_name,
commit_url, commit_url,
) )
report_path = os.path.join(temp_path, "report.html") report_path = temp_path / "report.html"
with open(report_path, "w", encoding="utf-8") as fd: report_path.write_text(report, encoding="utf-8")
fd.write(report)
logging.info("Going to upload prepared report") logging.info("Going to upload prepared report")
context_name_for_path = build_check_name.lower().replace(" ", "_") context_name_for_path = build_check_name.lower().replace(" ", "_")
@ -259,27 +152,20 @@ def main():
print(f"::notice ::Report url: {url}") print(f"::notice ::Report url: {url}")
# Prepare a commit status # Prepare a commit status
ok_groups = 0 summary_status = get_worst_status(br.status for br in build_results)
summary_status = "success"
for build_result in build_results:
if build_result.status == "failure" and summary_status != "error":
summary_status = "failure"
if build_result.status == "error" or not build_result.status:
summary_status = "error"
if build_result.status == "success":
ok_groups += 1
# Check if there are no builds at all, do not override bad status # Check if there are no builds at all, do not override bad status
if summary_status == "success": if summary_status == SUCCESS:
if some_builds_are_missing: if missing_builds:
summary_status = "pending" summary_status = PENDING
elif ok_groups == 0: elif ok_groups == 0:
summary_status = "error" summary_status = ERROR
addition = "" addition = ""
if some_builds_are_missing: if missing_builds:
addition = f" ({len(build_reports)} of {required_builds} builds are OK)" addition = (
f" ({required_builds - missing_builds} of {required_builds} builds are OK)"
)
description = format_description( description = format_description(
f"{ok_groups}/{total_groups} artifact groups are OK{addition}" f"{ok_groups}/{total_groups} artifact groups are OK{addition}"
@ -289,7 +175,7 @@ def main():
commit, summary_status, url, description, build_check_name, pr_info commit, summary_status, url, description, build_check_name, pr_info
) )
if summary_status == "error": if summary_status == ERROR:
sys.exit(1) sys.exit(1)

View File

@ -3,7 +3,7 @@
import csv import csv
import os import os
import time import time
from typing import Dict, List, Literal, Optional, Union from typing import Dict, List, Optional, Union
import logging import logging
from github import Github from github import Github
@ -16,7 +16,16 @@ from github.Repository import Repository
from ci_config import CI_CONFIG, REQUIRED_CHECKS, CHECK_DESCRIPTIONS, CheckDescription from ci_config import CI_CONFIG, REQUIRED_CHECKS, CHECK_DESCRIPTIONS, CheckDescription
from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL
from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL
from report import TestResult, TestResults from report import (
ERROR,
FAILURE,
PENDING,
StatusType,
SUCCESS,
TestResult,
TestResults,
get_worst_status,
)
from s3_helper import S3Helper from s3_helper import S3Helper
from upload_result_helper import upload_results from upload_result_helper import upload_results
@ -37,8 +46,8 @@ class RerunHelper:
# currently we agree even for failed statuses # currently we agree even for failed statuses
for status in self.statuses: for status in self.statuses:
if self.check_name in status.context and status.state in ( if self.check_name in status.context and status.state in (
"success", SUCCESS,
"failure", FAILURE,
): ):
return True return True
return False return False
@ -53,12 +62,12 @@ class RerunHelper:
def override_status(status: str, check_name: str, invert: bool = False) -> str: def override_status(status: str, check_name: str, invert: bool = False) -> str:
test_config = CI_CONFIG.test_configs.get(check_name) test_config = CI_CONFIG.test_configs.get(check_name)
if test_config and test_config.force_tests: if test_config and test_config.force_tests:
return "success" return SUCCESS
if invert: if invert:
if status == "success": if status == SUCCESS:
return "error" return ERROR
return "success" return SUCCESS
return status return status
@ -137,7 +146,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None:
# W/o pr_info to avoid recursion, and yes, one extra create_ci_report # W/o pr_info to avoid recursion, and yes, one extra create_ci_report
post_commit_status( post_commit_status(
commit, commit,
"pending", PENDING,
create_ci_report(pr_info, statuses), create_ci_report(pr_info, statuses),
"The report for running CI", "The report for running CI",
CI_STATUS_NAME, CI_STATUS_NAME,
@ -172,11 +181,11 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str:
"""The method generates the comment body, as well it updates the CI report""" """The method generates the comment body, as well it updates the CI report"""
def beauty_state(state: str) -> str: def beauty_state(state: str) -> str:
if state == "success": if state == SUCCESS:
return f"🟢 {state}" return f"🟢 {state}"
if state == "pending": if state == PENDING:
return f"🟡 {state}" return f"🟡 {state}"
if state in ["error", "failure"]: if state in [ERROR, FAILURE]:
return f"🔴 {state}" return f"🔴 {state}"
return state return state
@ -235,20 +244,7 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str:
def get_worst_state(statuses: CommitStatuses) -> str: def get_worst_state(statuses: CommitStatuses) -> str:
worst_status = None return get_worst_status(status.state for status in statuses)
states = {"error": 0, "failure": 1, "pending": 2, "success": 3}
for status in statuses:
if worst_status is None:
worst_status = status
continue
if states[status.state] < states[worst_status.state]:
worst_status = status
if worst_status.state == "error":
break
if worst_status is None:
return ""
return worst_status.state
def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str:
@ -324,7 +320,7 @@ def format_description(description: str) -> str:
def set_mergeable_check( def set_mergeable_check(
commit: Commit, commit: Commit,
description: str = "", description: str = "",
state: Literal["success", "failure"] = "success", state: StatusType = "success",
) -> None: ) -> None:
commit.create_status( commit.create_status(
context=MERGEABLE_NAME, context=MERGEABLE_NAME,
@ -363,7 +359,7 @@ def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None
success = [] success = []
fail = [] fail = []
for status in required_checks: for status in required_checks:
if status.state == "success": if status.state == SUCCESS:
success.append(status.context) success.append(status.context)
else: else:
fail.append(status.context) fail.append(status.context)
@ -372,7 +368,7 @@ def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None
description = "failed: " + ", ".join(fail) description = "failed: " + ", ".join(fail)
description = format_description(description) description = format_description(description)
if mergeable_status is None or mergeable_status.description != description: if mergeable_status is None or mergeable_status.description != description:
set_mergeable_check(commit, description, "failure") set_mergeable_check(commit, description, FAILURE)
return return
description = ", ".join(success) description = ", ".join(success)

View File

@ -6,7 +6,8 @@ import time
import subprocess import subprocess
import logging import logging
from typing import List, Optional from pathlib import Path
from typing import List, Optional, Union
class DockerImage: class DockerImage:
@ -22,7 +23,7 @@ class DockerImage:
def get_images_with_versions( def get_images_with_versions(
reports_path: str, reports_path: Union[Path, str],
required_images: List[str], required_images: List[str],
pull: bool = True, pull: bool = True,
version: Optional[str] = None, version: Optional[str] = None,
@ -80,7 +81,10 @@ def get_images_with_versions(
def get_image_with_version( def get_image_with_version(
reports_path: str, image: str, pull: bool = True, version: Optional[str] = None reports_path: Union[Path, str],
image: str,
pull: bool = True,
version: Optional[str] = None,
) -> DockerImage: ) -> DockerImage:
logging.info("Looking for images file in %s", reports_path) logging.info("Looking for images file in %s", reports_path)
return get_images_with_versions(reports_path, [image], pull, version=version)[0] return get_images_with_versions(reports_path, [image], pull, version=version)[0]

View File

@ -1,6 +1,9 @@
#!/usr/bin/env python
import logging import logging
import os import os
from os import path as p from os import path as p
from typing import Tuple
from build_download_helper import get_gh_api from build_download_helper import get_gh_api
@ -40,13 +43,27 @@ _GITHUB_JOB_URL = ""
def GITHUB_JOB_ID() -> str: def GITHUB_JOB_ID() -> str:
global _GITHUB_JOB_ID global _GITHUB_JOB_ID
global _GITHUB_JOB_URL global _GITHUB_JOB_URL
if GITHUB_RUN_ID == "0":
_GITHUB_JOB_ID = "0"
if _GITHUB_JOB_ID: if _GITHUB_JOB_ID:
return _GITHUB_JOB_ID return _GITHUB_JOB_ID
_GITHUB_JOB_ID, _GITHUB_JOB_URL = get_job_id_url(GITHUB_JOB)
return _GITHUB_JOB_ID
def GITHUB_JOB_URL() -> str:
GITHUB_JOB_ID()
return _GITHUB_JOB_URL
def get_job_id_url(job_name: str) -> Tuple[str, str]:
job_id = ""
job_url = ""
if GITHUB_RUN_ID == "0":
job_id = "0"
if job_id:
return job_id, job_url
jobs = [] jobs = []
page = 1 page = 1
while not _GITHUB_JOB_ID: while not job_id:
response = get_gh_api( response = get_gh_api(
f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" f"https://api.github.com/repos/{GITHUB_REPOSITORY}/"
f"actions/runs/{GITHUB_RUN_ID}/jobs?per_page=100&page={page}" f"actions/runs/{GITHUB_RUN_ID}/jobs?per_page=100&page={page}"
@ -55,46 +72,41 @@ def GITHUB_JOB_ID() -> str:
data = response.json() data = response.json()
jobs.extend(data["jobs"]) jobs.extend(data["jobs"])
for job in data["jobs"]: for job in data["jobs"]:
if job["name"] != GITHUB_JOB: if job["name"] != job_name:
continue continue
_GITHUB_JOB_ID = job["id"] job_id = job["id"]
_GITHUB_JOB_URL = job["html_url"] job_url = job["html_url"]
return _GITHUB_JOB_ID return job_id, job_url
if ( if (
len(jobs) >= data["total_count"] # just in case of inconsistency len(jobs) >= data["total_count"] # just in case of inconsistency
or len(data["jobs"]) == 0 # if we excided pages or len(data["jobs"]) == 0 # if we excided pages
): ):
_GITHUB_JOB_ID = "0" job_id = "0"
# FIXME: until it's here, we can't move to reusable workflows # FIXME: until it's here, we can't move to reusable workflows
if not _GITHUB_JOB_URL: if not job_url:
# This is a terrible workaround for the case of another broken part of # This is a terrible workaround for the case of another broken part of
# GitHub actions. For nested workflows it doesn't provide a proper GITHUB_JOB # GitHub actions. For nested workflows it doesn't provide a proper job_name
# value, but only the final one. So, for `OriginalJob / NestedJob / FinalJob` # value, but only the final one. So, for `OriginalJob / NestedJob / FinalJob`
# full name, GITHUB_JOB contains only FinalJob # full name, job_name contains only FinalJob
matched_jobs = [] matched_jobs = []
for job in jobs: for job in jobs:
nested_parts = job["name"].split(" / ") nested_parts = job["name"].split(" / ")
if len(nested_parts) <= 1: if len(nested_parts) <= 1:
continue continue
if nested_parts[-1] == GITHUB_JOB: if nested_parts[-1] == job_name:
matched_jobs.append(job) matched_jobs.append(job)
if len(matched_jobs) == 1: if len(matched_jobs) == 1:
# The best case scenario # The best case scenario
_GITHUB_JOB_ID = matched_jobs[0]["id"] job_id = matched_jobs[0]["id"]
_GITHUB_JOB_URL = matched_jobs[0]["html_url"] job_url = matched_jobs[0]["html_url"]
return _GITHUB_JOB_ID return job_id, job_url
if matched_jobs: if matched_jobs:
logging.error( logging.error(
"We could not get the ID and URL for the current job name %s, there " "We could not get the ID and URL for the current job name %s, there "
"are more than one jobs match it for the nested workflows. Please, " "are more than one jobs match it for the nested workflows. Please, "
"refer to https://github.com/actions/runner/issues/2577", "refer to https://github.com/actions/runner/issues/2577",
GITHUB_JOB, job_name,
) )
return _GITHUB_JOB_ID return job_id, job_url
def GITHUB_JOB_URL() -> str:
GITHUB_JOB_ID()
return _GITHUB_JOB_URL

View File

@ -54,23 +54,21 @@ def get_fasttest_cmd(workspace, output_path, repo_path, pr_number, commit_sha, i
) )
def process_results(result_folder: str) -> Tuple[str, str, TestResults, List[str]]: def process_results(result_folder: Path) -> Tuple[str, str, TestResults, List[str]]:
test_results = [] # type: TestResults test_results = [] # type: TestResults
additional_files = [] additional_files = []
# Just upload all files from result_folder. # Just upload all files from result_folder.
# If task provides processed results, then it's responsible for content of # If task provides processed results, then it's responsible for content of
# result_folder # result_folder
if os.path.exists(result_folder): if result_folder.exists():
test_files = [ test_files = [
f f for f in result_folder.iterdir() if f.is_file()
for f in os.listdir(result_folder) ] # type: List[Path]
if os.path.isfile(os.path.join(result_folder, f)) additional_files = [f.absolute().as_posix() for f in test_files]
]
additional_files = [os.path.join(result_folder, f) for f in test_files]
status = [] status = []
status_path = os.path.join(result_folder, "check_status.tsv") status_path = result_folder / "check_status.tsv"
if os.path.exists(status_path): if status_path.exists():
logging.info("Found test_results.tsv") logging.info("Found test_results.tsv")
with open(status_path, "r", encoding="utf-8") as status_file: with open(status_path, "r", encoding="utf-8") as status_file:
status = list(csv.reader(status_file, delimiter="\t")) status = list(csv.reader(status_file, delimiter="\t"))
@ -80,7 +78,7 @@ def process_results(result_folder: str) -> Tuple[str, str, TestResults, List[str
state, description = status[0][0], status[0][1] state, description = status[0][0], status[0][1]
try: try:
results_path = Path(result_folder) / "test_results.tsv" results_path = result_folder / "test_results.tsv"
test_results = read_test_results(results_path) test_results = read_test_results(results_path)
if len(test_results) == 0: if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results, additional_files return "error", "Empty test_results.tsv", test_results, additional_files
@ -100,10 +98,9 @@ def main():
stopwatch = Stopwatch() stopwatch = Stopwatch()
temp_path = TEMP_PATH temp_path = Path(TEMP_PATH)
if not os.path.exists(temp_path): temp_path.mkdir(parents=True, exist_ok=True)
os.makedirs(temp_path)
pr_info = PRInfo() pr_info = PRInfo()
@ -124,17 +121,14 @@ def main():
s3_helper = S3Helper() s3_helper = S3Helper()
workspace = os.path.join(temp_path, "fasttest-workspace") workspace = temp_path / "fasttest-workspace"
if not os.path.exists(workspace): workspace.mkdir(parents=True, exist_ok=True)
os.makedirs(workspace)
output_path = os.path.join(temp_path, "fasttest-output") output_path = temp_path / "fasttest-output"
if not os.path.exists(output_path): output_path.mkdir(parents=True, exist_ok=True)
os.makedirs(output_path)
repo_path = os.path.join(temp_path, "fasttest-repo") repo_path = temp_path / "fasttest-repo"
if not os.path.exists(repo_path): repo_path.mkdir(parents=True, exist_ok=True)
os.makedirs(repo_path)
run_cmd = get_fasttest_cmd( run_cmd = get_fasttest_cmd(
workspace, workspace,
@ -146,11 +140,10 @@ def main():
) )
logging.info("Going to run fasttest with cmd %s", run_cmd) logging.info("Going to run fasttest with cmd %s", run_cmd)
logs_path = os.path.join(temp_path, "fasttest-logs") logs_path = temp_path / "fasttest-logs"
if not os.path.exists(logs_path): logs_path.mkdir(parents=True, exist_ok=True)
os.makedirs(logs_path)
run_log_path = os.path.join(logs_path, "run.log") run_log_path = logs_path / "run.log"
with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process: with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process:
retcode = process.wait() retcode = process.wait()
if retcode == 0: if retcode == 0:
@ -161,9 +154,7 @@ def main():
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
test_output_files = os.listdir(output_path) test_output_files = os.listdir(output_path)
additional_logs = [] additional_logs = [os.path.join(output_path, f) for f in test_output_files]
for f in test_output_files:
additional_logs.append(os.path.join(output_path, f))
test_log_exists = ( test_log_exists = (
"test_log.txt" in test_output_files or "test_result.txt" in test_output_files "test_log.txt" in test_output_files or "test_result.txt" in test_output_files
@ -194,8 +185,8 @@ def main():
pr_info.sha, pr_info.sha,
"fast_tests", "fast_tests",
) )
build_urls = s3_helper.upload_build_folder_to_s3( build_urls = s3_helper.upload_build_directory_to_s3(
os.path.join(output_path, "binaries"), output_path / "binaries",
s3_path_prefix, s3_path_prefix,
keep_dirs_in_s3_path=False, keep_dirs_in_s3_path=False,
upload_symlinks=False, upload_symlinks=False,
@ -206,7 +197,7 @@ def main():
pr_info.number, pr_info.number,
pr_info.sha, pr_info.sha,
test_results, test_results,
[run_log_path] + additional_logs, [run_log_path.as_posix()] + additional_logs,
NAME, NAME,
build_urls, build_urls,
) )

View File

@ -29,7 +29,7 @@ from docker_pull_helper import get_image_with_version, DockerImage
from env_helper import CI, TEMP_PATH as TEMP, REPORTS_PATH from env_helper import CI, TEMP_PATH as TEMP, REPORTS_PATH
from get_robot_token import get_best_robot_token from get_robot_token import get_best_robot_token
from pr_info import PRInfo from pr_info import PRInfo
from report import TestResults, TestResult from report import TestResults, TestResult, FAILURE, FAIL, OK, SUCCESS
from s3_helper import S3Helper from s3_helper import S3Helper
from stopwatch import Stopwatch from stopwatch import Stopwatch
from tee_popen import TeePopen from tee_popen import TeePopen
@ -40,10 +40,6 @@ RPM_IMAGE = "clickhouse/install-rpm-test"
DEB_IMAGE = "clickhouse/install-deb-test" DEB_IMAGE = "clickhouse/install-deb-test"
TEMP_PATH = Path(TEMP) TEMP_PATH = Path(TEMP)
LOGS_PATH = TEMP_PATH / "tests_logs" LOGS_PATH = TEMP_PATH / "tests_logs"
SUCCESS = "success"
FAILURE = "failure"
OK = "OK"
FAIL = "FAIL"
def prepare_test_scripts(): def prepare_test_scripts():

View File

@ -7,6 +7,7 @@ import json
import subprocess import subprocess
import traceback import traceback
import re import re
from pathlib import Path
from typing import Dict from typing import Dict
from github import Github from github import Github
@ -218,15 +219,17 @@ if __name__ == "__main__":
uploaded = {} # type: Dict[str, str] uploaded = {} # type: Dict[str, str]
for name, path in paths.items(): for name, path in paths.items():
try: try:
uploaded[name] = s3_helper.upload_test_report_to_s3(path, s3_prefix + name) uploaded[name] = s3_helper.upload_test_report_to_s3(
Path(path), s3_prefix + name
)
except Exception: except Exception:
uploaded[name] = "" uploaded[name] = ""
traceback.print_exc() traceback.print_exc()
# Upload all images and flamegraphs to S3 # Upload all images and flamegraphs to S3
try: try:
s3_helper.upload_test_folder_to_s3( s3_helper.upload_test_directory_to_s3(
os.path.join(result_path, "images"), s3_prefix + "images" Path(result_path) / "images", s3_prefix + "images"
) )
except Exception: except Exception:
traceback.print_exc() traceback.print_exc()

View File

@ -2,17 +2,58 @@
from ast import literal_eval from ast import literal_eval
from dataclasses import dataclass from dataclasses import dataclass
from pathlib import Path from pathlib import Path
from typing import List, Optional, Tuple from typing import Dict, Final, Iterable, List, Literal, Optional, Tuple
from html import escape from html import escape
import csv import csv
import os
import datetime import datetime
import json
import logging
import os
from ci_config import BuildConfig, CI_CONFIG
from env_helper import get_job_id_url
logger = logging.getLogger(__name__)
ERROR: Final = "error"
FAILURE: Final = "failure"
PENDING: Final = "pending"
SUCCESS: Final = "success"
OK: Final = "OK"
FAIL: Final = "FAIL"
StatusType = Literal["error", "failure", "pending", "success"]
# The order of statuses from the worst to the best
_STATES = {ERROR: 0, FAILURE: 1, PENDING: 2, SUCCESS: 3}
def get_worst_status(statuses: Iterable[str]) -> str:
worst_status = None
for status in statuses:
if _STATES.get(status) is None:
continue
if worst_status is None:
worst_status = status
continue
if _STATES.get(status) < _STATES.get(worst_status):
worst_status = status
if worst_status == ERROR:
break
if worst_status is None:
return ""
return worst_status
### BEST FRONTEND PRACTICES BELOW ### BEST FRONTEND PRACTICES BELOW
HTML_BASE_TEST_TEMPLATE = """ HEAD_HTML_TEMPLATE = """
<!DOCTYPE html> <!DOCTYPE html>
<html> <html>
<head>
<style> <style>
:root {{ :root {{
@ -98,15 +139,9 @@ tr:hover {{ filter: var(--tr-hover-filter); }}
<div class="main"> <div class="main">
<span class="nowrap themes"><span id="toggle-dark">🌚</span><span id="toggle-light">🌞</span></span> <span class="nowrap themes"><span id="toggle-dark">🌚</span><span id="toggle-light">🌞</span></span>
<h1><span class="gradient">{header}</span></h1> <h1><span class="gradient">{header}</span></h1>
<p class="links"> """
<a href="{raw_log_url}">{raw_log_name}</a>
<a href="{commit_url}">Commit</a> FOOTER_HTML_TEMPLATE = """<img id="fish" src="https://presentations.clickhouse.com/images/fish.png" />
{additional_urls}
<a href="{task_url}">Task (github actions)</a>
<a href="{job_url}">Job (github actions)</a>
</p>
{test_part}
<img id="fish" src="https://presentations.clickhouse.com/images/fish.png" />
<script type="text/javascript"> <script type="text/javascript">
/// Straight from https://stackoverflow.com/questions/14267781/sorting-html-table-with-javascript /// Straight from https://stackoverflow.com/questions/14267781/sorting-html-table-with-javascript
@ -161,6 +196,21 @@ tr:hover {{ filter: var(--tr-hover-filter); }}
</html> </html>
""" """
HTML_BASE_TEST_TEMPLATE = (
f"{HEAD_HTML_TEMPLATE}"
"""<p class="links">
<a href="{raw_log_url}">{raw_log_name}</a>
<a href="{commit_url}">Commit</a>
{additional_urls}
<a href="{task_url}">Task (github actions)</a>
<a href="{job_url}">Job (github actions)</a>
</p>
{test_part}
"""
f"{FOOTER_HTML_TEMPLATE}"
)
HTML_TEST_PART = """ HTML_TEST_PART = """
<table> <table>
<tr> <tr>
@ -238,12 +288,159 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes
@dataclass @dataclass
class BuildResult: class BuildResult:
compiler: str build_name: str
debug_build: bool log_url: str
sanitizer: str build_urls: List[str]
status: str version: str
status: StatusType
elapsed_seconds: int elapsed_seconds: int
comment: str job_name: str
_job_link: Optional[str] = None
_grouped_urls: Optional[List[List[str]]] = None
@property
def build_config(self) -> Optional[BuildConfig]:
return CI_CONFIG.build_config.get(self.build_name, None)
@property
def comment(self) -> str:
if self.build_config is None:
return self._wrong_config_message
return self.build_config.comment
@property
def compiler(self) -> str:
if self.build_config is None:
return self._wrong_config_message
return self.build_config.compiler
@property
def debug_build(self) -> bool:
if self.build_config is None:
return False
return self.build_config.debug_build
@property
def sanitizer(self) -> str:
if self.build_config is None:
return self._wrong_config_message
return self.build_config.sanitizer
@property
def grouped_urls(self) -> List[List[str]]:
"Combine and preserve build_urls by artifact types"
if self._grouped_urls is not None:
return self._grouped_urls
if not self.build_urls:
self._grouped_urls = [[]]
return self._grouped_urls
artifacts_groups = {
"apk": [],
"deb": [],
"binary": [],
"tgz": [],
"rpm": [],
"performance": [],
} # type: Dict[str, List[str]]
for url in self.build_urls:
if url.endswith("performance.tar.zst"):
artifacts_groups["performance"].append(url)
elif (
url.endswith(".deb")
or url.endswith(".buildinfo")
or url.endswith(".changes")
or url.endswith(".tar.gz")
):
artifacts_groups["deb"].append(url)
elif url.endswith(".apk"):
artifacts_groups["apk"].append(url)
elif url.endswith(".rpm"):
artifacts_groups["rpm"].append(url)
elif url.endswith(".tgz") or url.endswith(".tgz.sha512"):
artifacts_groups["tgz"].append(url)
else:
artifacts_groups["binary"].append(url)
self._grouped_urls = [urls for urls in artifacts_groups.values() if urls]
return self._grouped_urls
@property
def _wrong_config_message(self) -> str:
return "missing"
@property
def file_name(self) -> Path:
return self.get_report_name(self.build_name)
@property
def is_missing(self) -> bool:
"The report is created for missing json file"
return not (
self.log_url
or self.build_urls
or self.version != "missing"
or self.status != ERROR
)
@property
def job_link(self) -> str:
if self._job_link is not None:
return self._job_link
_, job_url = get_job_id_url(self.job_name)
self._job_link = f'<a href="{job_url}">{self.job_name}</a>'
return self._job_link
@staticmethod
def get_report_name(name: str) -> Path:
return Path(f"build_report_{name}.json")
@staticmethod
def read_json(directory: Path, build_name: str) -> "BuildResult":
path = directory / BuildResult.get_report_name(build_name)
try:
with open(path, "r", encoding="utf-8") as pf:
data = json.load(pf) # type: dict
except FileNotFoundError:
logger.warning(
"File %s for build named '%s' is not found", path, build_name
)
return BuildResult.missing_result(build_name)
return BuildResult(
data.get("build_name", build_name),
data.get("log_url", ""),
data.get("build_urls", []),
data.get("version", ""),
data.get("status", ERROR),
data.get("elapsed_seconds", 0),
data.get("job_name", ""),
)
@staticmethod
def missing_result(build_name: str) -> "BuildResult":
return BuildResult(build_name, "", [], "missing", ERROR, 0, "missing")
def write_json(self, directory: Path) -> Path:
path = directory / self.file_name
path.write_text(
json.dumps(
{
"build_name": self.build_name,
"log_url": self.log_url,
"build_urls": self.build_urls,
"version": self.version,
"status": self.status,
"elapsed_seconds": self.elapsed_seconds,
"job_name": self.job_name,
}
),
encoding="utf-8",
)
# TODO: remove after the artifacts are in S3 completely
env_path = Path(os.getenv("GITHUB_ENV", "/dev/null"))
with env_path.open("a", encoding="utf-8") as ef:
ef.write(f"BUILD_URLS={path.stem}")
return path
BuildResults = List[BuildResult] BuildResults = List[BuildResult]
@ -281,8 +478,8 @@ def _format_header(
def _get_status_style(status: str, colortheme: Optional[ColorTheme] = None) -> str: def _get_status_style(status: str, colortheme: Optional[ColorTheme] = None) -> str:
ok_statuses = ("OK", "success", "PASSED") ok_statuses = (OK, SUCCESS, "PASSED")
fail_statuses = ("FAIL", "failure", "error", "FAILED", "Timeout", "NOT_FAILED") fail_statuses = (FAIL, FAILURE, ERROR, "FAILED", "Timeout", "NOT_FAILED")
if colortheme is None: if colortheme is None:
colortheme = ReportColorTheme.default colortheme = ReportColorTheme.default
@ -333,7 +530,7 @@ def create_test_html_report(
additional_urls = [] additional_urls = []
if test_results: if test_results:
rows_part = "" rows_part = []
num_fails = 0 num_fails = 0
has_test_time = False has_test_time = False
has_log_urls = False has_log_urls = False
@ -348,11 +545,13 @@ def create_test_html_report(
if test_result.log_files is not None: if test_result.log_files is not None:
has_log_urls = True has_log_urls = True
row = "<tr>" row = []
has_error = test_result.status in ("FAIL", "NOT_FAILED") has_error = test_result.status in ("FAIL", "NOT_FAILED")
if has_error and test_result.raw_logs is not None: if has_error and test_result.raw_logs is not None:
row = '<tr class="failed">' row.append('<tr class="failed">')
row += "<td>" + test_result.name + "</td>" else:
row.append("<tr>")
row.append(f"<td>{test_result.name}</td>")
colspan += 1 colspan += 1
style = _get_status_style(test_result.status, colortheme=statuscolors) style = _get_status_style(test_result.status, colortheme=statuscolors)
@ -362,12 +561,12 @@ def create_test_html_report(
num_fails = num_fails + 1 num_fails = num_fails + 1
fail_id = f'id="fail{num_fails}" ' fail_id = f'id="fail{num_fails}" '
row += f'<td {fail_id}style="{style}">{test_result.status}</td>' row.append(f'<td {fail_id}style="{style}">{test_result.status}</td>')
colspan += 1 colspan += 1
if test_result.time is not None: if test_result.time is not None:
has_test_time = True has_test_time = True
row += f"<td>{test_result.time}</td>" row.append(f"<td>{test_result.time}</td>")
colspan += 1 colspan += 1
if test_result.log_urls is not None: if test_result.log_urls is not None:
@ -375,19 +574,19 @@ def create_test_html_report(
test_logs_html = "<br>".join( test_logs_html = "<br>".join(
[_get_html_url(url) for url in test_result.log_urls] [_get_html_url(url) for url in test_result.log_urls]
) )
row += "<td>" + test_logs_html + "</td>" row.append(f"<td>{test_logs_html}</td>")
colspan += 1 colspan += 1
row += "</tr>" row.append("</tr>")
rows_part += row rows_part.append("".join(row))
if test_result.raw_logs is not None: if test_result.raw_logs is not None:
raw_logs = escape(test_result.raw_logs) raw_logs = escape(test_result.raw_logs)
row = ( row_raw_logs = (
'<tr class="failed-content">' '<tr class="failed-content">'
f'<td colspan="{colspan}"><pre>{raw_logs}</pre></td>' f'<td colspan="{colspan}"><pre>{raw_logs}</pre></td>'
"</tr>" "</tr>"
) )
rows_part += row rows_part.append(row_raw_logs)
headers = BASE_HEADERS.copy() headers = BASE_HEADERS.copy()
if has_test_time: if has_test_time:
@ -396,7 +595,7 @@ def create_test_html_report(
headers.append("Logs") headers.append("Logs")
headers_html = "".join(["<th>" + h + "</th>" for h in headers]) headers_html = "".join(["<th>" + h + "</th>" for h in headers])
test_part = HTML_TEST_PART.format(headers=headers_html, rows=rows_part) test_part = HTML_TEST_PART.format(headers=headers_html, rows="".join(rows_part))
else: else:
test_part = "" test_part = ""
@ -423,31 +622,18 @@ def create_test_html_report(
return html return html
HTML_BASE_BUILD_TEMPLATE = """ HTML_BASE_BUILD_TEMPLATE = (
<!DOCTYPE html> f"{HEAD_HTML_TEMPLATE}"
<html> """<p class="links">
<head> <a href="{commit_url}">Commit</a>
<style> <a href="{task_url}">Task (github actions)</a>
body {{ font-family: "DejaVu Sans", "Noto Sans", Arial, sans-serif; background: #EEE; }} </p>
h1 {{ margin-left: 10px; }}
th, td {{ border: 0; padding: 5px 10px 5px 10px; text-align: left; vertical-align: top; line-height: 1.5; background-color: #FFF;
border: 0; box-shadow: 0 0 0 1px rgba(0, 0, 0, 0.05), 0 8px 25px -5px rgba(0, 0, 0, 0.1); }}
a {{ color: #06F; text-decoration: none; }}
a:hover, a:active {{ color: #F40; text-decoration: underline; }}
table {{ border: 0; }}
.main {{ margin: auto; }}
p.links a {{ padding: 5px; margin: 3px; background: #FFF; line-height: 2; white-space: nowrap; box-shadow: 0 0 0 1px rgba(0, 0, 0, 0.05), 0 8px 25px -5px rgba(0, 0, 0, 0.1); }}
tr:hover td {{filter: brightness(95%);}}
</style>
<title>{title}</title>
</head>
<body>
<div class="main">
<h1>{header}</h1>
<table> <table>
<tr> <tr>
<th>Config/job name</th>
<th>Compiler</th> <th>Compiler</th>
<th>Build type</th> <th>Build type</th>
<th>Version</th>
<th>Sanitizer</th> <th>Sanitizer</th>
<th>Status</th> <th>Status</th>
<th>Build log</th> <th>Build log</th>
@ -457,13 +643,9 @@ tr:hover td {{filter: brightness(95%);}}
</tr> </tr>
{rows} {rows}
</table> </table>
<p class="links">
<a href="{commit_url}">Commit</a>
<a href="{task_url}">Task (github actions)</a>
</p>
</body>
</html>
""" """
f"{FOOTER_HTML_TEMPLATE}"
)
LINK_TEMPLATE = '<a href="{url}">{text}</a>' LINK_TEMPLATE = '<a href="{url}">{text}</a>'
@ -471,64 +653,63 @@ LINK_TEMPLATE = '<a href="{url}">{text}</a>'
def create_build_html_report( def create_build_html_report(
header: str, header: str,
build_results: BuildResults, build_results: BuildResults,
build_logs_urls: List[str],
artifact_urls_list: List[List[str]],
task_url: str, task_url: str,
branch_url: str, branch_url: str,
branch_name: str, branch_name: str,
commit_url: str, commit_url: str,
) -> str: ) -> str:
rows = "" rows = []
for build_result, build_log_url, artifact_urls in zip( for build_result in build_results:
build_results, build_logs_urls, artifact_urls_list for artifact_urls in build_result.grouped_urls:
): row = ["<tr>"]
row = "<tr>" row.append(
row += f"<td>{build_result.compiler}</td>" f"<td>{build_result.build_name}<br/>{build_result.job_link}</td>"
)
row.append(f"<td>{build_result.compiler}</td>")
if build_result.debug_build: if build_result.debug_build:
row += "<td>debug</td>" row.append("<td>debug</td>")
else: else:
row += "<td>relwithdebuginfo</td>" row.append("<td>relwithdebuginfo</td>")
row.append(f"<td>{build_result.version}</td>")
if build_result.sanitizer: if build_result.sanitizer:
row += f"<td>{build_result.sanitizer}</td>" row.append(f"<td>{build_result.sanitizer}</td>")
else: else:
row += "<td>none</td>" row.append("<td>none</td>")
if build_result.status: if build_result.status:
style = _get_status_style(build_result.status) style = _get_status_style(build_result.status)
row += f'<td style="{style}">{build_result.status}</td>' row.append(f'<td style="{style}">{build_result.status}</td>')
else: else:
style = _get_status_style("error") style = _get_status_style(ERROR)
row += f'<td style="{style}">error</td>' row.append(f'<td style="{style}">error</td>')
row += f'<td><a href="{build_log_url}">link</a></td>' row.append(f'<td><a href="{build_result.log_url}">link</a></td>')
delta = "unknown"
if build_result.elapsed_seconds: if build_result.elapsed_seconds:
delta = datetime.timedelta(seconds=build_result.elapsed_seconds) delta = str(datetime.timedelta(seconds=build_result.elapsed_seconds))
else:
delta = "unknown" # type: ignore
row += f"<td>{delta}</td>" row.append(f"<td>{delta}</td>")
links = "" links = []
link_separator = "<br/>" link_separator = "<br/>"
if artifact_urls: if artifact_urls:
for artifact_url in artifact_urls: for artifact_url in artifact_urls:
links += LINK_TEMPLATE.format( links.append(
LINK_TEMPLATE.format(
text=_get_html_url_name(artifact_url), url=artifact_url text=_get_html_url_name(artifact_url), url=artifact_url
) )
links += link_separator )
if links: row.append(f"<td>{link_separator.join(links)}</td>")
links = links[: -len(link_separator)]
row += f"<td>{links}</td>"
row += f"<td>{build_result.comment}</td>" row.append(f"<td>{build_result.comment}</td>")
row += "</tr>" row.append("</tr>")
rows += row rows.append("".join(row))
return HTML_BASE_BUILD_TEMPLATE.format( return HTML_BASE_BUILD_TEMPLATE.format(
title=_format_header(header, branch_name), title=_format_header(header, branch_name),
header=_format_header(header, branch_name, branch_url), header=_format_header(header, branch_name, branch_url),
rows=rows, rows="".join(rows),
task_url=task_url, task_url=task_url,
branch_name=branch_name, branch_name=branch_name,
commit_url=commit_url, commit_url=commit_url,

View File

@ -52,12 +52,14 @@ class S3Helper:
self.host = S3_URL self.host = S3_URL
self.download_host = S3_DOWNLOAD self.download_host = S3_DOWNLOAD
def _upload_file_to_s3(self, bucket_name: str, file_path: str, s3_path: str) -> str: def _upload_file_to_s3(
self, bucket_name: str, file_path: Path, s3_path: str
) -> str:
logging.debug( logging.debug(
"Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path "Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path
) )
metadata = {} metadata = {}
if os.path.getsize(file_path) < 64 * 1024 * 1024: if file_path.stat().st_size < 64 * 1024 * 1024:
if ( if (
s3_path.endswith("txt") s3_path.endswith("txt")
or s3_path.endswith("log") or s3_path.endswith("log")
@ -97,17 +99,14 @@ class S3Helper:
if re.search(r"\.(txt|log|err|out)$", s3_path) or re.search( if re.search(r"\.(txt|log|err|out)$", s3_path) or re.search(
r"\.log\..*(?<!\.zst)$", s3_path r"\.log\..*(?<!\.zst)$", s3_path
): ):
compressed_path = file_path.with_suffix(file_path.suffix + ".zst")
logging.info( logging.info(
"Going to compress file log file %s to %s", "Going to compress file log file %s to %s",
file_path, file_path,
file_path + ".zst", compressed_path,
) )
# FIXME: rewrite S3 to Path compress_file_fast(file_path, compressed_path)
_file_path = Path(file_path) file_path = compressed_path
compress_file_fast(
_file_path, _file_path.with_suffix(_file_path.suffix + ".zst")
)
file_path += ".zst"
s3_path += ".zst" s3_path += ".zst"
else: else:
logging.info("Processing file without compression") logging.info("Processing file without compression")
@ -121,22 +120,20 @@ class S3Helper:
logging.info("Upload %s to %s. Meta: %s", file_path, url, metadata) logging.info("Upload %s to %s. Meta: %s", file_path, url, metadata)
return url return url
def upload_test_report_to_s3(self, file_path: str, s3_path: str) -> str: def upload_test_report_to_s3(self, file_path: Path, s3_path: str) -> str:
if CI: if CI:
return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path)
else:
return S3Helper.copy_file_to_local(
S3_TEST_REPORTS_BUCKET, file_path, s3_path
)
def upload_build_file_to_s3(self, file_path, s3_path): return S3Helper.copy_file_to_local(S3_TEST_REPORTS_BUCKET, file_path, s3_path)
def upload_build_file_to_s3(self, file_path: Path, s3_path: str) -> str:
if CI: if CI:
return self._upload_file_to_s3(S3_BUILDS_BUCKET, file_path, s3_path) return self._upload_file_to_s3(S3_BUILDS_BUCKET, file_path, s3_path)
else:
return S3Helper.copy_file_to_local(S3_BUILDS_BUCKET, file_path, s3_path) return S3Helper.copy_file_to_local(S3_BUILDS_BUCKET, file_path, s3_path)
def fast_parallel_upload_dir( def fast_parallel_upload_dir(
self, dir_path: Union[str, Path], s3_dir_path: str, bucket_name: str self, dir_path: Path, s3_dir_path: str, bucket_name: str
) -> List[str]: ) -> List[str]:
all_files = [] all_files = []
@ -196,37 +193,37 @@ class S3Helper:
logging.basicConfig(level=original_level) logging.basicConfig(level=original_level)
return result return result
def _upload_folder_to_s3( def _upload_directory_to_s3(
self, self,
folder_path, directory_path: Path,
s3_folder_path, s3_directory_path: str,
bucket_name, bucket_name: str,
keep_dirs_in_s3_path, keep_dirs_in_s3_path: bool,
upload_symlinks, upload_symlinks: bool,
): ) -> List[str]:
logging.info( logging.info(
"Upload folder '%s' to bucket=%s of s3 folder '%s'", "Upload directory '%s' to bucket=%s of s3 directory '%s'",
folder_path, directory_path,
bucket_name, bucket_name,
s3_folder_path, s3_directory_path,
) )
if not os.path.exists(folder_path): if not directory_path.exists():
return [] return []
files = os.listdir(folder_path) files = list(directory_path.iterdir())
if not files: if not files:
return [] return []
p = Pool(min(len(files), 5)) p = Pool(min(len(files), 5))
def task(file_name): def task(file_path: Path) -> Union[str, List[str]]:
full_fs_path = os.path.join(folder_path, file_name) full_fs_path = file_path.absolute()
if keep_dirs_in_s3_path: if keep_dirs_in_s3_path:
full_s3_path = s3_folder_path + "/" + os.path.basename(folder_path) full_s3_path = os.path.join(s3_directory_path, directory_path.name)
else: else:
full_s3_path = s3_folder_path full_s3_path = s3_directory_path
if os.path.isdir(full_fs_path): if os.path.isdir(full_fs_path):
return self._upload_folder_to_s3( return self._upload_directory_to_s3(
full_fs_path, full_fs_path,
full_s3_path, full_s3_path,
bucket_name, bucket_name,
@ -234,60 +231,63 @@ class S3Helper:
upload_symlinks, upload_symlinks,
) )
if os.path.islink(full_fs_path): if full_fs_path.is_symlink():
if upload_symlinks: if upload_symlinks:
if CI: if CI:
return self._upload_file_to_s3( return self._upload_file_to_s3(
bucket_name, full_fs_path, full_s3_path + "/" + file_name bucket_name,
full_fs_path,
full_s3_path + "/" + file_path.name,
) )
else:
return S3Helper.copy_file_to_local( return S3Helper.copy_file_to_local(
bucket_name, full_fs_path, full_s3_path + "/" + file_name bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
) )
return [] return []
if CI: if CI:
return self._upload_file_to_s3( return self._upload_file_to_s3(
bucket_name, full_fs_path, full_s3_path + "/" + file_name bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
) )
else:
return S3Helper.copy_file_to_local( return S3Helper.copy_file_to_local(
bucket_name, full_fs_path, full_s3_path + "/" + file_name bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
) )
return sorted(_flatten_list(list(p.map(task, files)))) return sorted(_flatten_list(list(p.map(task, files))))
def upload_build_folder_to_s3( def upload_build_directory_to_s3(
self, self,
folder_path, directory_path: Path,
s3_folder_path, s3_directory_path: str,
keep_dirs_in_s3_path=True, keep_dirs_in_s3_path: bool = True,
upload_symlinks=True, upload_symlinks: bool = True,
): ) -> List[str]:
return self._upload_folder_to_s3( return self._upload_directory_to_s3(
folder_path, directory_path,
s3_folder_path, s3_directory_path,
S3_BUILDS_BUCKET, S3_BUILDS_BUCKET,
keep_dirs_in_s3_path, keep_dirs_in_s3_path,
upload_symlinks, upload_symlinks,
) )
def upload_test_folder_to_s3( def upload_test_directory_to_s3(
self, self,
folder_path, directory_path: Path,
s3_folder_path, s3_directory_path: str,
keep_dirs_in_s3_path=True, keep_dirs_in_s3_path: bool = True,
upload_symlinks=True, upload_symlinks: bool = True,
): ) -> List[str]:
return self._upload_folder_to_s3( return self._upload_directory_to_s3(
folder_path, directory_path,
s3_folder_path, s3_directory_path,
S3_TEST_REPORTS_BUCKET, S3_TEST_REPORTS_BUCKET,
keep_dirs_in_s3_path, keep_dirs_in_s3_path,
upload_symlinks, upload_symlinks,
) )
def list_prefix(self, s3_prefix_path, bucket=S3_BUILDS_BUCKET): def list_prefix(
self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET
) -> List[str]:
objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path) objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path)
result = [] result = []
if "Contents" in objects: if "Contents" in objects:
@ -296,7 +296,7 @@ class S3Helper:
return result return result
def exists(self, key, bucket=S3_BUILDS_BUCKET): def exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> bool:
try: try:
self.client.head_object(Bucket=bucket, Key=key) self.client.head_object(Bucket=bucket, Key=key)
return True return True
@ -304,13 +304,12 @@ class S3Helper:
return False return False
@staticmethod @staticmethod
def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str: def copy_file_to_local(bucket_name: str, file_path: Path, s3_path: str) -> str:
local_path = os.path.abspath( local_path = (
os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path) Path(RUNNER_TEMP) / "s3" / os.path.join(bucket_name, s3_path)
) ).absolute()
local_dir = os.path.dirname(local_path) local_dir = local_path.parent
if not os.path.exists(local_dir): local_dir.mkdir(parents=True, exist_ok=True)
os.makedirs(local_dir)
shutil.copy(file_path, local_path) shutil.copy(file_path, local_path)
logging.info("Copied %s to %s", file_path, local_path) logging.info("Copied %s to %s", file_path, local_path)

View File

@ -4,6 +4,8 @@ import logging
import subprocess import subprocess
import os import os
import sys import sys
from pathlib import Path
from typing import Dict
from github import Github from github import Github
@ -47,13 +49,12 @@ def main():
stopwatch = Stopwatch() stopwatch = Stopwatch()
temp_path = TEMP_PATH temp_path = Path(TEMP_PATH)
reports_path = REPORTS_PATH reports_path = Path(REPORTS_PATH)
check_name = sys.argv[1] check_name = sys.argv[1]
if not os.path.exists(temp_path): temp_path.mkdir(parents=True, exist_ok=True)
os.makedirs(temp_path)
pr_info = PRInfo() pr_info = PRInfo()
@ -82,7 +83,7 @@ def main():
logging.info("Got build url %s", build_url) logging.info("Got build url %s", build_url)
workspace_path = os.path.join(temp_path, "workspace") workspace_path = temp_path / "workspace"
if not os.path.exists(workspace_path): if not os.path.exists(workspace_path):
os.makedirs(workspace_path) os.makedirs(workspace_path)
@ -91,7 +92,7 @@ def main():
) )
logging.info("Going to run %s", run_command) logging.info("Going to run %s", run_command)
run_log_path = os.path.join(temp_path, "run.log") run_log_path = temp_path / "run.log"
with open(run_log_path, "w", encoding="utf-8") as log: with open(run_log_path, "w", encoding="utf-8") as log:
with subprocess.Popen( with subprocess.Popen(
run_command, shell=True, stderr=log, stdout=log run_command, shell=True, stderr=log, stdout=log
@ -110,23 +111,24 @@ def main():
s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/"
paths = { paths = {
"run.log": run_log_path, "run.log": run_log_path,
"server.log.zst": os.path.join(workspace_path, "server.log.zst"), "server.log.zst": workspace_path / "server.log.zst",
"server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), "server.err.log.zst": workspace_path / "server.err.log.zst",
"report.html": os.path.join(workspace_path, "report.html"), "report.html": workspace_path / "report.html",
"test.log": os.path.join(workspace_path, "test.log"), "test.log": workspace_path / "test.log",
} }
path_urls = {} # type: Dict[str, str]
s3_helper = S3Helper() s3_helper = S3Helper()
for f in paths: for f in paths:
try: try:
paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f) path_urls[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f)
except Exception as ex: except Exception as ex:
logging.info("Exception uploading file %s text %s", f, ex) logging.info("Exception uploading file %s text %s", f, ex)
paths[f] = "" path_urls[f] = ""
report_url = GITHUB_RUN_URL report_url = GITHUB_RUN_URL
if paths["report.html"]: if path_urls["report.html"]:
report_url = paths["report.html"] report_url = path_urls["report.html"]
status = "success" status = "success"
description = "See the report" description = "See the report"

View File

@ -34,7 +34,7 @@ def process_logs(
test_result.log_urls.append(processed_logs[path]) test_result.log_urls.append(processed_logs[path])
elif path: elif path:
url = s3_client.upload_test_report_to_s3( url = s3_client.upload_test_report_to_s3(
path.as_posix(), s3_path_prefix + "/" + path.name path, s3_path_prefix + "/" + path.name
) )
test_result.log_urls.append(url) test_result.log_urls.append(url)
processed_logs[path] = url processed_logs[path] = url
@ -44,7 +44,7 @@ def process_logs(
if log_path: if log_path:
additional_urls.append( additional_urls.append(
s3_client.upload_test_report_to_s3( s3_client.upload_test_report_to_s3(
log_path, s3_path_prefix + "/" + os.path.basename(log_path) Path(log_path), s3_path_prefix + "/" + os.path.basename(log_path)
) )
) )
@ -100,9 +100,9 @@ def upload_results(
additional_urls, additional_urls,
statuscolors=statuscolors, statuscolors=statuscolors,
) )
with open("report.html", "w", encoding="utf-8") as f: report_path = Path("report.html")
f.write(html_report) report_path.write_text(html_report, encoding="utf-8")
url = s3_client.upload_test_report_to_s3("report.html", s3_path_prefix + ".html") url = s3_client.upload_test_report_to_s3(report_path, s3_path_prefix + ".html")
logging.info("Search result in url %s", url) logging.info("Search result in url %s", url)
return url return url

View File

@ -245,8 +245,10 @@ def get_version_from_string(
def get_version_from_tag(tag: str) -> ClickHouseVersion: def get_version_from_tag(tag: str) -> ClickHouseVersion:
Git.check_tag(tag) Git.check_tag(tag)
tag = tag[1:].split("-")[0] tag, description = tag[1:].split("-", 1)
return get_version_from_string(tag) version = get_version_from_string(tag)
version.with_description(description)
return version
def version_arg(version: str) -> ClickHouseVersion: def version_arg(version: str) -> ClickHouseVersion:

View File

@ -1,6 +1,13 @@
<clickhouse> <clickhouse>
<storage_configuration>
<disks>
<backups> <backups>
<allowed_disk>default</allowed_disk> <type>local</type>
<allowed_path>/backups</allowed_path> <path>/var/lib/clickhouse/disks/backups/</path>
</backups>
</disks>
</storage_configuration>
<backups>
<allowed_disk>backups</allowed_disk>
</backups> </backups>
</clickhouse> </clickhouse>

View File

@ -9,7 +9,7 @@ server_ip = sys.argv[2]
mutex = threading.Lock() mutex = threading.Lock()
success_counter = 0 success_counter = 0
number_of_threads = 100 number_of_threads = 100
number_of_iterations = 100 number_of_iterations = 50
def perform_request(): def perform_request():

View File

@ -10,6 +10,8 @@
<account_name>devstoreaccount1</account_name> <account_name>devstoreaccount1</account_name>
<account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key> <account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key>
<max_single_part_upload_size>100000</max_single_part_upload_size> <max_single_part_upload_size>100000</max_single_part_upload_size>
<max_single_download_retries>10</max_single_download_retries>
<max_single_read_retries>10</max_single_read_retries>
<!-- NOTE: container_already_exists is omitted to: <!-- NOTE: container_already_exists is omitted to:
a) create it a) create it
b) ignore if it already exists, since there are two instances, that conflicts with each other b) ignore if it already exists, since there are two instances, that conflicts with each other

View File

@ -0,0 +1,4 @@
Decimal(9, 8)
Decimal(18, 0)
Decimal(10, 0)
Decimal(18, 0) Decimal(10, 0)

View File

@ -0,0 +1,19 @@
DROP TABLE IF EXISTS decimal;
CREATE TABLE IF NOT EXISTS decimal
(
d1 DECIMAL(9, 8),
d2 DECIMAL(18),
d3 DECIMAL
)
ENGINE = MergeTree
PARTITION BY toInt32(d1)
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 toTypeName(d2), toTypeName(d3) FROM decimal LIMIT 1;
DROP TABLE decimal;

View File

@ -20,3 +20,5 @@ world
3 3
--- ---
4 4
---
1

View File

@ -27,3 +27,7 @@ SELECT count() FROM
UNION ALL UNION ALL
SELECT * FROM test WHERE isZeroOrNull(x != 'xyz') SELECT * FROM test WHERE isZeroOrNull(x != 'xyz')
); );
SELECT '---';
select isZeroOrNull(Null);

View File

@ -42,3 +42,8 @@
2023-05-30 2023-05-30
2149-06-06 2149-06-06
1970-01-20 1970-01-20
Tests for issue #38585
\N
\N
\N
\N

View File

@ -49,3 +49,10 @@ SELECT accurateCastOrNull('1xxx', 'Date');
SELECT accurateCastOrNull('2023-05-30', 'Date'); SELECT accurateCastOrNull('2023-05-30', 'Date');
SELECT accurateCastOrNull('2180-01-01', 'Date'); SELECT accurateCastOrNull('2180-01-01', 'Date');
SELECT accurateCastOrNull(19, 'Date'); SELECT accurateCastOrNull(19, 'Date');
SELECT 'Tests for issue #38585';
select accurateCastOrNull('test', 'Bool');
select accurateCast('test', 'Nullable(Bool)');
select accurateCastOrNull('test', 'IPv4');
select accurateCastOrNull('test', 'IPv6');

View File

@ -17,7 +17,7 @@ true
\N \N
0.0.0.0 0.0.0.0
\N \N
0.0.0.0 \N
\N \N
\N \N
\N \N
@ -25,7 +25,7 @@ true
\N \N
:: ::
\N \N
:: \N
\N \N
\N \N
\N \N

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