mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Merge remote-tracking branch 'origin/master' into pr-distributed-prefer-localhost-replica
This commit is contained in:
commit
f938c38eb9
@ -23,7 +23,7 @@ array or map columns and strings correspondingly in generated data.
|
||||
|
||||
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}
|
||||
|
||||
|
@ -4,15 +4,17 @@ sidebar_position: 42
|
||||
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).
|
||||
|
||||
## 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.
|
||||
|
||||
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:
|
||||
- P from \[ 1 : 9 \] - for Decimal32(S)
|
||||
- P from \[ 10 : 18 \] - for Decimal64(S)
|
||||
|
@ -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
|
||||
|
||||
Returns the leftmost non-`NULL` argument.
|
||||
|
@ -395,11 +395,15 @@ These codecs are designed to make compression more effective by using specific f
|
||||
|
||||
#### 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, it’s 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, it’s 1. Delta is a data preparation codec, i.e. cannot be used stand-alone.
|
||||
|
||||
#### DoubleDelta
|
||||
|
||||
`DoubleDelta(bytes_size)` — Calculates delta of deltas and writes it in compact binary form. Possible `bytes_size` values: 1, 2, 4, 8, the default value is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, it’s 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, it’s 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
|
||||
|
||||
|
@ -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.
|
||||
- `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).
|
||||
- `T64` — Метод сжатия который обрезает неиспользуемые старшие биты целочисленных значений (включая `Enum`, `Date` и `DateTime`). На каждом шаге алгоритма, кодек помещает блок из 64 значений в матрицу 64✕64, транспонирует её, обрезает неиспользуемые биты, а то, что осталось возвращает в виде последовательности. Неиспользуемые биты, это биты, которые не изменяются от минимального к максимальному на всём диапазоне значений куска данных.
|
||||
|
||||
|
@ -555,11 +555,13 @@ catch (...)
|
||||
|
||||
void Keeper::logRevision() const
|
||||
{
|
||||
Poco::Logger::root().information("Starting ClickHouse Keeper " + std::string{VERSION_STRING}
|
||||
+ "(revision : " + std::to_string(ClickHouseRevision::getVersionRevision())
|
||||
+ ", git hash: " + (git_hash.empty() ? "<unknown>" : git_hash)
|
||||
+ ", build id: " + (build_id.empty() ? "<unknown>" : build_id) + ")"
|
||||
+ ", PID " + std::to_string(getpid()));
|
||||
LOG_INFO(&Poco::Logger::get("Application"),
|
||||
"Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}",
|
||||
VERSION_STRING,
|
||||
ClickHouseRevision::getVersionRevision(),
|
||||
git_hash.empty() ? "<unknown>" : git_hash,
|
||||
build_id.empty() ? "<unknown>" : build_id,
|
||||
getpid());
|
||||
}
|
||||
|
||||
|
||||
|
@ -31,6 +31,7 @@ namespace ErrorCodes
|
||||
M(Bool, read_from_filesystem_cache) \
|
||||
M(UInt64, shard_num) \
|
||||
M(UInt64, replica_num) \
|
||||
M(Bool, check_parts) \
|
||||
M(Bool, internal) \
|
||||
M(String, host_id) \
|
||||
M(OptionalUUID, backup_uuid)
|
||||
|
@ -59,6 +59,9 @@ struct BackupSettings
|
||||
/// Can only be used with BACKUP ON CLUSTER.
|
||||
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.
|
||||
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
|
||||
bool internal = false;
|
||||
|
@ -164,6 +164,7 @@ namespace
|
||||
M(RestoreUDFCreationMode, create_function) \
|
||||
M(Bool, allow_s3_native_copy) \
|
||||
M(Bool, use_same_s3_credentials_for_base_backup) \
|
||||
M(Bool, restore_broken_parts_as_detached) \
|
||||
M(Bool, internal) \
|
||||
M(String, host_id) \
|
||||
M(OptionalString, storage_policy) \
|
||||
|
@ -113,6 +113,10 @@ struct RestoreSettings
|
||||
/// Whether base backup from S3 should inherit credentials from the RESTORE query.
|
||||
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.
|
||||
bool internal = false;
|
||||
|
||||
|
@ -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"));
|
||||
|
||||
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(
|
||||
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);
|
||||
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);
|
||||
|
||||
struct rlimit limit;
|
||||
|
@ -134,6 +134,8 @@ void Connection::disconnect()
|
||||
if (!is_initialized)
|
||||
return;
|
||||
|
||||
// If driver->free_me, then mysql_close will deallocate memory by calling 'free' function.
|
||||
assert(driver && !driver->free_me);
|
||||
mysql_close(driver.get());
|
||||
memset(driver.get(), 0, sizeof(*driver));
|
||||
|
||||
|
@ -67,7 +67,7 @@ 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 delta compress, data size {} is not aligned to {}", source_size, sizeof(T));
|
||||
throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot delta compress, data size {} is not aligned to {}", source_size, sizeof(T));
|
||||
|
||||
T prev_src = 0;
|
||||
const char * const source_end = source + source_size;
|
||||
@ -88,7 +88,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
|
||||
const char * const output_end = dest + output_size;
|
||||
|
||||
if (source_size % sizeof(T) != 0)
|
||||
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot delta decompress, data size {} is not aligned to {}", source_size, sizeof(T));
|
||||
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot delta decompress, data size {} is not aligned to {}", source_size, sizeof(T));
|
||||
|
||||
T accumulator{};
|
||||
const char * const source_end = source + source_size;
|
||||
|
271
src/Compression/CompressionCodecGCD.cpp
Normal file
271
src/Compression/CompressionCodecGCD.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
@ -168,7 +168,9 @@ void registerCodecLZ4(CompressionCodecFactory & factory);
|
||||
void registerCodecLZ4HC(CompressionCodecFactory & factory);
|
||||
void registerCodecZSTD(CompressionCodecFactory & factory);
|
||||
void registerCodecMultiple(CompressionCodecFactory & factory);
|
||||
#ifdef ENABLE_QPL_COMPRESSION
|
||||
void registerCodecDeflateQpl(CompressionCodecFactory & factory);
|
||||
#endif
|
||||
|
||||
/// Keeper use only general-purpose codecs, so we don't need these special codecs
|
||||
/// in standalone build
|
||||
@ -179,6 +181,7 @@ void registerCodecDoubleDelta(CompressionCodecFactory & factory);
|
||||
void registerCodecGorilla(CompressionCodecFactory & factory);
|
||||
void registerCodecEncrypted(CompressionCodecFactory & factory);
|
||||
void registerCodecFPC(CompressionCodecFactory & factory);
|
||||
void registerCodecGCD(CompressionCodecFactory & factory);
|
||||
#endif
|
||||
|
||||
CompressionCodecFactory::CompressionCodecFactory()
|
||||
@ -198,6 +201,7 @@ CompressionCodecFactory::CompressionCodecFactory()
|
||||
#ifdef ENABLE_QPL_COMPRESSION
|
||||
registerCodecDeflateQpl(*this);
|
||||
#endif
|
||||
registerCodecGCD(*this);
|
||||
#endif
|
||||
|
||||
default_codec = get("LZ4", {});
|
||||
|
@ -47,6 +47,7 @@ enum class CompressionMethodByte : uint8_t
|
||||
AES_256_GCM_SIV = 0x97,
|
||||
FPC = 0x98,
|
||||
DeflateQpl = 0x99,
|
||||
GCD = 0x9a,
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -18,3 +18,6 @@ target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms)
|
||||
|
||||
clickhouse_add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp)
|
||||
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)
|
||||
|
45
src/Compression/fuzzers/gcd_decompress_fuzzer.cpp
Normal file
45
src/Compression/fuzzers/gcd_decompress_fuzzer.cpp
Normal 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;
|
||||
}
|
@ -48,7 +48,7 @@
|
||||
/// the number is unmotivated
|
||||
#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/"
|
||||
|
||||
|
@ -74,21 +74,30 @@ SerializationPtr DataTypeDecimal<T>::doGetDefaultSerialization() const
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Decimal data type family must have exactly two arguments: precision and scale");
|
||||
UInt64 precision = 10;
|
||||
UInt64 scale = 0;
|
||||
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 * scale = arguments->children[1]->as<ASTLiteral>();
|
||||
const auto * precision_arg = arguments->children[0]->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 ||
|
||||
!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");
|
||||
if (arguments->children.size() == 2)
|
||||
{
|
||||
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>();
|
||||
UInt64 scale_value = scale->value.get<UInt64>();
|
||||
|
||||
return createDecimal<DataTypeDecimal>(precision_value, scale_value);
|
||||
return createDecimal<DataTypeDecimal>(precision, scale);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -395,11 +395,15 @@ void FlatDictionary::updateData()
|
||||
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
|
||||
{
|
||||
QueryPipeline pipeline(source_ptr->loadUpdatedAll());
|
||||
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
update_field_loaded_block.reset();
|
||||
Block 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
|
||||
|
@ -409,11 +409,17 @@ void HashedArrayDictionary<dictionary_key_type>::updateData()
|
||||
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
|
||||
{
|
||||
QueryPipeline pipeline(source_ptr->loadUpdatedAll());
|
||||
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
update_field_loaded_block.reset();
|
||||
Block 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
|
||||
if (!update_field_loaded_block)
|
||||
update_field_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty());
|
||||
|
@ -709,11 +709,15 @@ void HashedDictionary<dictionary_key_type, sparse, sharded>::updateData()
|
||||
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
|
||||
{
|
||||
QueryPipeline pipeline(source_ptr->loadUpdatedAll());
|
||||
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
update_field_loaded_block.reset();
|
||||
Block 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
|
||||
|
@ -919,11 +919,17 @@ void RangeHashedDictionary<dictionary_key_type>::updateData()
|
||||
if (!update_field_loaded_block || update_field_loaded_block->rows() == 0)
|
||||
{
|
||||
QueryPipeline pipeline(source_ptr->loadUpdatedAll());
|
||||
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
update_field_loaded_block.reset();
|
||||
Block 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
|
||||
if (!update_field_loaded_block)
|
||||
update_field_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty());
|
||||
|
@ -124,11 +124,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl()
|
||||
read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds);
|
||||
break;
|
||||
}
|
||||
catch (const Azure::Core::Http::TransportException & e)
|
||||
{
|
||||
handle_exception(e, i);
|
||||
}
|
||||
catch (const Azure::Storage::StorageException & e)
|
||||
catch (const Azure::Core::RequestFailedException & e)
|
||||
{
|
||||
handle_exception(e, i);
|
||||
}
|
||||
@ -240,10 +236,6 @@ void ReadBufferFromAzureBlobStorage::initialize()
|
||||
data_stream = std::move(download_response.Value.BodyStream);
|
||||
break;
|
||||
}
|
||||
catch (const Azure::Core::Http::TransportException & e)
|
||||
{
|
||||
handle_exception(e, i);
|
||||
}
|
||||
catch (const Azure::Core::RequestFailedException & e)
|
||||
{
|
||||
handle_exception(e,i);
|
||||
|
@ -55,7 +55,7 @@ std::unique_ptr<ReadBuffer> ReadBufferFromWebServer::initialize()
|
||||
|
||||
const auto & settings = context->getSettingsRef();
|
||||
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>(
|
||||
uri,
|
||||
|
@ -62,10 +62,6 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function<void()> func,
|
||||
func();
|
||||
break;
|
||||
}
|
||||
catch (const Azure::Core::Http::TransportException & e)
|
||||
{
|
||||
handle_exception(e, i);
|
||||
}
|
||||
catch (const Azure::Core::RequestFailedException & e)
|
||||
{
|
||||
handle_exception(e, i);
|
||||
|
@ -62,7 +62,8 @@ std::unique_ptr<S3::Client> getClient(
|
||||
client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000);
|
||||
client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100);
|
||||
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.wait_on_pool_size_limit = false;
|
||||
|
||||
|
@ -35,7 +35,7 @@ TEST(AzureBlobContainerClient, CurlMemoryLeak)
|
||||
options.Retry.MaxRetries = 0;
|
||||
|
||||
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
|
||||
|
@ -82,6 +82,7 @@ try
|
||||
bool is_eof = false;
|
||||
try
|
||||
{
|
||||
read_buffer_iterator.setPreviousReadBuffer(std::move(buf));
|
||||
buf = read_buffer_iterator.next();
|
||||
if (!buf)
|
||||
break;
|
||||
|
@ -11,6 +11,8 @@ struct IReadBufferIterator
|
||||
{
|
||||
virtual ~IReadBufferIterator() = default;
|
||||
|
||||
virtual void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> /* buffer */) {}
|
||||
|
||||
virtual std::unique_ptr<ReadBuffer> next() = 0;
|
||||
|
||||
virtual std::optional<ColumnsDescription> getCachedColumns() { return std::nullopt; }
|
||||
@ -21,7 +23,7 @@ struct IReadBufferIterator
|
||||
struct SingleReadBufferIterator : public IReadBufferIterator
|
||||
{
|
||||
public:
|
||||
SingleReadBufferIterator(std::unique_ptr<ReadBuffer> buf_) : buf(std::move(buf_))
|
||||
explicit SingleReadBufferIterator(std::unique_ptr<ReadBuffer> buf_) : buf(std::move(buf_))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -87,6 +87,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
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);
|
||||
ReadBufferFromMemory read_buffer(val.data, val.size);
|
||||
serialization_from.deserializeWholeText(column_to, read_buffer, format_settings);
|
||||
try
|
||||
{
|
||||
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())
|
||||
{
|
||||
@ -4054,15 +4067,21 @@ private:
|
||||
{
|
||||
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](
|
||||
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t)
|
||||
-> ColumnPtr
|
||||
ret = [cast_ipv4_ipv6_default_on_conversion_error_value,
|
||||
input_format_ipv4_default_on_conversion_error_value,
|
||||
requested_result_is_nullable](
|
||||
ColumnsWithTypeAndName & arguments,
|
||||
const DataTypePtr & result_type,
|
||||
const ColumnNullable * column_nullable,
|
||||
size_t) -> ColumnPtr
|
||||
{
|
||||
if (!WhichDataType(result_type).isIPv4())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName());
|
||||
|
||||
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);
|
||||
else
|
||||
return convertToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
|
||||
@ -4073,16 +4092,22 @@ private:
|
||||
|
||||
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](
|
||||
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t)
|
||||
-> ColumnPtr
|
||||
ret = [cast_ipv4_ipv6_default_on_conversion_error_value,
|
||||
input_format_ipv6_default_on_conversion_error_value,
|
||||
requested_result_is_nullable](
|
||||
ColumnsWithTypeAndName & arguments,
|
||||
const DataTypePtr & result_type,
|
||||
const ColumnNullable * column_nullable,
|
||||
size_t) -> ColumnPtr
|
||||
{
|
||||
if (!WhichDataType(result_type).isIPv6())
|
||||
throw Exception(
|
||||
ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName());
|
||||
|
||||
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);
|
||||
else
|
||||
return convertToIPv6<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
|
||||
@ -4093,7 +4118,18 @@ private:
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
@ -4108,7 +4144,9 @@ private:
|
||||
ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName());
|
||||
|
||||
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);
|
||||
else
|
||||
return convertIPv6ToIPv4<IPStringToNumExceptionMode::Throw>(arguments[0].column, null_map);
|
||||
|
@ -102,17 +102,13 @@ private:
|
||||
|
||||
if (key_argument_data_type.isArray())
|
||||
{
|
||||
DataTypePtr value_type;
|
||||
if (1 < arguments.size())
|
||||
value_type = arguments[1];
|
||||
|
||||
if (arguments.size() < 2 || (value_type && !isArray(value_type)))
|
||||
if (arguments.size() < 2 || !arguments[1] || !isArray(arguments[1]))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Function {} if array argument is passed as key, additional array argument as value must be passed",
|
||||
getName());
|
||||
|
||||
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();
|
||||
value_argument_series_type = value_array_type.getNestedType();
|
||||
|
@ -44,14 +44,18 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & types) const override
|
||||
{
|
||||
if (!isNumber(removeNullable(types.at(0))))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The argument of function {} must have simple numeric type, possibly Nullable", name);
|
||||
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 or Null", name);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
ColumnPtr res;
|
||||
@ -72,7 +76,10 @@ public:
|
||||
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
|
||||
@ -89,7 +96,10 @@ public:
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -11,6 +11,9 @@
|
||||
# include <aws/core/utils/UUID.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 <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);
|
||||
}
|
||||
|
||||
const char SSO_CREDENTIALS_PROVIDER_LOG_TAG[] = "SSOCredentialsProvider";
|
||||
|
||||
}
|
||||
|
||||
AWSEC2MetadataClient::AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration, const char * endpoint_)
|
||||
@ -449,6 +454,139 @@ void AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::refreshIfExpired()
|
||||
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(
|
||||
const DB::S3::PocoHTTPClientConfiguration & configuration,
|
||||
const Aws::Auth::AWSCredentials & credentials,
|
||||
@ -494,6 +632,18 @@ S3CredentialsProviderChain::S3CredentialsProviderChain(
|
||||
|
||||
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.
|
||||
const auto relative_uri = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI);
|
||||
|
@ -8,6 +8,7 @@
|
||||
# include <aws/core/internal/AWSHttpResourceClient.h>
|
||||
# include <aws/core/config/AWSProfileConfigLoader.h>
|
||||
# include <aws/core/auth/AWSCredentialsProviderChain.h>
|
||||
# include <aws/core/auth/bearer-token-provider/SSOBearerTokenProvider.h>
|
||||
|
||||
# include <IO/S3/PocoHTTPClient.h>
|
||||
|
||||
@ -124,6 +125,39 @@ private:
|
||||
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
|
||||
{
|
||||
bool use_environment_credentials = false;
|
||||
|
@ -2506,6 +2506,96 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
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_)
|
||||
:actions(actions_)
|
||||
{
|
||||
|
@ -384,6 +384,16 @@ public:
|
||||
const ContextPtr & context,
|
||||
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:
|
||||
NodeRawConstPtrs getParents(const Node * target) const;
|
||||
|
||||
|
@ -345,7 +345,7 @@ Block createBlockForSet(
|
||||
{
|
||||
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.
|
||||
auto element = func->arguments->children.at(0);
|
||||
@ -356,6 +356,7 @@ Block createBlockForSet(
|
||||
return evaluateConstantExpression(func, context).second;
|
||||
};
|
||||
|
||||
assert(right_arg);
|
||||
const DataTypePtr & right_arg_type = get_tuple_type_from_ast(right_arg);
|
||||
|
||||
size_t left_tuple_depth = getTypeDepth(left_arg_type);
|
||||
|
@ -123,7 +123,7 @@ void SelectStreamFactory::createForShard(
|
||||
auto emplace_local_stream = [&]()
|
||||
{
|
||||
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)
|
||||
|
@ -604,15 +604,13 @@ std::optional<bool> tryEvaluateConstCondition(ASTPtr expr, ContextPtr context)
|
||||
|
||||
Field eval_res;
|
||||
DataTypePtr eval_res_type;
|
||||
try
|
||||
{
|
||||
std::tie(eval_res, eval_res_type) = evaluateConstantExpression(expr, context);
|
||||
}
|
||||
catch (DB::Exception &)
|
||||
{
|
||||
/// not a constant expression
|
||||
return {};
|
||||
auto constant_expression_result = tryEvaluateConstantExpression(expr, context);
|
||||
if (!constant_expression_result)
|
||||
return {};
|
||||
std::tie(eval_res, eval_res_type) = std::move(constant_expression_result.value());
|
||||
}
|
||||
|
||||
/// UInt8, maybe Nullable, maybe LowCardinality, and NULL are allowed
|
||||
eval_res_type = removeNullable(removeLowCardinality(eval_res_type));
|
||||
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.
|
||||
/// 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.
|
||||
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
|
||||
required_source_columns = source_columns;
|
||||
@ -1178,6 +1176,8 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
|
||||
ss << " '" << name << "'";
|
||||
}
|
||||
|
||||
if (no_throw)
|
||||
return false;
|
||||
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);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const
|
||||
@ -1395,7 +1396,9 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
||||
else
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -87,7 +87,7 @@ struct TreeRewriterResult
|
||||
bool add_special = true);
|
||||
|
||||
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(); }
|
||||
const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; }
|
||||
NameSet getArrayJoinSourceNameSet() const;
|
||||
@ -108,7 +108,10 @@ using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
|
||||
class TreeRewriter : WithContext
|
||||
{
|
||||
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
|
||||
TreeRewriterResultPtr analyze(
|
||||
@ -132,6 +135,9 @@ public:
|
||||
|
||||
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);
|
||||
|
||||
/// Do not throw exception from analyze on unknown identifiers, but only return nullptr.
|
||||
bool no_throw = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
||||
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);
|
||||
/// 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};
|
||||
}
|
||||
|
||||
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>())
|
||||
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)
|
||||
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.
|
||||
/// 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);
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
|
@ -17,13 +17,16 @@ class IDataType;
|
||||
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
using EvaluateConstantExpressionResult = std::pair<Field, std::shared_ptr<const IDataType>>;
|
||||
|
||||
/** Evaluate constant expression and its type.
|
||||
* Used in rare cases - for elements of set for IN, for data to INSERT.
|
||||
* Throws exception if it's not a constant expression.
|
||||
* 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.
|
||||
*/
|
||||
|
@ -720,6 +720,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
case TypeIndex::Decimal256:
|
||||
return readColumnWithBigNumberFromBinaryData<ColumnDecimal<Decimal256>>(arrow_column, column_name, type_hint);
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
return readColumnWithStringData<arrow::BinaryArray>(arrow_column, column_name);
|
||||
@ -739,6 +740,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
case TypeIndex::UInt256:
|
||||
return readColumnWithBigIntegerFromFixedBinaryData<UInt256>(arrow_column, column_name, type_hint);
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -245,6 +245,7 @@ static void insertString(IColumn & column, DataTypePtr type, const char * value,
|
||||
insertFromBinaryRepresentation<ColumnDecimal<Decimal256>>(column, type, value, size);
|
||||
return;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -44,11 +44,7 @@ std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t shard_num,
|
||||
size_t shard_count,
|
||||
size_t replica_num,
|
||||
size_t replica_count,
|
||||
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator,
|
||||
UUID group_uuid)
|
||||
size_t shard_count)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
@ -67,26 +63,6 @@ std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
.setShardInfo(static_cast<UInt32>(shard_num), static_cast<UInt32>(shard_count))
|
||||
.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)
|
||||
{
|
||||
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options);
|
||||
|
@ -19,10 +19,5 @@ std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t shard_num,
|
||||
size_t shard_count,
|
||||
size_t replica_num,
|
||||
size_t replica_count,
|
||||
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator,
|
||||
UUID group_uuid = UUIDHelpers::Nil);
|
||||
|
||||
size_t shard_count);
|
||||
}
|
||||
|
@ -1226,6 +1226,7 @@ static void buildIndexes(
|
||||
std::optional<ReadFromMergeTree::Indexes> & indexes,
|
||||
ActionsDAGPtr filter_actions_dag,
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const ContextPtr & context,
|
||||
const SelectQueryInfo & query_info,
|
||||
const StorageMetadataPtr & metadata_snapshot)
|
||||
@ -1248,7 +1249,7 @@ static void buildIndexes(
|
||||
context,
|
||||
primary_key_column_names,
|
||||
primary_key.expression,
|
||||
array_join_name_set}, {}, {}, {}, false});
|
||||
array_join_name_set}, {}, {}, {}, false, {}});
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1256,7 +1257,7 @@ static void buildIndexes(
|
||||
query_info,
|
||||
context,
|
||||
primary_key_column_names,
|
||||
primary_key.expression}, {}, {}, {}, false});
|
||||
primary_key.expression}, {}, {}, {}, false, {}});
|
||||
}
|
||||
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
@ -1269,6 +1270,9 @@ static void buildIndexes(
|
||||
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;
|
||||
bool final = query_info.isFinal();
|
||||
|
||||
@ -1346,7 +1350,7 @@ static void buildIndexes(
|
||||
void ReadFromMergeTree::applyFilters()
|
||||
{
|
||||
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(
|
||||
@ -1424,11 +1428,6 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
|
||||
|
||||
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;
|
||||
|
||||
/// 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;
|
||||
|
||||
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())
|
||||
{
|
||||
@ -1467,7 +1469,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
|
||||
indexes->minmax_idx_condition,
|
||||
parts,
|
||||
alter_conversions,
|
||||
part_values,
|
||||
indexes->part_values,
|
||||
metadata_snapshot_base,
|
||||
data,
|
||||
context,
|
||||
|
@ -171,6 +171,7 @@ public:
|
||||
std::optional<KeyCondition> minmax_idx_condition;
|
||||
UsefulSkipIndexes skip_indexes;
|
||||
bool use_skip_indexes;
|
||||
std::optional<std::unordered_set<String>> part_values;
|
||||
};
|
||||
|
||||
static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(
|
||||
|
@ -187,7 +187,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream
|
||||
if (try_results.empty() || local_delay < max_remote_delay)
|
||||
{
|
||||
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(
|
||||
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: {}",
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -57,7 +57,7 @@ private:
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits;
|
||||
Poco::Logger * log;
|
||||
UInt32 shard_count;
|
||||
String cluster_name;
|
||||
const String cluster_name;
|
||||
|
||||
void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
|
||||
void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
|
||||
|
@ -595,7 +595,7 @@ void HTTPHandler::processQuery(
|
||||
size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE;
|
||||
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>(
|
||||
response,
|
||||
|
@ -88,7 +88,7 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe
|
||||
|
||||
Output used_output;
|
||||
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>(
|
||||
response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout);
|
||||
|
||||
|
@ -18,7 +18,7 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
|
||||
try
|
||||
{
|
||||
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);
|
||||
|
||||
|
@ -79,7 +79,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
|
@ -90,7 +90,7 @@ static inline void trySendExceptionToClient(
|
||||
|
||||
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);
|
||||
|
||||
try
|
||||
|
@ -30,7 +30,7 @@ WebUIRequestHandler::WebUIRequestHandler(IServer & server_)
|
||||
|
||||
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");
|
||||
|
||||
|
@ -57,7 +57,7 @@ LocalFileHolder::~LocalFileHolder()
|
||||
{
|
||||
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);
|
||||
}
|
||||
}
|
||||
@ -122,7 +122,7 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence)
|
||||
{
|
||||
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(
|
||||
local_file_holder->original_readbuffer->buffer().begin(),
|
||||
local_file_holder->original_readbuffer->buffer().size(),
|
||||
@ -147,7 +147,7 @@ off_t RemoteReadBuffer::getPosition()
|
||||
{
|
||||
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();
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
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()
|
||||
{
|
||||
calculateColumnsSizesOnDisk();
|
||||
|
@ -489,6 +489,12 @@ public:
|
||||
|
||||
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.
|
||||
/// TODO: remove this method after some time.
|
||||
void removeDeleteOnDestroyMarker();
|
||||
@ -534,7 +540,6 @@ protected:
|
||||
|
||||
void removeIfNeeded();
|
||||
|
||||
virtual void checkConsistency(bool require_part_metadata) const;
|
||||
void checkConsistencyBase() const;
|
||||
|
||||
/// Fill each_columns_size and total_size with sizes from columns files on
|
||||
|
@ -1244,32 +1244,13 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart(
|
||||
.withPartFormatFromDisk()
|
||||
.build();
|
||||
}
|
||||
catch (const Exception & e)
|
||||
catch (...)
|
||||
{
|
||||
/// Don't count the part as broken if there was a retryalbe error
|
||||
/// during loading, such as "not enough memory" or network error.
|
||||
if (isRetryableException(e))
|
||||
if (isRetryableException(std::current_exception()))
|
||||
throw;
|
||||
|
||||
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 (...)
|
||||
{
|
||||
LOG_DEBUG(log, "Failed to load data part {}, unknown exception", part_name);
|
||||
mark_broken();
|
||||
return res;
|
||||
}
|
||||
@ -1294,18 +1275,12 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart(
|
||||
{
|
||||
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
|
||||
/// during loading, such as "not enough memory" or network error.
|
||||
if (isRetryableException(e))
|
||||
if (isRetryableException(std::current_exception()))
|
||||
throw;
|
||||
|
||||
mark_broken();
|
||||
return res;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
mark_broken();
|
||||
return res;
|
||||
}
|
||||
@ -1416,11 +1391,28 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPartWithRetries(
|
||||
size_t max_backoff_ms,
|
||||
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)
|
||||
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",
|
||||
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);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
catch (...)
|
||||
{
|
||||
if (isRetryableException(e))
|
||||
handle_exception(e.message(),try_no);
|
||||
if (isRetryableException(std::current_exception()))
|
||||
handle_exception(std::current_exception(),try_no);
|
||||
else
|
||||
throw;
|
||||
}
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
catch (const Azure::Core::Http::TransportException & e)
|
||||
{
|
||||
handle_exception(e.Message,try_no);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
@ -4636,7 +4622,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy, DataPar
|
||||
|
||||
/// Move parts are non replicated operations, so we take lock here.
|
||||
/// All other locks are taken in StorageReplicatedMergeTree
|
||||
lockSharedData(*part_copy);
|
||||
lockSharedData(*part_copy, /* replace_existing_lock */ true);
|
||||
|
||||
return;
|
||||
}
|
||||
@ -5255,6 +5241,9 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts(
|
||||
if (hold_table_lock && !table_lock)
|
||||
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;
|
||||
part->getDataPartStorage().backup(
|
||||
part->checksums,
|
||||
@ -5315,8 +5304,8 @@ void MergeTreeData::restoreDataFromBackup(RestorerFromBackup & restorer, const S
|
||||
class MergeTreeData::RestoredPartsHolder
|
||||
{
|
||||
public:
|
||||
RestoredPartsHolder(const std::shared_ptr<MergeTreeData> & storage_, const BackupPtr & backup_, size_t num_parts_)
|
||||
: storage(storage_), backup(backup_), num_parts(num_parts_)
|
||||
RestoredPartsHolder(const std::shared_ptr<MergeTreeData> & storage_, const BackupPtr & backup_)
|
||||
: storage(storage_), backup(backup_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -5329,6 +5318,13 @@ public:
|
||||
attachIfAllPartsRestored();
|
||||
}
|
||||
|
||||
void increaseNumBrokenParts()
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
++num_broken_parts;
|
||||
attachIfAllPartsRestored();
|
||||
}
|
||||
|
||||
void addPart(MutableDataPartPtr part)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
@ -5348,7 +5344,7 @@ public:
|
||||
private:
|
||||
void attachIfAllPartsRestored()
|
||||
{
|
||||
if (!num_parts || (parts.size() < num_parts))
|
||||
if (!num_parts || (parts.size() + num_broken_parts < num_parts))
|
||||
return;
|
||||
|
||||
/// Sort parts by min_block (because we need to preserve the order of parts).
|
||||
@ -5363,9 +5359,10 @@ private:
|
||||
num_parts = 0;
|
||||
}
|
||||
|
||||
std::shared_ptr<MergeTreeData> storage;
|
||||
BackupPtr backup;
|
||||
const std::shared_ptr<MergeTreeData> storage;
|
||||
const BackupPtr backup;
|
||||
size_t num_parts = 0;
|
||||
size_t num_broken_parts = 0;
|
||||
MutableDataPartsVector parts;
|
||||
std::map<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>> temp_dirs;
|
||||
mutable std::mutex mutex;
|
||||
@ -5381,8 +5378,9 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
|
||||
Strings part_names = backup->listFiles(data_path_in_backup);
|
||||
boost::remove_erase(part_names, "mutations");
|
||||
|
||||
auto restored_parts_holder
|
||||
= std::make_shared<RestoredPartsHolder>(std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup, part_names.size());
|
||||
bool restore_broken_parts_as_detached = restorer.getRestoreSettings().restore_broken_parts_as_detached;
|
||||
|
||||
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;
|
||||
size_t num_parts = 0;
|
||||
@ -5404,8 +5402,9 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
|
||||
backup,
|
||||
part_path_in_backup = data_path_in_backup_fs / part_name,
|
||||
my_part_info = *part_info,
|
||||
restore_broken_parts_as_detached,
|
||||
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;
|
||||
}
|
||||
@ -5413,11 +5412,12 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
|
||||
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);
|
||||
auto backup = restored_parts_holder->getBackup();
|
||||
|
||||
/// Calculate the total size of the part.
|
||||
UInt64 total_size_of_part = 0;
|
||||
Strings filenames = backup->listFiles(part_path_in_backup, /* recursive= */ true);
|
||||
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);
|
||||
|
||||
std::shared_ptr<IReservation> reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part);
|
||||
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();
|
||||
disk->createDirectories(temp_part_dir);
|
||||
|
||||
/// For example:
|
||||
/// Calculate paths, for example:
|
||||
/// part_name = 0_1_1_0
|
||||
/// part_path_in_backup = /data/test/table/0_1_1_0
|
||||
/// tmp_dir = tmp/1aaaaaa
|
||||
/// 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.
|
||||
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)
|
||||
{
|
||||
/// 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);
|
||||
}
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
|
||||
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);
|
||||
if (auto part = loadPartRestoredFromBackup(disk, temp_part_dir.parent_path(), part_name, detach_if_broken))
|
||||
restored_parts_holder->addPart(part);
|
||||
else
|
||||
restored_parts_holder->increaseNumBrokenParts();
|
||||
}
|
||||
|
||||
restored_parts_holder->addPart(part);
|
||||
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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -1357,7 +1357,8 @@ protected:
|
||||
|
||||
/// Restores the parts of this table from backup.
|
||||
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.
|
||||
virtual void attachRestoredParts(MutableDataPartsVector && parts) = 0;
|
||||
|
@ -43,6 +43,7 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
|
||||
@ -772,6 +773,37 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling(
|
||||
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(
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
|
@ -170,6 +170,12 @@ public:
|
||||
const ASTPtr & query,
|
||||
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.
|
||||
static void filterPartsByPartition(
|
||||
std::optional<PartitionPruner> & partition_pruner,
|
||||
|
@ -163,7 +163,8 @@ struct DetachedPartInfo : public MergeTreePartInfo
|
||||
"tmp-fetch",
|
||||
"covered-by-broken",
|
||||
"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>({
|
||||
@ -175,7 +176,8 @@ struct DetachedPartInfo : public MergeTreePartInfo
|
||||
"deleting",
|
||||
"clone",
|
||||
"merge-not-byte-identical",
|
||||
"mutate-not-byte-identical"
|
||||
"mutate-not-byte-identical",
|
||||
"broken-from-backup",
|
||||
});
|
||||
|
||||
/// NOTE: It may parse part info incorrectly.
|
||||
|
@ -112,15 +112,10 @@ void MergeTreeReaderCompact::initialize()
|
||||
compressed_data_buffer = non_cached_buffer.get();
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (!isRetryableException(e))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
data_part_info_for_read->reportBroken();
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -239,18 +234,21 @@ size_t MergeTreeReaderCompact::readRows(
|
||||
"Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.",
|
||||
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();
|
||||
|
||||
/// Better diagnostics.
|
||||
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
data_part_info_for_read->reportBroken();
|
||||
try
|
||||
{
|
||||
rethrow_exception(std::current_exception());
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -396,15 +394,10 @@ try
|
||||
seekToMark(all_mark_ranges.front().begin, 0);
|
||||
data_buffer->prefetch(priority);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (!isRetryableException(e))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
data_part_info_for_read->reportBroken();
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
|
||||
|
@ -47,15 +47,10 @@ MergeTreeReaderWide::MergeTreeReaderWide(
|
||||
for (size_t i = 0; i < columns_to_read.size(); ++i)
|
||||
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 (...)
|
||||
{
|
||||
data_part_info_for_read->reportBroken();
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -78,15 +73,10 @@ void MergeTreeReaderWide::prefetchBeginOfRange(Priority priority)
|
||||
/// of range only once so there is no such problem.
|
||||
/// 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 (...)
|
||||
{
|
||||
data_part_info_for_read->reportBroken();
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -184,18 +174,21 @@ size_t MergeTreeReaderWide::readRows(
|
||||
/// 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.
|
||||
}
|
||||
catch (Exception & e)
|
||||
catch (...)
|
||||
{
|
||||
if (!isRetryableException(e))
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
|
||||
/// Better diagnostics.
|
||||
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
data_part_info_for_read->reportBroken();
|
||||
try
|
||||
{
|
||||
rethrow_exception(std::current_exception());
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
|
@ -377,13 +377,9 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St
|
||||
result.action = ReplicatedCheckResult::DoNothing;
|
||||
return result;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
catch (...)
|
||||
{
|
||||
/// Don't count the part as broken if we got known retryable 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))
|
||||
if (isRetryableException(std::current_exception()))
|
||||
throw;
|
||||
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
@ -395,6 +391,7 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St
|
||||
result.status = {part_name, false, message};
|
||||
result.action = ReplicatedCheckResult::TryFetchMissing;
|
||||
return result;
|
||||
|
||||
}
|
||||
}
|
||||
else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < current_time)
|
||||
|
@ -15,6 +15,11 @@
|
||||
#include <IO/HashingReadBuffer.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Poco/Net/NetException.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
#include <azure/core/http/http.hpp>
|
||||
#endif
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
@ -49,19 +54,41 @@ bool isNotEnoughMemoryErrorCode(int code)
|
||||
|| code == ErrorCodes::CANNOT_MREMAP;
|
||||
}
|
||||
|
||||
bool isRetryableException(const Exception & e)
|
||||
bool isRetryableException(const std::exception_ptr exception_ptr)
|
||||
{
|
||||
if (isNotEnoughMemoryErrorCode(e.code()))
|
||||
return true;
|
||||
|
||||
if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT)
|
||||
return true;
|
||||
|
||||
try
|
||||
{
|
||||
rethrow_exception(exception_ptr);
|
||||
}
|
||||
#if USE_AWS_S3
|
||||
const auto * s3_exception = dynamic_cast<const S3Exception *>(&e);
|
||||
if (s3_exception && s3_exception->isRetryableError())
|
||||
return true;
|
||||
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()))
|
||||
return true;
|
||||
|
||||
if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT)
|
||||
return true;
|
||||
}
|
||||
catch (const Poco::Net::NetException &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
catch (const Poco::TimeoutException &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
/// In fact, there can be other similar situations.
|
||||
/// But it is OK, because there is a safety guard against deleting too many parts.
|
||||
@ -321,15 +348,10 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
require_checksums,
|
||||
is_cancelled);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (isRetryableException(e))
|
||||
throw;
|
||||
|
||||
return drop_cache_and_check();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (isRetryableException(std::current_exception()))
|
||||
throw;
|
||||
return drop_cache_and_check();
|
||||
}
|
||||
}
|
||||
|
@ -13,6 +13,6 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
std::function<bool()> is_cancelled = []{ return false; });
|
||||
|
||||
bool isNotEnoughMemoryErrorCode(int code);
|
||||
bool isRetryableException(const Exception & e);
|
||||
bool isRetryableException(const std::exception_ptr exception_ptr);
|
||||
|
||||
}
|
||||
|
@ -520,7 +520,6 @@ namespace
|
||||
std::unique_ptr<ReadBuffer> next() override
|
||||
{
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
struct stat file_stat;
|
||||
while (true)
|
||||
{
|
||||
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];
|
||||
struct stat file_stat;
|
||||
file_stat = getFileStat(archive, false, -1, "File");
|
||||
if (file_stat.st_size == 0)
|
||||
{
|
||||
@ -554,30 +554,6 @@ namespace
|
||||
|
||||
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())
|
||||
{
|
||||
read_buf = archive_reader->readFile(archive_info.path_in_archive, false);
|
||||
@ -586,33 +562,26 @@ namespace
|
||||
continue;
|
||||
|
||||
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)
|
||||
return nullptr;
|
||||
}
|
||||
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 (getContext()->getSettingsRef().engine_file_skip_empty_files)
|
||||
{
|
||||
read_files_from_archive.clear();
|
||||
++current_archive_index;
|
||||
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);
|
||||
++current_archive_index;
|
||||
continue;
|
||||
}
|
||||
|
||||
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())
|
||||
{
|
||||
@ -625,18 +594,16 @@ namespace
|
||||
|
||||
if (!archive_reader)
|
||||
{
|
||||
read_files_from_archive.clear();
|
||||
++current_archive_index;
|
||||
continue;
|
||||
}
|
||||
|
||||
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)
|
||||
return nullptr;
|
||||
|
||||
read_files_from_archive.insert(*filename);
|
||||
read_buf = archive_reader->readFile(std::move(file_enumerator));
|
||||
}
|
||||
|
||||
@ -647,6 +614,16 @@ namespace
|
||||
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
|
||||
{
|
||||
if (!getContext()->getSettingsRef().use_cache_for_count_from_files)
|
||||
@ -657,20 +634,79 @@ namespace
|
||||
}
|
||||
|
||||
std::vector<std::string> processed_files;
|
||||
std::optional<ColumnsDescription> columns_from_cache;
|
||||
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;
|
||||
|
||||
size_t current_archive_index = 0;
|
||||
std::unordered_set<std::string> read_files_from_archive;
|
||||
|
||||
bool is_first = true;
|
||||
|
||||
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;
|
||||
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)
|
||||
@ -724,95 +760,46 @@ ColumnsDescription StorageFile::getTableStructureFromFile(
|
||||
"You must specify table structure manually", format);
|
||||
|
||||
ColumnsDescription columns;
|
||||
if (archive_info)
|
||||
std::vector<std::string> archive_paths_for_schema_cache;
|
||||
std::optional<ColumnsDescription> columns_from_cache;
|
||||
|
||||
if (context->getSettingsRef().schema_inference_use_cache_for_file)
|
||||
{
|
||||
std::vector<std::string> paths_for_schema_cache;
|
||||
std::optional<ColumnsDescription> columns_from_cache;
|
||||
|
||||
if (context->getSettingsRef().schema_inference_use_cache_for_file)
|
||||
{
|
||||
paths_for_schema_cache.reserve(archive_info->paths_to_archives.size());
|
||||
struct stat file_stat{};
|
||||
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));
|
||||
|
||||
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)
|
||||
{
|
||||
columns = std::move(*columns_from_cache);
|
||||
}
|
||||
if (archive_info)
|
||||
columns_from_cache = tryGetColumnsFromCacheForArchives(*archive_info, archive_paths_for_schema_cache, format, format_settings, context);
|
||||
else
|
||||
{
|
||||
ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context);
|
||||
try
|
||||
{
|
||||
columns = readSchemaFromFormat(
|
||||
format,
|
||||
format_settings,
|
||||
read_buffer_iterator,
|
||||
/*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(),
|
||||
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;
|
||||
}
|
||||
}
|
||||
columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context);
|
||||
}
|
||||
|
||||
for (auto & file : read_buffer_iterator.processed_files)
|
||||
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);
|
||||
if (columns_from_cache)
|
||||
{
|
||||
columns = std::move(*columns_from_cache);
|
||||
}
|
||||
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)
|
||||
if (archive_info)
|
||||
{
|
||||
columns = *columns_from_cache;
|
||||
ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context);
|
||||
columns = readSchemaFromFormat(
|
||||
format,
|
||||
format_settings,
|
||||
read_buffer_iterator,
|
||||
/*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(),
|
||||
context);
|
||||
|
||||
for (auto & file : read_buffer_iterator.processed_files)
|
||||
archive_paths_for_schema_cache.push_back(std::move(file));
|
||||
}
|
||||
else
|
||||
{
|
||||
ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context);
|
||||
columns = readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context);
|
||||
}
|
||||
|
||||
if (context->getSettingsRef().schema_inference_use_cache_for_file)
|
||||
addColumnsToCache(archive_info ? archive_info->paths_to_archives : paths, columns, format, format_settings, context);
|
||||
}
|
||||
|
||||
if (context->getSettingsRef().schema_inference_use_cache_for_file)
|
||||
addColumnsToCache(archive_info.has_value() ? archive_paths_for_schema_cache : paths, columns, format, format_settings, context);
|
||||
|
||||
return columns;
|
||||
}
|
||||
|
||||
|
@ -2511,7 +2511,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
|
||||
renameTempPartAndReplace(part_desc->res_part, transaction);
|
||||
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
|
||||
/// 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;
|
||||
for (int attempts = 5; attempts > 0; --attempts)
|
||||
{
|
||||
@ -9700,6 +9709,9 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(
|
||||
|
||||
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);
|
||||
/// Part was locked before, unfortunately it's possible during moves
|
||||
if (ops[failed_op]->getPath() == zookeeper_node)
|
||||
|
@ -490,7 +490,8 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
|
||||
continue;
|
||||
|
||||
auto & task = list_tasks[list_task_idx];
|
||||
context->getProcessListElement()->checkTimeLimit();
|
||||
if (auto elem = context->getProcessListElement())
|
||||
elem->checkTimeLimit();
|
||||
|
||||
Strings nodes = std::move(list_result.names);
|
||||
|
||||
@ -525,7 +526,8 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns)
|
||||
|
||||
auto & get_task = get_tasks[i];
|
||||
auto & list_task = list_tasks[get_task.list_task_idx];
|
||||
context->getProcessListElement()->checkTimeLimit();
|
||||
if (auto elem = context->getProcessListElement())
|
||||
elem->checkTimeLimit();
|
||||
|
||||
// Deduplication
|
||||
String key = list_task.path_part + '/' + get_task.node;
|
||||
|
@ -197,22 +197,8 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
|
||||
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())
|
||||
{
|
||||
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()))
|
||||
{
|
||||
auto plan = set_from_subquery->build(context);
|
||||
|
||||
if (!plan)
|
||||
continue;
|
||||
|
||||
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
||||
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;
|
||||
actions->execute(block_with_filter);
|
||||
|
@ -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.
|
||||
/// If `expression_ast` is passed, use it to filter block.
|
||||
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
|
||||
template <typename T>
|
||||
|
@ -160,7 +160,7 @@ def main():
|
||||
s3_helper = S3Helper()
|
||||
for f in paths:
|
||||
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:
|
||||
logging.info("Exception uploading file %s text %s", f, ex)
|
||||
paths[f] = ""
|
||||
|
@ -1,10 +1,9 @@
|
||||
#!/usr/bin/env python3
|
||||
|
||||
from pathlib import Path
|
||||
from typing import List, Tuple
|
||||
from typing import Tuple
|
||||
import subprocess
|
||||
import logging
|
||||
import json
|
||||
import os
|
||||
import sys
|
||||
import time
|
||||
@ -22,6 +21,7 @@ from env_helper import (
|
||||
)
|
||||
from git_helper import Git, git_runner
|
||||
from pr_info import PRInfo
|
||||
from report import BuildResult, FAILURE, StatusType, SUCCESS
|
||||
from s3_helper import S3Helper
|
||||
from tee_popen import TeePopen
|
||||
from version_helper import (
|
||||
@ -98,7 +98,7 @@ def get_packager_cmd(
|
||||
|
||||
def build_clickhouse(
|
||||
packager_cmd: str, logs_path: Path, build_output_path: Path
|
||||
) -> Tuple[Path, bool]:
|
||||
) -> Tuple[Path, StatusType]:
|
||||
build_log_path = logs_path / BUILD_LOG_NAME
|
||||
success = False
|
||||
with TeePopen(packager_cmd, build_log_path) as process:
|
||||
@ -118,15 +118,16 @@ def build_clickhouse(
|
||||
)
|
||||
else:
|
||||
logging.info("Build failed")
|
||||
return build_log_path, success
|
||||
return build_log_path, SUCCESS if success else FAILURE
|
||||
|
||||
|
||||
def check_for_success_run(
|
||||
s3_helper: S3Helper,
|
||||
s3_prefix: str,
|
||||
build_name: str,
|
||||
build_config: BuildConfig,
|
||||
version: ClickHouseVersion,
|
||||
) -> None:
|
||||
# TODO: Remove after S3 artifacts
|
||||
# the final empty argument is necessary for distinguish build and build_suffix
|
||||
logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix, "")
|
||||
logging.info("Checking for artifacts in %s", logged_prefix)
|
||||
@ -155,15 +156,16 @@ def check_for_success_run(
|
||||
return
|
||||
|
||||
success = len(build_urls) > 0
|
||||
create_json_artifact(
|
||||
TEMP_PATH,
|
||||
build_result = BuildResult(
|
||||
build_name,
|
||||
log_url,
|
||||
build_urls,
|
||||
build_config,
|
||||
version.describe,
|
||||
SUCCESS if success else FAILURE,
|
||||
0,
|
||||
success,
|
||||
GITHUB_JOB,
|
||||
)
|
||||
build_result.write_json(Path(TEMP_PATH))
|
||||
# Fail build job if not successeded
|
||||
if not success:
|
||||
sys.exit(1)
|
||||
@ -171,36 +173,6 @@ def check_for_success_run(
|
||||
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]:
|
||||
"Return prefixes for S3 artifacts paths"
|
||||
# FIXME performance
|
||||
@ -269,7 +241,7 @@ def main():
|
||||
|
||||
# If this is rerun, then we try to find already created artifacts and just
|
||||
# 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)
|
||||
image_version = docker_image.version
|
||||
@ -312,16 +284,17 @@ def main():
|
||||
os.makedirs(logs_path, exist_ok=True)
|
||||
|
||||
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)
|
||||
subprocess.check_call(
|
||||
f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True
|
||||
)
|
||||
logging.info("Build finished with %s, log path %s", success, log_path)
|
||||
if success:
|
||||
logging.info("Build finished as %s, log path %s", build_status, log_path)
|
||||
if build_status == SUCCESS:
|
||||
cargo_cache.upload()
|
||||
|
||||
if not success:
|
||||
else:
|
||||
# We check if docker works, because if it's down, it's infrastructure
|
||||
try:
|
||||
subprocess.check_call("docker info", shell=True)
|
||||
@ -345,7 +318,7 @@ def main():
|
||||
os.remove(performance_path)
|
||||
|
||||
build_urls = (
|
||||
s3_helper.upload_build_folder_to_s3(
|
||||
s3_helper.upload_build_directory_to_s3(
|
||||
build_output_path,
|
||||
s3_path_prefix,
|
||||
keep_dirs_in_s3_path=False,
|
||||
@ -367,8 +340,20 @@ def main():
|
||||
|
||||
print(f"::notice ::Log URL: {log_url}")
|
||||
|
||||
create_json_artifact(
|
||||
TEMP_PATH, build_name, log_url, build_urls, build_config, elapsed, success
|
||||
build_result = BuildResult(
|
||||
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)
|
||||
@ -449,7 +434,7 @@ FORMAT JSONCompactEachRow"""
|
||||
prepared_events = prepare_tests_results_for_clickhouse(
|
||||
pr_info,
|
||||
[],
|
||||
"success" if success else "failure",
|
||||
build_status,
|
||||
stopwatch.duration_seconds,
|
||||
stopwatch.start_time_str,
|
||||
log_url,
|
||||
@ -458,7 +443,7 @@ FORMAT JSONCompactEachRow"""
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
# Fail the build job if it didn't succeed
|
||||
if not success:
|
||||
if build_status != SUCCESS:
|
||||
sys.exit(1)
|
||||
|
||||
|
||||
|
@ -6,7 +6,7 @@ import os
|
||||
import sys
|
||||
import time
|
||||
from pathlib import Path
|
||||
from typing import Any, Callable, List
|
||||
from typing import Any, Callable, List, Union
|
||||
|
||||
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
|
||||
|
||||
|
||||
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 f in files:
|
||||
if build_name in f:
|
||||
|
@ -5,19 +5,25 @@ import logging
|
||||
import os
|
||||
import sys
|
||||
import atexit
|
||||
from typing import Dict, List, Tuple
|
||||
from pathlib import Path
|
||||
|
||||
from github import Github
|
||||
|
||||
from env_helper import (
|
||||
GITHUB_JOB_URL,
|
||||
GITHUB_REPOSITORY,
|
||||
GITHUB_RUN_URL,
|
||||
GITHUB_SERVER_URL,
|
||||
REPORTS_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 get_robot_token import get_best_robot_token
|
||||
from pr_info import NeedsDataType, PRInfo
|
||||
@ -34,95 +40,17 @@ from ci_config import CI_CONFIG
|
||||
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():
|
||||
logging.basicConfig(level=logging.INFO)
|
||||
temp_path = TEMP_PATH
|
||||
logging.info("Reports path %s", REPORTS_PATH)
|
||||
temp_path = Path(TEMP_PATH)
|
||||
temp_path.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
if not os.path.exists(temp_path):
|
||||
os.makedirs(temp_path)
|
||||
logging.info("Reports path %s", REPORTS_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]
|
||||
needs_data = {} # type: NeedsDataType
|
||||
@ -132,11 +60,11 @@ def main():
|
||||
needs_data = json.load(file_handler)
|
||||
required_builds = len(needs_data)
|
||||
|
||||
if needs_data and all(i["result"] == "skipped" for i in needs_data.values()):
|
||||
logging.info("All builds are skipped, exiting")
|
||||
sys.exit(0)
|
||||
|
||||
logging.info("The next builds are required: %s", ", ".join(needs_data))
|
||||
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")
|
||||
sys.exit(0)
|
||||
|
||||
gh = Github(get_best_robot_token(), per_page=100)
|
||||
pr_info = PRInfo()
|
||||
@ -153,73 +81,41 @@ def main():
|
||||
required_builds = required_builds or len(builds_for_check)
|
||||
|
||||
# Collect reports from json artifacts
|
||||
builds_report_map = {}
|
||||
for root, _, files in os.walk(REPORTS_PATH):
|
||||
for f in files:
|
||||
if f.startswith("build_urls_") and f.endswith(".json"):
|
||||
logging.info("Found build report json %s", f)
|
||||
build_name = get_build_name_from_file_name(f)
|
||||
if build_name in builds_for_check:
|
||||
with open(os.path.join(root, f), "rb") as file_handler:
|
||||
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,
|
||||
)
|
||||
build_results = []
|
||||
for build_name in builds_for_check:
|
||||
report_name = BuildResult.get_report_name(build_name).stem
|
||||
build_result = BuildResult.read_json(reports_path / report_name, build_name)
|
||||
if build_result.is_missing:
|
||||
logging.warning("Build results for %s are missing", build_name)
|
||||
continue
|
||||
build_results.append(build_result)
|
||||
|
||||
# Sort reports by config order
|
||||
build_reports = [
|
||||
builds_report_map[build_name]
|
||||
for build_name in builds_for_check
|
||||
if build_name in builds_report_map
|
||||
# The code to collect missing reports for failed jobs
|
||||
missing_job_names = [
|
||||
name
|
||||
for name in needs_data
|
||||
if not any(1 for build_result in build_results if build_result.job_name == name)
|
||||
]
|
||||
|
||||
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
|
||||
for name in needs_data
|
||||
if not any(rep for rep in build_reports if rep["job_name"] == name)
|
||||
]
|
||||
else:
|
||||
logging.info("Got exactly %s builds", len(builds_report_map))
|
||||
|
||||
# Group build artifacts by groups
|
||||
build_results = [] # type: BuildResults
|
||||
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
|
||||
)
|
||||
missing_builds = len(missing_job_names)
|
||||
for job_name in reversed(missing_job_names):
|
||||
build_result = BuildResult.missing_result("missing")
|
||||
build_result.job_name = job_name
|
||||
build_result.status = PENDING
|
||||
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_artifacts.extend(build_artifacts_url)
|
||||
build_logs.extend(build_logs_url)
|
||||
build_results.insert(0, build_result)
|
||||
|
||||
for failed_job in missing_build_names:
|
||||
_build_results, build_artifacts_url, build_logs_url = get_failed_report(
|
||||
failed_job
|
||||
)
|
||||
build_results.extend(_build_results)
|
||||
build_artifacts.extend(build_artifacts_url)
|
||||
build_logs.extend(build_logs_url)
|
||||
|
||||
total_groups = len(build_results)
|
||||
# Calculate artifact groups like packages and binaries
|
||||
total_groups = sum(len(br.grouped_urls) for br in build_results)
|
||||
ok_groups = sum(
|
||||
len(br.grouped_urls) for br in build_results if br.status == SUCCESS
|
||||
)
|
||||
logging.info("Totally got %s artifact groups", total_groups)
|
||||
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)
|
||||
|
||||
s3_helper = S3Helper()
|
||||
@ -234,17 +130,14 @@ def main():
|
||||
report = create_build_html_report(
|
||||
build_check_name,
|
||||
build_results,
|
||||
build_logs,
|
||||
build_artifacts,
|
||||
task_url,
|
||||
branch_url,
|
||||
branch_name,
|
||||
commit_url,
|
||||
)
|
||||
|
||||
report_path = os.path.join(temp_path, "report.html")
|
||||
with open(report_path, "w", encoding="utf-8") as fd:
|
||||
fd.write(report)
|
||||
report_path = temp_path / "report.html"
|
||||
report_path.write_text(report, encoding="utf-8")
|
||||
|
||||
logging.info("Going to upload prepared report")
|
||||
context_name_for_path = build_check_name.lower().replace(" ", "_")
|
||||
@ -259,27 +152,20 @@ def main():
|
||||
print(f"::notice ::Report url: {url}")
|
||||
|
||||
# Prepare a commit status
|
||||
ok_groups = 0
|
||||
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
|
||||
summary_status = get_worst_status(br.status for br in build_results)
|
||||
|
||||
# Check if there are no builds at all, do not override bad status
|
||||
if summary_status == "success":
|
||||
if some_builds_are_missing:
|
||||
summary_status = "pending"
|
||||
if summary_status == SUCCESS:
|
||||
if missing_builds:
|
||||
summary_status = PENDING
|
||||
elif ok_groups == 0:
|
||||
summary_status = "error"
|
||||
summary_status = ERROR
|
||||
|
||||
addition = ""
|
||||
if some_builds_are_missing:
|
||||
addition = f" ({len(build_reports)} of {required_builds} builds are OK)"
|
||||
if missing_builds:
|
||||
addition = (
|
||||
f" ({required_builds - missing_builds} of {required_builds} builds are OK)"
|
||||
)
|
||||
|
||||
description = format_description(
|
||||
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
|
||||
)
|
||||
|
||||
if summary_status == "error":
|
||||
if summary_status == ERROR:
|
||||
sys.exit(1)
|
||||
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
import csv
|
||||
import os
|
||||
import time
|
||||
from typing import Dict, List, Literal, Optional, Union
|
||||
from typing import Dict, List, Optional, Union
|
||||
import logging
|
||||
|
||||
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 env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL
|
||||
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 upload_result_helper import upload_results
|
||||
|
||||
@ -37,8 +46,8 @@ class RerunHelper:
|
||||
# currently we agree even for failed statuses
|
||||
for status in self.statuses:
|
||||
if self.check_name in status.context and status.state in (
|
||||
"success",
|
||||
"failure",
|
||||
SUCCESS,
|
||||
FAILURE,
|
||||
):
|
||||
return True
|
||||
return False
|
||||
@ -53,12 +62,12 @@ class RerunHelper:
|
||||
def override_status(status: str, check_name: str, invert: bool = False) -> str:
|
||||
test_config = CI_CONFIG.test_configs.get(check_name)
|
||||
if test_config and test_config.force_tests:
|
||||
return "success"
|
||||
return SUCCESS
|
||||
|
||||
if invert:
|
||||
if status == "success":
|
||||
return "error"
|
||||
return "success"
|
||||
if status == SUCCESS:
|
||||
return ERROR
|
||||
return SUCCESS
|
||||
|
||||
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
|
||||
post_commit_status(
|
||||
commit,
|
||||
"pending",
|
||||
PENDING,
|
||||
create_ci_report(pr_info, statuses),
|
||||
"The report for running CI",
|
||||
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"""
|
||||
|
||||
def beauty_state(state: str) -> str:
|
||||
if state == "success":
|
||||
if state == SUCCESS:
|
||||
return f"🟢 {state}"
|
||||
if state == "pending":
|
||||
if state == PENDING:
|
||||
return f"🟡 {state}"
|
||||
if state in ["error", "failure"]:
|
||||
if state in [ERROR, FAILURE]:
|
||||
return f"🔴 {state}"
|
||||
return state
|
||||
|
||||
@ -235,20 +244,7 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str:
|
||||
|
||||
|
||||
def get_worst_state(statuses: CommitStatuses) -> str:
|
||||
worst_status = None
|
||||
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
|
||||
return get_worst_status(status.state for status in statuses)
|
||||
|
||||
|
||||
def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str:
|
||||
@ -324,7 +320,7 @@ def format_description(description: str) -> str:
|
||||
def set_mergeable_check(
|
||||
commit: Commit,
|
||||
description: str = "",
|
||||
state: Literal["success", "failure"] = "success",
|
||||
state: StatusType = "success",
|
||||
) -> None:
|
||||
commit.create_status(
|
||||
context=MERGEABLE_NAME,
|
||||
@ -363,7 +359,7 @@ def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None
|
||||
success = []
|
||||
fail = []
|
||||
for status in required_checks:
|
||||
if status.state == "success":
|
||||
if status.state == SUCCESS:
|
||||
success.append(status.context)
|
||||
else:
|
||||
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 = format_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
|
||||
|
||||
description = ", ".join(success)
|
||||
|
@ -6,7 +6,8 @@ import time
|
||||
import subprocess
|
||||
import logging
|
||||
|
||||
from typing import List, Optional
|
||||
from pathlib import Path
|
||||
from typing import List, Optional, Union
|
||||
|
||||
|
||||
class DockerImage:
|
||||
@ -22,7 +23,7 @@ class DockerImage:
|
||||
|
||||
|
||||
def get_images_with_versions(
|
||||
reports_path: str,
|
||||
reports_path: Union[Path, str],
|
||||
required_images: List[str],
|
||||
pull: bool = True,
|
||||
version: Optional[str] = None,
|
||||
@ -80,7 +81,10 @@ def get_images_with_versions(
|
||||
|
||||
|
||||
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:
|
||||
logging.info("Looking for images file in %s", reports_path)
|
||||
return get_images_with_versions(reports_path, [image], pull, version=version)[0]
|
||||
|
@ -1,6 +1,9 @@
|
||||
#!/usr/bin/env python
|
||||
|
||||
import logging
|
||||
import os
|
||||
from os import path as p
|
||||
from typing import Tuple
|
||||
|
||||
from build_download_helper import get_gh_api
|
||||
|
||||
@ -40,13 +43,27 @@ _GITHUB_JOB_URL = ""
|
||||
def GITHUB_JOB_ID() -> str:
|
||||
global _GITHUB_JOB_ID
|
||||
global _GITHUB_JOB_URL
|
||||
if GITHUB_RUN_ID == "0":
|
||||
_GITHUB_JOB_ID = "0"
|
||||
if _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 = []
|
||||
page = 1
|
||||
while not _GITHUB_JOB_ID:
|
||||
while not job_id:
|
||||
response = get_gh_api(
|
||||
f"https://api.github.com/repos/{GITHUB_REPOSITORY}/"
|
||||
f"actions/runs/{GITHUB_RUN_ID}/jobs?per_page=100&page={page}"
|
||||
@ -55,46 +72,41 @@ def GITHUB_JOB_ID() -> str:
|
||||
data = response.json()
|
||||
jobs.extend(data["jobs"])
|
||||
for job in data["jobs"]:
|
||||
if job["name"] != GITHUB_JOB:
|
||||
if job["name"] != job_name:
|
||||
continue
|
||||
_GITHUB_JOB_ID = job["id"]
|
||||
_GITHUB_JOB_URL = job["html_url"]
|
||||
return _GITHUB_JOB_ID
|
||||
job_id = job["id"]
|
||||
job_url = job["html_url"]
|
||||
return job_id, job_url
|
||||
if (
|
||||
len(jobs) >= data["total_count"] # just in case of inconsistency
|
||||
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
|
||||
if not _GITHUB_JOB_URL:
|
||||
if not job_url:
|
||||
# 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`
|
||||
# full name, GITHUB_JOB contains only FinalJob
|
||||
# full name, job_name contains only FinalJob
|
||||
matched_jobs = []
|
||||
for job in jobs:
|
||||
nested_parts = job["name"].split(" / ")
|
||||
if len(nested_parts) <= 1:
|
||||
continue
|
||||
if nested_parts[-1] == GITHUB_JOB:
|
||||
if nested_parts[-1] == job_name:
|
||||
matched_jobs.append(job)
|
||||
if len(matched_jobs) == 1:
|
||||
# The best case scenario
|
||||
_GITHUB_JOB_ID = matched_jobs[0]["id"]
|
||||
_GITHUB_JOB_URL = matched_jobs[0]["html_url"]
|
||||
return _GITHUB_JOB_ID
|
||||
job_id = matched_jobs[0]["id"]
|
||||
job_url = matched_jobs[0]["html_url"]
|
||||
return job_id, job_url
|
||||
if matched_jobs:
|
||||
logging.error(
|
||||
"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, "
|
||||
"refer to https://github.com/actions/runner/issues/2577",
|
||||
GITHUB_JOB,
|
||||
job_name,
|
||||
)
|
||||
|
||||
return _GITHUB_JOB_ID
|
||||
|
||||
|
||||
def GITHUB_JOB_URL() -> str:
|
||||
GITHUB_JOB_ID()
|
||||
return _GITHUB_JOB_URL
|
||||
return job_id, job_url
|
||||
|
@ -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
|
||||
additional_files = []
|
||||
# Just upload all files from result_folder.
|
||||
# If task provides processed results, then it's responsible for content of
|
||||
# result_folder
|
||||
if os.path.exists(result_folder):
|
||||
if result_folder.exists():
|
||||
test_files = [
|
||||
f
|
||||
for f in os.listdir(result_folder)
|
||||
if os.path.isfile(os.path.join(result_folder, f))
|
||||
]
|
||||
additional_files = [os.path.join(result_folder, f) for f in test_files]
|
||||
f for f in result_folder.iterdir() if f.is_file()
|
||||
] # type: List[Path]
|
||||
additional_files = [f.absolute().as_posix() for f in test_files]
|
||||
|
||||
status = []
|
||||
status_path = os.path.join(result_folder, "check_status.tsv")
|
||||
if os.path.exists(status_path):
|
||||
status_path = result_folder / "check_status.tsv"
|
||||
if status_path.exists():
|
||||
logging.info("Found test_results.tsv")
|
||||
with open(status_path, "r", encoding="utf-8") as status_file:
|
||||
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]
|
||||
|
||||
try:
|
||||
results_path = Path(result_folder) / "test_results.tsv"
|
||||
results_path = result_folder / "test_results.tsv"
|
||||
test_results = read_test_results(results_path)
|
||||
if len(test_results) == 0:
|
||||
return "error", "Empty test_results.tsv", test_results, additional_files
|
||||
@ -100,10 +98,9 @@ def main():
|
||||
|
||||
stopwatch = Stopwatch()
|
||||
|
||||
temp_path = TEMP_PATH
|
||||
temp_path = Path(TEMP_PATH)
|
||||
|
||||
if not os.path.exists(temp_path):
|
||||
os.makedirs(temp_path)
|
||||
temp_path.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
pr_info = PRInfo()
|
||||
|
||||
@ -124,17 +121,14 @@ def main():
|
||||
|
||||
s3_helper = S3Helper()
|
||||
|
||||
workspace = os.path.join(temp_path, "fasttest-workspace")
|
||||
if not os.path.exists(workspace):
|
||||
os.makedirs(workspace)
|
||||
workspace = temp_path / "fasttest-workspace"
|
||||
workspace.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
output_path = os.path.join(temp_path, "fasttest-output")
|
||||
if not os.path.exists(output_path):
|
||||
os.makedirs(output_path)
|
||||
output_path = temp_path / "fasttest-output"
|
||||
output_path.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
repo_path = os.path.join(temp_path, "fasttest-repo")
|
||||
if not os.path.exists(repo_path):
|
||||
os.makedirs(repo_path)
|
||||
repo_path = temp_path / "fasttest-repo"
|
||||
repo_path.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
run_cmd = get_fasttest_cmd(
|
||||
workspace,
|
||||
@ -146,11 +140,10 @@ def main():
|
||||
)
|
||||
logging.info("Going to run fasttest with cmd %s", run_cmd)
|
||||
|
||||
logs_path = os.path.join(temp_path, "fasttest-logs")
|
||||
if not os.path.exists(logs_path):
|
||||
os.makedirs(logs_path)
|
||||
logs_path = temp_path / "fasttest-logs"
|
||||
logs_path.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
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:
|
||||
retcode = process.wait()
|
||||
if retcode == 0:
|
||||
@ -161,9 +154,7 @@ def main():
|
||||
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
|
||||
|
||||
test_output_files = os.listdir(output_path)
|
||||
additional_logs = []
|
||||
for f in test_output_files:
|
||||
additional_logs.append(os.path.join(output_path, f))
|
||||
additional_logs = [os.path.join(output_path, f) for f in test_output_files]
|
||||
|
||||
test_log_exists = (
|
||||
"test_log.txt" in test_output_files or "test_result.txt" in test_output_files
|
||||
@ -194,8 +185,8 @@ def main():
|
||||
pr_info.sha,
|
||||
"fast_tests",
|
||||
)
|
||||
build_urls = s3_helper.upload_build_folder_to_s3(
|
||||
os.path.join(output_path, "binaries"),
|
||||
build_urls = s3_helper.upload_build_directory_to_s3(
|
||||
output_path / "binaries",
|
||||
s3_path_prefix,
|
||||
keep_dirs_in_s3_path=False,
|
||||
upload_symlinks=False,
|
||||
@ -206,7 +197,7 @@ def main():
|
||||
pr_info.number,
|
||||
pr_info.sha,
|
||||
test_results,
|
||||
[run_log_path] + additional_logs,
|
||||
[run_log_path.as_posix()] + additional_logs,
|
||||
NAME,
|
||||
build_urls,
|
||||
)
|
||||
|
@ -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 get_robot_token import get_best_robot_token
|
||||
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 stopwatch import Stopwatch
|
||||
from tee_popen import TeePopen
|
||||
@ -40,10 +40,6 @@ RPM_IMAGE = "clickhouse/install-rpm-test"
|
||||
DEB_IMAGE = "clickhouse/install-deb-test"
|
||||
TEMP_PATH = Path(TEMP)
|
||||
LOGS_PATH = TEMP_PATH / "tests_logs"
|
||||
SUCCESS = "success"
|
||||
FAILURE = "failure"
|
||||
OK = "OK"
|
||||
FAIL = "FAIL"
|
||||
|
||||
|
||||
def prepare_test_scripts():
|
||||
|
@ -7,6 +7,7 @@ import json
|
||||
import subprocess
|
||||
import traceback
|
||||
import re
|
||||
from pathlib import Path
|
||||
from typing import Dict
|
||||
|
||||
from github import Github
|
||||
@ -218,15 +219,17 @@ if __name__ == "__main__":
|
||||
uploaded = {} # type: Dict[str, str]
|
||||
for name, path in paths.items():
|
||||
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:
|
||||
uploaded[name] = ""
|
||||
traceback.print_exc()
|
||||
|
||||
# Upload all images and flamegraphs to S3
|
||||
try:
|
||||
s3_helper.upload_test_folder_to_s3(
|
||||
os.path.join(result_path, "images"), s3_prefix + "images"
|
||||
s3_helper.upload_test_directory_to_s3(
|
||||
Path(result_path) / "images", s3_prefix + "images"
|
||||
)
|
||||
except Exception:
|
||||
traceback.print_exc()
|
||||
|
@ -2,17 +2,58 @@
|
||||
from ast import literal_eval
|
||||
from dataclasses import dataclass
|
||||
from pathlib import Path
|
||||
from typing import List, Optional, Tuple
|
||||
from typing import Dict, Final, Iterable, List, Literal, Optional, Tuple
|
||||
from html import escape
|
||||
import csv
|
||||
import os
|
||||
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
|
||||
|
||||
HTML_BASE_TEST_TEMPLATE = """
|
||||
HEAD_HTML_TEMPLATE = """
|
||||
<!DOCTYPE html>
|
||||
<html>
|
||||
<head>
|
||||
<style>
|
||||
|
||||
:root {{
|
||||
@ -98,15 +139,9 @@ tr:hover {{ filter: var(--tr-hover-filter); }}
|
||||
<div class="main">
|
||||
<span class="nowrap themes"><span id="toggle-dark">🌚</span><span id="toggle-light">🌞</span></span>
|
||||
<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>
|
||||
{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" />
|
||||
"""
|
||||
|
||||
FOOTER_HTML_TEMPLATE = """<img id="fish" src="https://presentations.clickhouse.com/images/fish.png" />
|
||||
<script type="text/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_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 = """
|
||||
<table>
|
||||
<tr>
|
||||
@ -238,12 +288,159 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes
|
||||
|
||||
@dataclass
|
||||
class BuildResult:
|
||||
compiler: str
|
||||
debug_build: bool
|
||||
sanitizer: str
|
||||
status: str
|
||||
build_name: str
|
||||
log_url: str
|
||||
build_urls: List[str]
|
||||
version: str
|
||||
status: StatusType
|
||||
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]
|
||||
@ -281,8 +478,8 @@ def _format_header(
|
||||
|
||||
|
||||
def _get_status_style(status: str, colortheme: Optional[ColorTheme] = None) -> str:
|
||||
ok_statuses = ("OK", "success", "PASSED")
|
||||
fail_statuses = ("FAIL", "failure", "error", "FAILED", "Timeout", "NOT_FAILED")
|
||||
ok_statuses = (OK, SUCCESS, "PASSED")
|
||||
fail_statuses = (FAIL, FAILURE, ERROR, "FAILED", "Timeout", "NOT_FAILED")
|
||||
|
||||
if colortheme is None:
|
||||
colortheme = ReportColorTheme.default
|
||||
@ -333,7 +530,7 @@ def create_test_html_report(
|
||||
additional_urls = []
|
||||
|
||||
if test_results:
|
||||
rows_part = ""
|
||||
rows_part = []
|
||||
num_fails = 0
|
||||
has_test_time = False
|
||||
has_log_urls = False
|
||||
@ -348,11 +545,13 @@ def create_test_html_report(
|
||||
if test_result.log_files is not None:
|
||||
has_log_urls = True
|
||||
|
||||
row = "<tr>"
|
||||
row = []
|
||||
has_error = test_result.status in ("FAIL", "NOT_FAILED")
|
||||
if has_error and test_result.raw_logs is not None:
|
||||
row = '<tr class="failed">'
|
||||
row += "<td>" + test_result.name + "</td>"
|
||||
row.append('<tr class="failed">')
|
||||
else:
|
||||
row.append("<tr>")
|
||||
row.append(f"<td>{test_result.name}</td>")
|
||||
colspan += 1
|
||||
style = _get_status_style(test_result.status, colortheme=statuscolors)
|
||||
|
||||
@ -362,12 +561,12 @@ def create_test_html_report(
|
||||
num_fails = num_fails + 1
|
||||
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
|
||||
|
||||
if test_result.time is not None:
|
||||
has_test_time = True
|
||||
row += f"<td>{test_result.time}</td>"
|
||||
row.append(f"<td>{test_result.time}</td>")
|
||||
colspan += 1
|
||||
|
||||
if test_result.log_urls is not None:
|
||||
@ -375,19 +574,19 @@ def create_test_html_report(
|
||||
test_logs_html = "<br>".join(
|
||||
[_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
|
||||
|
||||
row += "</tr>"
|
||||
rows_part += row
|
||||
row.append("</tr>")
|
||||
rows_part.append("".join(row))
|
||||
if test_result.raw_logs is not None:
|
||||
raw_logs = escape(test_result.raw_logs)
|
||||
row = (
|
||||
row_raw_logs = (
|
||||
'<tr class="failed-content">'
|
||||
f'<td colspan="{colspan}"><pre>{raw_logs}</pre></td>'
|
||||
"</tr>"
|
||||
)
|
||||
rows_part += row
|
||||
rows_part.append(row_raw_logs)
|
||||
|
||||
headers = BASE_HEADERS.copy()
|
||||
if has_test_time:
|
||||
@ -396,7 +595,7 @@ def create_test_html_report(
|
||||
headers.append("Logs")
|
||||
|
||||
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:
|
||||
test_part = ""
|
||||
|
||||
@ -423,31 +622,18 @@ def create_test_html_report(
|
||||
return html
|
||||
|
||||
|
||||
HTML_BASE_BUILD_TEMPLATE = """
|
||||
<!DOCTYPE html>
|
||||
<html>
|
||||
<head>
|
||||
<style>
|
||||
body {{ font-family: "DejaVu Sans", "Noto Sans", Arial, sans-serif; background: #EEE; }}
|
||||
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>
|
||||
HTML_BASE_BUILD_TEMPLATE = (
|
||||
f"{HEAD_HTML_TEMPLATE}"
|
||||
"""<p class="links">
|
||||
<a href="{commit_url}">Commit</a>
|
||||
<a href="{task_url}">Task (github actions)</a>
|
||||
</p>
|
||||
<table>
|
||||
<tr>
|
||||
<th>Config/job name</th>
|
||||
<th>Compiler</th>
|
||||
<th>Build type</th>
|
||||
<th>Version</th>
|
||||
<th>Sanitizer</th>
|
||||
<th>Status</th>
|
||||
<th>Build log</th>
|
||||
@ -457,13 +643,9 @@ tr:hover td {{filter: brightness(95%);}}
|
||||
</tr>
|
||||
{rows}
|
||||
</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>'
|
||||
|
||||
@ -471,64 +653,63 @@ LINK_TEMPLATE = '<a href="{url}">{text}</a>'
|
||||
def create_build_html_report(
|
||||
header: str,
|
||||
build_results: BuildResults,
|
||||
build_logs_urls: List[str],
|
||||
artifact_urls_list: List[List[str]],
|
||||
task_url: str,
|
||||
branch_url: str,
|
||||
branch_name: str,
|
||||
commit_url: str,
|
||||
) -> str:
|
||||
rows = ""
|
||||
for build_result, build_log_url, artifact_urls in zip(
|
||||
build_results, build_logs_urls, artifact_urls_list
|
||||
):
|
||||
row = "<tr>"
|
||||
row += f"<td>{build_result.compiler}</td>"
|
||||
if build_result.debug_build:
|
||||
row += "<td>debug</td>"
|
||||
else:
|
||||
row += "<td>relwithdebuginfo</td>"
|
||||
if build_result.sanitizer:
|
||||
row += f"<td>{build_result.sanitizer}</td>"
|
||||
else:
|
||||
row += "<td>none</td>"
|
||||
rows = []
|
||||
for build_result in build_results:
|
||||
for artifact_urls in build_result.grouped_urls:
|
||||
row = ["<tr>"]
|
||||
row.append(
|
||||
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:
|
||||
row.append("<td>debug</td>")
|
||||
else:
|
||||
row.append("<td>relwithdebuginfo</td>")
|
||||
row.append(f"<td>{build_result.version}</td>")
|
||||
if build_result.sanitizer:
|
||||
row.append(f"<td>{build_result.sanitizer}</td>")
|
||||
else:
|
||||
row.append("<td>none</td>")
|
||||
|
||||
if build_result.status:
|
||||
style = _get_status_style(build_result.status)
|
||||
row += f'<td style="{style}">{build_result.status}</td>'
|
||||
else:
|
||||
style = _get_status_style("error")
|
||||
row += f'<td style="{style}">error</td>'
|
||||
if build_result.status:
|
||||
style = _get_status_style(build_result.status)
|
||||
row.append(f'<td style="{style}">{build_result.status}</td>')
|
||||
else:
|
||||
style = _get_status_style(ERROR)
|
||||
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>')
|
||||
|
||||
if build_result.elapsed_seconds:
|
||||
delta = datetime.timedelta(seconds=build_result.elapsed_seconds)
|
||||
else:
|
||||
delta = "unknown" # type: ignore
|
||||
delta = "unknown"
|
||||
if build_result.elapsed_seconds:
|
||||
delta = str(datetime.timedelta(seconds=build_result.elapsed_seconds))
|
||||
|
||||
row += f"<td>{delta}</td>"
|
||||
row.append(f"<td>{delta}</td>")
|
||||
|
||||
links = ""
|
||||
link_separator = "<br/>"
|
||||
if artifact_urls:
|
||||
for artifact_url in artifact_urls:
|
||||
links += LINK_TEMPLATE.format(
|
||||
text=_get_html_url_name(artifact_url), url=artifact_url
|
||||
)
|
||||
links += link_separator
|
||||
if links:
|
||||
links = links[: -len(link_separator)]
|
||||
row += f"<td>{links}</td>"
|
||||
links = []
|
||||
link_separator = "<br/>"
|
||||
if artifact_urls:
|
||||
for artifact_url in artifact_urls:
|
||||
links.append(
|
||||
LINK_TEMPLATE.format(
|
||||
text=_get_html_url_name(artifact_url), url=artifact_url
|
||||
)
|
||||
)
|
||||
row.append(f"<td>{link_separator.join(links)}</td>")
|
||||
|
||||
row += f"<td>{build_result.comment}</td>"
|
||||
row.append(f"<td>{build_result.comment}</td>")
|
||||
|
||||
row += "</tr>"
|
||||
rows += row
|
||||
row.append("</tr>")
|
||||
rows.append("".join(row))
|
||||
return HTML_BASE_BUILD_TEMPLATE.format(
|
||||
title=_format_header(header, branch_name),
|
||||
header=_format_header(header, branch_name, branch_url),
|
||||
rows=rows,
|
||||
rows="".join(rows),
|
||||
task_url=task_url,
|
||||
branch_name=branch_name,
|
||||
commit_url=commit_url,
|
||||
|
@ -52,12 +52,14 @@ class S3Helper:
|
||||
self.host = S3_URL
|
||||
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(
|
||||
"Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path
|
||||
)
|
||||
metadata = {}
|
||||
if os.path.getsize(file_path) < 64 * 1024 * 1024:
|
||||
if file_path.stat().st_size < 64 * 1024 * 1024:
|
||||
if (
|
||||
s3_path.endswith("txt")
|
||||
or s3_path.endswith("log")
|
||||
@ -97,17 +99,14 @@ class S3Helper:
|
||||
if re.search(r"\.(txt|log|err|out)$", s3_path) or re.search(
|
||||
r"\.log\..*(?<!\.zst)$", s3_path
|
||||
):
|
||||
compressed_path = file_path.with_suffix(file_path.suffix + ".zst")
|
||||
logging.info(
|
||||
"Going to compress file log file %s to %s",
|
||||
file_path,
|
||||
file_path + ".zst",
|
||||
compressed_path,
|
||||
)
|
||||
# FIXME: rewrite S3 to Path
|
||||
_file_path = Path(file_path)
|
||||
compress_file_fast(
|
||||
_file_path, _file_path.with_suffix(_file_path.suffix + ".zst")
|
||||
)
|
||||
file_path += ".zst"
|
||||
compress_file_fast(file_path, compressed_path)
|
||||
file_path = compressed_path
|
||||
s3_path += ".zst"
|
||||
else:
|
||||
logging.info("Processing file without compression")
|
||||
@ -121,22 +120,20 @@ class S3Helper:
|
||||
logging.info("Upload %s to %s. Meta: %s", file_path, url, metadata)
|
||||
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:
|
||||
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:
|
||||
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(
|
||||
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]:
|
||||
all_files = []
|
||||
|
||||
@ -196,37 +193,37 @@ class S3Helper:
|
||||
logging.basicConfig(level=original_level)
|
||||
return result
|
||||
|
||||
def _upload_folder_to_s3(
|
||||
def _upload_directory_to_s3(
|
||||
self,
|
||||
folder_path,
|
||||
s3_folder_path,
|
||||
bucket_name,
|
||||
keep_dirs_in_s3_path,
|
||||
upload_symlinks,
|
||||
):
|
||||
directory_path: Path,
|
||||
s3_directory_path: str,
|
||||
bucket_name: str,
|
||||
keep_dirs_in_s3_path: bool,
|
||||
upload_symlinks: bool,
|
||||
) -> List[str]:
|
||||
logging.info(
|
||||
"Upload folder '%s' to bucket=%s of s3 folder '%s'",
|
||||
folder_path,
|
||||
"Upload directory '%s' to bucket=%s of s3 directory '%s'",
|
||||
directory_path,
|
||||
bucket_name,
|
||||
s3_folder_path,
|
||||
s3_directory_path,
|
||||
)
|
||||
if not os.path.exists(folder_path):
|
||||
if not directory_path.exists():
|
||||
return []
|
||||
files = os.listdir(folder_path)
|
||||
files = list(directory_path.iterdir())
|
||||
if not files:
|
||||
return []
|
||||
|
||||
p = Pool(min(len(files), 5))
|
||||
|
||||
def task(file_name):
|
||||
full_fs_path = os.path.join(folder_path, file_name)
|
||||
def task(file_path: Path) -> Union[str, List[str]]:
|
||||
full_fs_path = file_path.absolute()
|
||||
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:
|
||||
full_s3_path = s3_folder_path
|
||||
full_s3_path = s3_directory_path
|
||||
|
||||
if os.path.isdir(full_fs_path):
|
||||
return self._upload_folder_to_s3(
|
||||
return self._upload_directory_to_s3(
|
||||
full_fs_path,
|
||||
full_s3_path,
|
||||
bucket_name,
|
||||
@ -234,60 +231,63 @@ class S3Helper:
|
||||
upload_symlinks,
|
||||
)
|
||||
|
||||
if os.path.islink(full_fs_path):
|
||||
if full_fs_path.is_symlink():
|
||||
if upload_symlinks:
|
||||
if CI:
|
||||
return self._upload_file_to_s3(
|
||||
bucket_name, full_fs_path, full_s3_path + "/" + file_name
|
||||
)
|
||||
else:
|
||||
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 S3Helper.copy_file_to_local(
|
||||
bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
|
||||
)
|
||||
return []
|
||||
|
||||
if CI:
|
||||
return self._upload_file_to_s3(
|
||||
bucket_name, full_fs_path, full_s3_path + "/" + file_name
|
||||
)
|
||||
else:
|
||||
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 S3Helper.copy_file_to_local(
|
||||
bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
|
||||
)
|
||||
|
||||
return sorted(_flatten_list(list(p.map(task, files))))
|
||||
|
||||
def upload_build_folder_to_s3(
|
||||
def upload_build_directory_to_s3(
|
||||
self,
|
||||
folder_path,
|
||||
s3_folder_path,
|
||||
keep_dirs_in_s3_path=True,
|
||||
upload_symlinks=True,
|
||||
):
|
||||
return self._upload_folder_to_s3(
|
||||
folder_path,
|
||||
s3_folder_path,
|
||||
directory_path: Path,
|
||||
s3_directory_path: str,
|
||||
keep_dirs_in_s3_path: bool = True,
|
||||
upload_symlinks: bool = True,
|
||||
) -> List[str]:
|
||||
return self._upload_directory_to_s3(
|
||||
directory_path,
|
||||
s3_directory_path,
|
||||
S3_BUILDS_BUCKET,
|
||||
keep_dirs_in_s3_path,
|
||||
upload_symlinks,
|
||||
)
|
||||
|
||||
def upload_test_folder_to_s3(
|
||||
def upload_test_directory_to_s3(
|
||||
self,
|
||||
folder_path,
|
||||
s3_folder_path,
|
||||
keep_dirs_in_s3_path=True,
|
||||
upload_symlinks=True,
|
||||
):
|
||||
return self._upload_folder_to_s3(
|
||||
folder_path,
|
||||
s3_folder_path,
|
||||
directory_path: Path,
|
||||
s3_directory_path: str,
|
||||
keep_dirs_in_s3_path: bool = True,
|
||||
upload_symlinks: bool = True,
|
||||
) -> List[str]:
|
||||
return self._upload_directory_to_s3(
|
||||
directory_path,
|
||||
s3_directory_path,
|
||||
S3_TEST_REPORTS_BUCKET,
|
||||
keep_dirs_in_s3_path,
|
||||
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)
|
||||
result = []
|
||||
if "Contents" in objects:
|
||||
@ -296,7 +296,7 @@ class S3Helper:
|
||||
|
||||
return result
|
||||
|
||||
def exists(self, key, bucket=S3_BUILDS_BUCKET):
|
||||
def exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> bool:
|
||||
try:
|
||||
self.client.head_object(Bucket=bucket, Key=key)
|
||||
return True
|
||||
@ -304,13 +304,12 @@ class S3Helper:
|
||||
return False
|
||||
|
||||
@staticmethod
|
||||
def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str:
|
||||
local_path = os.path.abspath(
|
||||
os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path)
|
||||
)
|
||||
local_dir = os.path.dirname(local_path)
|
||||
if not os.path.exists(local_dir):
|
||||
os.makedirs(local_dir)
|
||||
def copy_file_to_local(bucket_name: str, file_path: Path, s3_path: str) -> str:
|
||||
local_path = (
|
||||
Path(RUNNER_TEMP) / "s3" / os.path.join(bucket_name, s3_path)
|
||||
).absolute()
|
||||
local_dir = local_path.parent
|
||||
local_dir.mkdir(parents=True, exist_ok=True)
|
||||
shutil.copy(file_path, local_path)
|
||||
|
||||
logging.info("Copied %s to %s", file_path, local_path)
|
||||
|
@ -4,6 +4,8 @@ import logging
|
||||
import subprocess
|
||||
import os
|
||||
import sys
|
||||
from pathlib import Path
|
||||
from typing import Dict
|
||||
|
||||
from github import Github
|
||||
|
||||
@ -47,13 +49,12 @@ def main():
|
||||
|
||||
stopwatch = Stopwatch()
|
||||
|
||||
temp_path = TEMP_PATH
|
||||
reports_path = REPORTS_PATH
|
||||
temp_path = Path(TEMP_PATH)
|
||||
reports_path = Path(REPORTS_PATH)
|
||||
|
||||
check_name = sys.argv[1]
|
||||
|
||||
if not os.path.exists(temp_path):
|
||||
os.makedirs(temp_path)
|
||||
temp_path.mkdir(parents=True, exist_ok=True)
|
||||
|
||||
pr_info = PRInfo()
|
||||
|
||||
@ -82,7 +83,7 @@ def main():
|
||||
|
||||
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):
|
||||
os.makedirs(workspace_path)
|
||||
|
||||
@ -91,7 +92,7 @@ def main():
|
||||
)
|
||||
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 subprocess.Popen(
|
||||
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}/"
|
||||
paths = {
|
||||
"run.log": run_log_path,
|
||||
"server.log.zst": os.path.join(workspace_path, "server.log.zst"),
|
||||
"server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"),
|
||||
"report.html": os.path.join(workspace_path, "report.html"),
|
||||
"test.log": os.path.join(workspace_path, "test.log"),
|
||||
"server.log.zst": workspace_path / "server.log.zst",
|
||||
"server.err.log.zst": workspace_path / "server.err.log.zst",
|
||||
"report.html": workspace_path / "report.html",
|
||||
"test.log": workspace_path / "test.log",
|
||||
}
|
||||
path_urls = {} # type: Dict[str, str]
|
||||
|
||||
s3_helper = S3Helper()
|
||||
for f in paths:
|
||||
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:
|
||||
logging.info("Exception uploading file %s text %s", f, ex)
|
||||
paths[f] = ""
|
||||
path_urls[f] = ""
|
||||
|
||||
report_url = GITHUB_RUN_URL
|
||||
if paths["report.html"]:
|
||||
report_url = paths["report.html"]
|
||||
if path_urls["report.html"]:
|
||||
report_url = path_urls["report.html"]
|
||||
|
||||
status = "success"
|
||||
description = "See the report"
|
||||
|
@ -34,7 +34,7 @@ def process_logs(
|
||||
test_result.log_urls.append(processed_logs[path])
|
||||
elif path:
|
||||
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)
|
||||
processed_logs[path] = url
|
||||
@ -44,7 +44,7 @@ def process_logs(
|
||||
if log_path:
|
||||
additional_urls.append(
|
||||
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,
|
||||
statuscolors=statuscolors,
|
||||
)
|
||||
with open("report.html", "w", encoding="utf-8") as f:
|
||||
f.write(html_report)
|
||||
report_path = Path("report.html")
|
||||
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)
|
||||
return url
|
||||
|
@ -245,8 +245,10 @@ def get_version_from_string(
|
||||
|
||||
def get_version_from_tag(tag: str) -> ClickHouseVersion:
|
||||
Git.check_tag(tag)
|
||||
tag = tag[1:].split("-")[0]
|
||||
return get_version_from_string(tag)
|
||||
tag, description = tag[1:].split("-", 1)
|
||||
version = get_version_from_string(tag)
|
||||
version.with_description(description)
|
||||
return version
|
||||
|
||||
|
||||
def version_arg(version: str) -> ClickHouseVersion:
|
||||
|
@ -1,6 +1,13 @@
|
||||
<clickhouse>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<backups>
|
||||
<type>local</type>
|
||||
<path>/var/lib/clickhouse/disks/backups/</path>
|
||||
</backups>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
<backups>
|
||||
<allowed_disk>default</allowed_disk>
|
||||
<allowed_path>/backups</allowed_path>
|
||||
<allowed_disk>backups</allowed_disk>
|
||||
</backups>
|
||||
</clickhouse>
|
||||
|
@ -9,7 +9,7 @@ server_ip = sys.argv[2]
|
||||
mutex = threading.Lock()
|
||||
success_counter = 0
|
||||
number_of_threads = 100
|
||||
number_of_iterations = 100
|
||||
number_of_iterations = 50
|
||||
|
||||
|
||||
def perform_request():
|
||||
|
@ -10,6 +10,8 @@
|
||||
<account_name>devstoreaccount1</account_name>
|
||||
<account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key>
|
||||
<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:
|
||||
a) create it
|
||||
b) ignore if it already exists, since there are two instances, that conflicts with each other
|
||||
|
@ -0,0 +1,4 @@
|
||||
Decimal(9, 8)
|
||||
Decimal(18, 0)
|
||||
Decimal(10, 0)
|
||||
Decimal(18, 0) Decimal(10, 0)
|
@ -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;
|
@ -20,3 +20,5 @@ world
|
||||
3
|
||||
---
|
||||
4
|
||||
---
|
||||
1
|
||||
|
@ -27,3 +27,7 @@ SELECT count() FROM
|
||||
UNION ALL
|
||||
SELECT * FROM test WHERE isZeroOrNull(x != 'xyz')
|
||||
);
|
||||
|
||||
SELECT '---';
|
||||
|
||||
select isZeroOrNull(Null);
|
||||
|
@ -42,3 +42,8 @@
|
||||
2023-05-30
|
||||
2149-06-06
|
||||
1970-01-20
|
||||
Tests for issue #38585
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
|
@ -49,3 +49,10 @@ SELECT accurateCastOrNull('1xxx', 'Date');
|
||||
SELECT accurateCastOrNull('2023-05-30', 'Date');
|
||||
SELECT accurateCastOrNull('2180-01-01', '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');
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user